From 424cec708bbec8eb759b7a2a0a462198da922843 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 4 Aug 2023 19:16:20 +0200 Subject: [PATCH 01/49] feat: Project Update API on TG --- .../renku/triplesgenerator/api/Client.scala | 80 ++++ .../triplesgenerator/api/ProjectUpdates.scala | 73 ++++ .../triplesgenerator/api/ClientSpec.scala | 73 ++++ .../triplesgenerator/api/Generators.scala | 34 ++ .../api/ProjectUpdatesSpec.scala | 56 +++ triples-generator/README.md | 63 +++- .../triplesgenerator/MicroserviceRoutes.scala | 30 +- .../projects/update/Endpoint.scala | 64 ++++ .../update/ProjectExistenceChecker.scala | 72 ++++ .../projects/update/ProjectUpdater.scala | 67 ++++ .../update/UpdateQueriesCalculator.scala | 269 +++++++++++++ .../projects/update/package.scala | 23 ++ .../MicroserviceRoutesSpec.scala | 62 ++- .../projects/update/EndpointSpec.scala | 89 +++++ .../update/ProjectExistenceCheckerSpec.scala | 60 +++ .../projects/update/ProjectUpdaterSpec.scala | 81 ++++ .../update/UpdateQueriesCalculatorSpec.scala | 353 ++++++++++++++++++ 17 files changed, 1523 insertions(+), 26 deletions(-) create mode 100644 triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala create mode 100644 triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala create mode 100644 triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala create mode 100644 triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/Generators.scala create mode 100644 triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceChecker.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectUpdater.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculator.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/package.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceCheckerSpec.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectUpdaterSpec.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculatorSpec.scala diff --git a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala new file mode 100644 index 0000000000..e85c360013 --- /dev/null +++ b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala @@ -0,0 +1,80 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.api + +import cats.effect.Async +import cats.syntax.all._ +import io.renku.control.Throttler +import io.renku.graph.config.TriplesGeneratorUrl +import io.renku.graph.model.projects +import io.renku.http.client.RestClient +import io.renku.metrics.MetricsRegistry +import io.renku.triplesgenerator.api.Client.Result +import org.http4s.Uri +import org.http4s.client.dsl.Http4sClientDsl +import org.http4s.dsl.Http4sDsl +import org.typelevel.log4cats.Logger + +trait Client[F[_]] { + def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result[Unit]] +} + +object Client { + + def apply[F[_]: Async: Logger: MetricsRegistry]: F[Client[F]] = + TriplesGeneratorUrl[F]() + .map(tgUrl => new ClientImpl[F](Uri.unsafeFromString(tgUrl.value))) + + sealed trait Result[+A] { + def toEither: Either[Throwable, A] + } + + object Result { + final case class Success[+A](value: A) extends Result[A] { + def toEither: Either[Throwable, A] = Right(value) + } + + final case class Failure(error: String) extends RuntimeException(error) with Result[Nothing] { + def toEither: Either[Throwable, Nothing] = Left(this) + } + + def success[A](value: A): Result[A] = Success(value) + + def failure[A](error: String): Result[A] = Failure(error) + } +} + +private class ClientImpl[F[_]: Async: Logger](tgUri: Uri) + extends RestClient[F, Nothing](Throttler.noThrottling) + with Client[F] + with Http4sDsl[F] + with Http4sClientDsl[F] { + + import io.circe.syntax._ + import io.renku.http.tinytypes.TinyTypeURIEncoder._ + import org.http4s.circe._ + + override def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result[Unit]] = + send(PUT(tgUri / "projects" / slug) withEntity updates.asJson) { + case (Ok, _, _) => Result.success(()).pure[F] + case (NotFound, _, _) => Result.failure[Unit]("Project for update does not exist").pure[F] + case (status, req, _) => + Result.failure[Unit](s"Updating project in TG failed: ${req.pathInfo.renderString}: $status").pure[F] + } +} diff --git a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala new file mode 100644 index 0000000000..f649ab3adb --- /dev/null +++ b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala @@ -0,0 +1,73 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.api + +import io.circe.literal._ +import io.circe.syntax._ +import cats.syntax.all._ +import io.circe.{Decoder, Encoder, Json} +import io.renku.graph.model.images.ImageUri +import io.renku.graph.model.projects +import io.renku.tinytypes.json.TinyTypeDecoders._ + +final case class ProjectUpdates(newDescription: Option[Option[projects.Description]], + newImages: Option[List[ImageUri]], + newKeywords: Option[Set[projects.Keyword]], + newVisibility: Option[projects.Visibility] +) + +object ProjectUpdates { + + lazy val empty: ProjectUpdates = + ProjectUpdates(newDescription = None, newImages = None, newKeywords = None, newVisibility = None) + + implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { + case ProjectUpdates(newDescription, newImages, newKeywords, newVisibility) => + def newOptionValue[T](option: Option[Option[T]])(implicit enc: Encoder[T]) = + option.map { + case Some(v) => Json.obj("newValue" -> v.asJson) + case None => Json.obj("newValue" -> Json.Null) + } + def newValue[T](option: Option[T])(implicit enc: Encoder[T]) = + option.map(v => Json.obj("newValue" -> v.asJson)) + + json"""{ + "description": ${newOptionValue(newDescription)}, + "images": ${newValue(newImages)}, + "keywords": ${newValue(newKeywords)}, + "visibility": ${newValue(newVisibility)} + }""" dropNullValues + } + + implicit val decoder: Decoder[ProjectUpdates] = Decoder.instance { cur => + for { + newDesc <- + cur + .downField("description") + .as[Option[Json]] + .flatMap(_.map(_.hcursor.downField("newValue").as[Option[projects.Description]]).sequence) + newImages <- + cur.downField("images").downField("newValue").as[Option[List[ImageUri]]] + newKeywords <- + cur.downField("keywords").downField("newValue").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) + newVisibility <- + cur.downField("visibility").downField("newValue").as[Option[projects.Visibility]] + } yield ProjectUpdates(newDesc, newImages, newKeywords, newVisibility) + } +} diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala new file mode 100644 index 0000000000..85d585cdd7 --- /dev/null +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala @@ -0,0 +1,73 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.api + +import Generators._ +import cats.effect.IO +import com.github.tomakehurst.wiremock.client.WireMock._ +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.http.client.UrlEncoder.urlEncode +import io.renku.interpreters.TestLogger +import io.renku.stubbing.ExternalServiceStubbing +import io.renku.testtools.CustomAsyncIOSpec +import org.http4s.Status.NotFound +import org.http4s.Uri +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec +import org.typelevel.log4cats.Logger + +class ClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matchers with ExternalServiceStubbing { + + private implicit val logger: Logger[IO] = TestLogger() + private lazy val client = new ClientImpl[IO](Uri.unsafeFromString(externalServiceBaseUrl)) + + "updateProject" should { + + "succeed if sending project update to the TG's Project Update API returned Ok" in { + + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + stubFor { + put(s"/projects/${urlEncode(slug.value)}") + .withRequestBody(equalToJson(updates.asJson.spaces2)) + .willReturn(ok()) + } + + client.updateProject(slug, updates).asserting(_ shouldBe Client.Result.success(())) + } + + "failed if sending project update to the TG's Project Update API returned other status" in { + + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + stubFor { + put(s"/projects/${urlEncode(slug.value)}") + .willReturn(aResponse.withStatus(NotFound.code)) + } + + client + .updateProject(slug, updates) + .asserting(_ shouldBe Client.Result.failure("Project for update does not exist")) + } + } +} diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/Generators.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/Generators.scala new file mode 100644 index 0000000000..14e165658b --- /dev/null +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/Generators.scala @@ -0,0 +1,34 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.api + +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators._ +import org.scalacheck.Gen + +object Generators { + + val projectUpdatesGen: Gen[ProjectUpdates] = + for { + maybeNewDesc <- projectDescriptions.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewImages <- imageUris.toGeneratorOfList().toGeneratorOfOptions + maybeNewKeywords <- projectKeywords.toGeneratorOfSet(min = 0).toGeneratorOfOptions + maybeNewVisibility <- projectVisibilities.toGeneratorOfOptions + } yield ProjectUpdates(maybeNewDesc, maybeNewImages, maybeNewKeywords, maybeNewVisibility) +} diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala new file mode 100644 index 0000000000..95ad5e3a2e --- /dev/null +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala @@ -0,0 +1,56 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.api + +import Generators._ +import io.circe.Json +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.blankStrings +import org.scalatest.EitherValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should +import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks + +class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaCheckPropertyChecks with EitherValues { + + it should "encode/decode " in { + forAll(projectUpdatesGen) { values => + values.asJson.hcursor.as[ProjectUpdates].value shouldBe values + } + } + + it should "description.value = null to be considered as descriptions removals" in { + val values = projectUpdatesGen.generateOne.copy(newDescription = Some(None)) + values.asJson.deepDropNullValues + .deepMerge(Json.obj("description" -> Json.obj("value" -> null))) + .hcursor + .as[ProjectUpdates] + .value shouldBe values + } + + it should "blank descriptions to be considered as description removals" in { + val values = projectUpdatesGen.generateOne.copy(newDescription = Some(None)) + values.asJson.deepDropNullValues + .deepMerge(Json.obj("description" -> Json.obj("value" -> blankStrings().generateOne.asJson))) + .hcursor + .as[ProjectUpdates] + .value shouldBe values + } +} diff --git a/triples-generator/README.md b/triples-generator/README.md index 17d5d9c7c6..db7a96c5e0 100644 --- a/triples-generator/README.md +++ b/triples-generator/README.md @@ -4,12 +4,13 @@ This microservice deals with all Triples Store administrative and provisioning e ## API -| Method | Path | Description | -|--------|------------------|------------------------------------| -| POST | ```/events``` | To send an event for processing | -| GET | ```/metrics``` | Serves Prometheus metrics | -| GET | ```/ping``` | To check if service is healthy | -| GET | ```/version``` | Returns info about service version | +| Method | Path | Description | +|--------|-----------------------|--------------------------------------| +| POST | ```/events``` | To send an event for processing | +| GET | ```/metrics``` | Serves Prometheus metrics | +| GET | ```/ping``` | To check if service is healthy | +| PUT | ```/projects/:slug``` | API to update project data in the TS | +| GET | ```/version``` | Returns info about service version | #### POST /events @@ -278,6 +279,56 @@ Verifies service health. | OK (200) | If service is healthy | | INTERNAL SERVER ERROR (500)| Otherwise | +#### PUT /knowledge-graph/projects/:slug + +API to update project data in the Triples Store. + +Each of the properties can be either set or be null (skipping the property means it's set to null). +In the case the value is null or specified, the value won't be changed in the TS. +In case it's set (even to an empty array) this value will be used in the update. +Blank descriptions are considered as Nones. +In case no properties are set, no data will be changed in the TS. + +**Request** + +* case when there are updates for all properties + +```json +{ + "description": { + "newValue": "project description" + }, + "images": { + "newValue": ["image.png", "http://image.com/image.png"] + }, + "keywords":{ + "newValue": ["id1cb2f6f12ae50c46"] + }, + "visibility": { + "newValue": "public|internal|private" + } +} +``` + +* case when there's an update only for `description` that clears it up + +```json +{ + "description": { + "newValue": null + } +} +``` + +**Response** + +| Status | Description | +|-----------------------------|---------------------------------------| +| OK (200) | When project is updated successfully | +| BAD_REQUEST (400) | When the given payload is malformed | +| NOT_FOUND (404) | When project does not exist in the TS | +| INTERNAL SERVER ERROR (500) | In case of failures | + #### GET /version Returns info about service version diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala index 266ed5b26a..2fd8fe3c71 100644 --- a/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala @@ -23,30 +23,35 @@ import cats.effect.{Async, Resource} import cats.syntax.all._ import com.typesafe.config.Config import io.renku.events.consumers.EventConsumersRegistry +import io.renku.graph.http.server.binders.ProjectSlug import io.renku.http.server.version import io.renku.metrics.{MetricsRegistry, RoutesMetrics} import io.renku.triplesgenerator.events.EventEndpoint +import io.renku.triplesstore.SparqlQueryTimeRecorder import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger import scala.jdk.CollectionConverters._ private class MicroserviceRoutes[F[_]: MonadThrow]( - eventEndpoint: EventEndpoint[F], - routesMetrics: RoutesMetrics[F], - versionRoutes: version.Routes[F], - config: Config + eventEndpoint: EventEndpoint[F], + projectUpdateEndpoint: projects.update.Endpoint[F], + routesMetrics: RoutesMetrics[F], + versionRoutes: version.Routes[F], + config: Config ) extends Http4sDsl[F] { import eventEndpoint._ import org.http4s.HttpRoutes + import projectUpdateEndpoint.`PUT /projects/:slug` import routesMetrics._ // format: off lazy val routes: Resource[F, HttpRoutes[F]] = HttpRoutes.of[F] { - case request @ POST -> Root / "events" => processEvent(request) - case GET -> Root / "ping" => Ok("pong") - case GET -> Root / "config-info" => Ok(configInfo) + case req @ POST -> Root / "events" => processEvent(req) + case GET -> Root / "ping" => Ok("pong") + case req @ PUT -> Root / "projects" / ProjectSlug(slug) => `PUT /projects/:slug`(slug, req) + case GET -> Root / "config-info" => Ok(configInfo) }.withMetrics.map(_ <+> versionRoutes()) // format: on @@ -58,10 +63,11 @@ private class MicroserviceRoutes[F[_]: MonadThrow]( } private object MicroserviceRoutes { - def apply[F[_]: Async: Logger: MetricsRegistry](consumersRegistry: EventConsumersRegistry[F], - config: Config + def apply[F[_]: Async: Logger: MetricsRegistry: SparqlQueryTimeRecorder](consumersRegistry: EventConsumersRegistry[F], + config: Config ): F[MicroserviceRoutes[F]] = for { - eventEndpoint <- EventEndpoint(consumersRegistry) - versionRoutes <- version.Routes[F] - } yield new MicroserviceRoutes(eventEndpoint, new RoutesMetrics[F], versionRoutes, config) + eventEndpoint <- EventEndpoint(consumersRegistry) + projectUpdateEndpoint <- projects.update.Endpoint[F] + versionRoutes <- version.Routes[F] + } yield new MicroserviceRoutes(eventEndpoint, projectUpdateEndpoint, new RoutesMetrics[F], versionRoutes, config) } diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala new file mode 100644 index 0000000000..28caec1f0d --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala @@ -0,0 +1,64 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.data.EitherT +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.data.Message +import io.renku.graph.model.projects +import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesstore.SparqlQueryTimeRecorder +import org.http4s.circe._ +import org.http4s.dsl.Http4sDsl +import org.http4s.{EntityDecoder, Request, Response} +import org.typelevel.log4cats.Logger + +trait Endpoint[F[_]] { + def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] +} + +object Endpoint { + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]: F[Endpoint[F]] = for { + projectUpdater <- ProjectUpdater[F] + } yield new EndpointImpl[F](projectUpdater) +} + +private class EndpointImpl[F[_]: Async](projectUpdater: ProjectUpdater[F]) extends Http4sDsl[F] with Endpoint[F] { + + override def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] = + EitherT(decodePayload(request)) + .semiflatMap(projectUpdater.updateProject(slug, _).map(toHttpResult)) + .merge + + private def decodePayload: Request[F] => F[Either[Response[F], ProjectUpdates]] = + _.as[ProjectUpdates].map(_.asRight[Response[F]]).handleError(badRequest) + + private lazy val badRequest: Throwable => Either[Response[F], ProjectUpdates] = { _ => + Response[F](BadRequest).withEntity(Message.Error("Invalid payload")).asLeft[ProjectUpdates] + } + + private lazy val toHttpResult: ProjectUpdater.Result => Response[F] = { + case ProjectUpdater.Result.Updated => Response[F](Ok).withEntity(Message.Info("Project updated")) + case ProjectUpdater.Result.NotExists => Response[F](NotFound).withEntity(Message.Info("Project not found")) + } + + private implicit lazy val entityDecoder: EntityDecoder[F, ProjectUpdates] = jsonOf[F, ProjectUpdates] +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceChecker.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceChecker.scala new file mode 100644 index 0000000000..c79dd0da8c --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceChecker.scala @@ -0,0 +1,72 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.DecodingFailure.Reason +import io.circe.{Decoder, DecodingFailure} +import io.renku.graph.model.Schemas.{renku, schema} +import io.renku.graph.model.projects +import io.renku.triplesstore.ResultsDecoder._ +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore.client.syntax._ +import io.renku.triplesstore._ +import org.typelevel.log4cats.Logger + +private trait ProjectExistenceChecker[F[_]] { + def checkExists(slug: projects.Slug): F[Boolean] +} + +private object ProjectExistenceChecker { + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]( + connectionConfig: DatasetConnectionConfig + ): ProjectExistenceChecker[F] = + new ProjectExistenceCheckerImpl[F](TSClient[F](connectionConfig)) +} + +private class ProjectExistenceCheckerImpl[F[_]](tsClient: TSClient[F]) extends ProjectExistenceChecker[F] { + + def checkExists(slug: projects.Slug): F[Boolean] = + tsClient.queryExpecting(query(slug))(decoder(slug)) + + private def query(slug: projects.Slug) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: check project exists", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|SELECT (COUNT(DISTINCT ?id) AS ?count) + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject} + | } + |} + |""".stripMargin + ) + + private def decoder(slug: projects.Slug): Decoder[Boolean] = + ResultsDecoder.single[Boolean](implicit cur => + extract[String]("count").map(_.toInt) >>= { + case 0 => false.asRight[DecodingFailure] + case 1 => true.asRight[DecodingFailure] + case _ => DecodingFailure(Reason.CustomReason(show"Cannot determine $slug existence"), cur).asLeft[Boolean] + } + ) +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectUpdater.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectUpdater.scala new file mode 100644 index 0000000000..f7c048257d --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/ProjectUpdater.scala @@ -0,0 +1,67 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import ProjectUpdater.Result +import cats.MonadThrow +import cats.effect.Async +import cats.syntax.all._ +import io.renku.graph.model.projects +import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesstore.{ProjectsConnectionConfig, SparqlQueryTimeRecorder, TSClient} +import org.typelevel.log4cats.Logger + +private trait ProjectUpdater[F[_]] { + def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result] +} + +private object ProjectUpdater { + + sealed trait Result { + lazy val widen: Result = this + } + object Result { + final case object Updated extends Result + final case object NotExists extends Result + } + + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]: F[ProjectUpdater[F]] = for { + connectionConfig <- ProjectsConnectionConfig[F]() + projectExistenceChecker = ProjectExistenceChecker[F](connectionConfig) + updateQueriesCalculator <- UpdateQueriesCalculator[F]() + } yield new ProjectUpdaterImpl[F](projectExistenceChecker, updateQueriesCalculator, TSClient(connectionConfig)) +} + +private class ProjectUpdaterImpl[F[_]: MonadThrow](projectExistenceChecker: ProjectExistenceChecker[F], + updateQueriesCalculator: UpdateQueriesCalculator[F], + tsClient: TSClient[F] +) extends ProjectUpdater[F] { + + import updateQueriesCalculator.calculateUpdateQueries + + override def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result] = + projectExistenceChecker.checkExists(slug) >>= { + case false => + Result.NotExists.widen.pure[F] + case true => + calculateUpdateQueries(slug, updates) + .flatMap(_.traverse_(tsClient.updateWithNoResult)) + .as(Result.Updated.widen) + } +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculator.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculator.scala new file mode 100644 index 0000000000..2d49f81758 --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculator.scala @@ -0,0 +1,269 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.syntax.all._ +import cats.{Applicative, MonadThrow} +import eu.timepit.refined.auto._ +import io.renku.graph.config.RenkuUrlLoader +import io.renku.graph.model.Schemas.{rdf, renku, schema} +import io.renku.graph.model.images.{Image, ImageUri} +import io.renku.graph.model.{GraphClass, RenkuUrl, projects} +import io.renku.jsonld.syntax._ +import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesstore.SparqlQuery +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore.client.sparql.Fragment +import io.renku.triplesstore.client.syntax._ +import org.typelevel.log4cats.Logger + +private trait UpdateQueriesCalculator[F[_]] { + def calculateUpdateQueries(slug: projects.Slug, updates: ProjectUpdates): F[List[SparqlQuery]] +} + +private object UpdateQueriesCalculator { + def apply[F[_]: MonadThrow: Logger](): F[UpdateQueriesCalculator[F]] = + RenkuUrlLoader[F]().map(implicit ru => new UpdateQueriesCalculatorImpl[F]) +} + +private class UpdateQueriesCalculatorImpl[F[_]: Applicative: Logger](implicit ru: RenkuUrl) + extends UpdateQueriesCalculator[F] { + + def calculateUpdateQueries(slug: projects.Slug, updates: ProjectUpdates): F[List[SparqlQuery]] = + logUpdateStatus(slug, updates) + .map(_ => calculateUpdates(slug, updates)) + + private def calculateUpdates(slug: projects.Slug, updates: ProjectUpdates) = + ( + updates.newDescription.map(descUpdates(slug, _)) combine + updates.newKeywords.map(keywordsUpdates(slug, _)) combine + updates.newImages.map(imagesUpdates(slug, _)) combine + updates.newVisibility.map(visibilityUpdates(slug, _)) + ).getOrElse(Nil) + + private def descUpdates(slug: projects.Slug, newValue: Option[projects.Description]): List[SparqlQuery] = List( + descInProjectUpdate(slug, newValue), + descInProjectsUpdate(slug, newValue) + ) + + private def descInProjectUpdate(slug: projects.Slug, newValue: Option[projects.Description]) = + newValue match { + case Some(value) => + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update desc in Project", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ?id { ?id schema:description ?maybeDesc } } + |INSERT { GRAPH ?id { ?id schema:description ${value.asObject} } } + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:description ?maybeDesc } + | } + |}""".stripMargin + ) + case None => + SparqlQuery.ofUnsafe( + show"$reportingPrefix: delete desc in Project", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ?id { ?id schema:description ?maybeDesc } } + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:description ?maybeDesc } + | } + |}""".stripMargin + ) + } + + private def descInProjectsUpdate(slug: projects.Slug, newValue: Option[projects.Description]) = + newValue match { + case Some(value) => + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update desc in Projects", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ${GraphClass.Projects.id} { ?id schema:description ?maybeDesc } } + |INSERT { GRAPH ${GraphClass.Projects.id} { ?id schema:description ${value.asObject} } } + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:description ?maybeDesc } + | } + |}""".stripMargin + ) + case None => + SparqlQuery.ofUnsafe( + show"$reportingPrefix: delete desc in Projects", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ${GraphClass.Projects.id} { ?id schema:description ?maybeDesc } } + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:description ?maybeDesc } + | } + |}""".stripMargin + ) + } + + private def keywordsUpdates(slug: projects.Slug, newValue: Set[projects.Keyword]): List[SparqlQuery] = List( + keywordsInProjectUpdate(slug, newValue), + keywordsInProjectsUpdate(slug, newValue) + ) + + private def keywordsInProjectUpdate(slug: projects.Slug, newValue: Set[projects.Keyword]) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update keywords in Project", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ?id { ?id schema:keywords ?keyword } } + |INSERT { GRAPH ?id { + | ${newValue.map(k => fr"""?id schema:keywords ${k.asObject}.""").toList.intercalate(fr"\n ")} + |} } + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:keywords ?keyword } + | } + |}""".stripMargin + ) + + private def keywordsInProjectsUpdate(slug: projects.Slug, newValue: Set[projects.Keyword]) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update keywords in Projects", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ${GraphClass.Projects.id} { ?id schema:keywords ?keyword } } + |INSERT { GRAPH ${GraphClass.Projects.id} { + | ${newValue.map(k => fr"""?id schema:keywords ${k.asObject}.""").toList.intercalate(fr"\n")} + |} } + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:slug ${slug.asObject}. + | OPTIONAL { ?id schema:keywords ?keyword } + | } + |}""".stripMargin + ) + + private def imagesUpdates(slug: projects.Slug, newValue: List[ImageUri]): List[SparqlQuery] = { + val newImages = Image.projectImage(projects.ResourceId(slug), newValue) + List( + imagesInProjectUpdate(slug, newImages), + imagesInProjectsUpdate(slug, newImages) + ) + } + + private def imagesInProjectUpdate(slug: projects.Slug, newValue: List[Image]) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update images in Project", + Prefixes of (rdf -> "rdf", renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ?id { + | ?id schema:image ?imageId. + | ?imageId ?p ?o + |} } + |INSERT { GRAPH ?id { + | ${newValue.flatMap(toTriple).intercalate(fr"\n ")} + |} } + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject}. + | OPTIONAL { + | ?id schema:image ?imageId. + | ?imageId ?p ?o + | } + | } + |}""".stripMargin + ) + + private def imagesInProjectsUpdate(slug: projects.Slug, newValue: List[Image]) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update keywords in Projects", + Prefixes of (rdf -> "rdf", renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ${GraphClass.Projects.id} { + | ?id schema:image ?imageId. + | ?imageId ?p ?o + |} } + |INSERT { GRAPH ${GraphClass.Projects.id} { + | ${newValue.flatMap(toTriple).intercalate(fr"\n ")} + |} } + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:slug ${slug.asObject}. + | OPTIONAL { + | ?id schema:image ?imageId. + | ?imageId ?p ?o + | } + | } + |}""".stripMargin + ) + + private lazy val toTriple: Image => List[Fragment] = { case Image(resourceId, uri, position) => + List( + fr"""?id schema:image ${resourceId.asEntityId}.""", + fr"""${resourceId.asEntityId} rdf:type ${Image.Ontology.typeClass.id}.""", + fr"""${resourceId.asEntityId} ${Image.Ontology.contentUrlProperty.id} ${uri.asObject}.""", + fr"""${resourceId.asEntityId} ${Image.Ontology.positionProperty.id} ${position.asObject}.""" + ) + } + + private def visibilityUpdates(slug: projects.Slug, newValue: projects.Visibility): List[SparqlQuery] = List( + visibilityInProjectUpdate(slug, newValue), + visibilityInProjectsUpdate(slug, newValue) + ) + + private def visibilityInProjectUpdate(slug: projects.Slug, newValue: projects.Visibility) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update visibility in Project", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ?id { ?id renku:projectVisibility ?visibility } } + |INSERT { GRAPH ?id { ?id renku:projectVisibility ${newValue.asObject} } } + |WHERE { + | GRAPH ?id { + | ?id a schema:Project; + | renku:slug ${slug.asObject}; + | renku:projectVisibility ?visibility. + | } + |}""".stripMargin + ) + + private def visibilityInProjectsUpdate(slug: projects.Slug, newValue: projects.Visibility) = + SparqlQuery.ofUnsafe( + show"$reportingPrefix: update visibility in Projects", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|DELETE { GRAPH ${GraphClass.Projects.id} { ?id renku:projectVisibility ?visibility } } + |INSERT { GRAPH ${GraphClass.Projects.id} { ?id renku:projectVisibility ${newValue.asObject} } } + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:slug ${slug.asObject}; + | renku:projectVisibility ?visibility. + | } + |}""".stripMargin + ) + + private def logUpdateStatus(slug: projects.Slug, updates: ProjectUpdates) = + List( + updates.newDescription.map(v => show"description to ${v.fold("null")(vv => show"'$vv'")}"), + updates.newKeywords.map(v => show"keywords to '${v.mkString(", ")}'"), + updates.newImages.map(v => show"images to '${v.mkString(", ")}'"), + updates.newVisibility.map(v => show"visibility to '$v'") + ).flatten match { + case Nil => ().pure[F] + case updates => Logger[F].info(show"$reportingPrefix: updating ${updates.mkString(", ")} for project $slug") + } +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/package.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/package.scala new file mode 100644 index 0000000000..6ea464089d --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/package.scala @@ -0,0 +1,23 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects + +package object update { + private[update] val reportingPrefix: String = "Project Update API" +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala index 123b784756..d192c83148 100644 --- a/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala @@ -21,16 +21,23 @@ package io.renku.triplesgenerator import cats.effect.IO import cats.syntax.all._ import com.typesafe.config.ConfigFactory +import eu.timepit.refined.auto._ +import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.httpStatuses import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs import io.renku.http.server.EndpointTester._ import io.renku.http.server.version +import io.renku.http.tinytypes.TinyTypeURIEncoder._ import io.renku.interpreters.TestRoutesMetrics import io.renku.testtools.IOSpec import io.renku.triplesgenerator.events.EventEndpoint +import org.http4s.MediaType.application import org.http4s.Method.{GET, POST} import org.http4s.Status._ import org.http4s._ +import org.http4s.headers.`Content-Type` import org.http4s.implicits._ import org.scalamock.scalatest.MockFactory import org.scalatest.matchers.should @@ -40,27 +47,55 @@ import scala.language.reflectiveCalls class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory with should.Matchers { - "routes" should { + "POST /events" should { + + "return Accepted for success" in new TestCase { - "define a POST /events endpoint" in new TestCase { val request = Request[IO](POST, uri"/events") val expectedStatus = Accepted (eventEndpoint.processEvent _).expects(request).returning(Response[IO](expectedStatus).pure[IO]) - val response = routes.call(request) - - response.status shouldBe expectedStatus + routes.call(request).status shouldBe expectedStatus } + } + + "GET /ping" should { - "define a GET /ping endpoint returning OK with 'pong' body" in new TestCase { + "return Ok with 'pong' response" in new TestCase { val response = routes.call(Request(Method.GET, uri"/ping")) response.status shouldBe Ok response.body[String] shouldBe "pong" } + } + + "PUT /projects/:slug" should { + + "return Ok with message in response" in new TestCase { + + val slug = projectSlugs.generateOne + val request = Request[IO](Method.PUT, uri"/projects" / slug) + val responseInfo = Message.Info("Project updated") + + givenProjectUpdateEndpoint(slug, request, returning = Response[IO](Ok).withEntity(responseInfo)) + + val response = routes.call(request) + + response.status shouldBe Ok + response.contentType shouldBe Some(`Content-Type`(application.json)) + response.body[Message] shouldBe responseInfo + } + + "return the default response for illegal slug" in new TestCase { + routes.call(Request[IO](Method.PUT, uri"/projects" / "illegal-slug")).status shouldBe ServiceUnavailable + } + } + + "GET /metrics" should { + + "return Ok with some prometheus metrics" in new TestCase { - "define a GET /metrics endpoint returning OK with some prometheus metrics" in new TestCase { val response = routes.call(Request(Method.GET, uri"/metrics")) response.status shouldBe Ok @@ -76,9 +111,15 @@ class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory wi private trait TestCase { val eventEndpoint = mock[EventEndpoint[IO]] + val projectUpdateEndpoint = mock[projects.update.Endpoint[IO]] private val routesMetrics = TestRoutesMetrics() private val versionRoutes = mock[version.Routes[IO]] - val routes = new MicroserviceRoutes[IO](eventEndpoint, routesMetrics, versionRoutes, ConfigFactory.empty()).routes + val routes = new MicroserviceRoutes[IO](eventEndpoint, + projectUpdateEndpoint, + routesMetrics, + versionRoutes, + ConfigFactory.empty() + ).routes .map(_.or(notAvailableResponse)) val versionEndpointResponse = Response[IO](httpStatuses.generateOne) @@ -88,5 +129,10 @@ class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory wi import org.http4s.dsl.io.{GET => _, _} HttpRoutes.of[IO] { case GET -> Root / "version" => versionEndpointResponse.pure[IO] } } + + def givenProjectUpdateEndpoint(slug: model.projects.Slug, request: Request[IO], returning: Response[IO]) = + (projectUpdateEndpoint.`PUT /projects/:slug` _) + .expects(slug, request) + .returning(returning.pure[IO]) } } diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala new file mode 100644 index 0000000000..e62be5a2ff --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala @@ -0,0 +1,89 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.effect.IO +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Json +import io.circe.syntax._ +import io.renku.data.Message +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.projects +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesgenerator.api.Generators.projectUpdatesGen +import io.renku.triplesgenerator.api.ProjectUpdates +import org.http4s.MediaType.application +import org.http4s.Request +import org.http4s.Status.{BadRequest, NotFound, Ok} +import org.http4s.circe._ +import org.http4s.headers.`Content-Type` +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { + + it should "decode the payload, pass it to the project updater and return Ok for Result.Updated" in { + + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + givenProjectUpdating(slug, updates, returning = ProjectUpdater.Result.Updated.pure[IO]) + + endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => + response.status.pure[IO].asserting(_ shouldBe Ok) >> + response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> + response.as[Message].asserting(_ shouldBe Message.Info("Project updated")) + } + } + + it should "return BadRequest if payload decoding fails" in { + + val slug = projectSlugs.generateOne + val request = Request[IO]().withEntity(Json.obj("visibility" -> Json.obj("newValue" -> "invalid".asJson))) + + endpoint.`PUT /projects/:slug`(slug, request) >>= { response => + response.status.pure[IO].asserting(_ shouldBe BadRequest) >> + response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> + response.as[Message].asserting(_ shouldBe Message.Error("Invalid payload")) + } + } + + it should "return NotFound if project with the given slug does not exist" in { + + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + givenProjectUpdating(slug, updates, returning = ProjectUpdater.Result.NotExists.pure[IO]) + + endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => + response.status.pure[IO].asserting(_ shouldBe NotFound) >> + response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> + response.as[Message].asserting(_ shouldBe Message.Info("Project not found")) + } + } + + private lazy val projectUpdater = mock[ProjectUpdater[IO]] + private lazy val endpoint = new EndpointImpl[IO](projectUpdater) + + private def givenProjectUpdating(slug: projects.Slug, updates: ProjectUpdates, returning: IO[ProjectUpdater.Result]) = + (projectUpdater.updateProject _) + .expects(slug, updates) + .returning(returning) +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceCheckerSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceCheckerSpec.scala new file mode 100644 index 0000000000..535282a2b1 --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectExistenceCheckerSpec.scala @@ -0,0 +1,60 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.effect.IO +import io.renku.entities.searchgraphs.SearchInfoDatasets +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.entities +import io.renku.graph.model.testentities._ +import io.renku.interpreters.TestLogger +import io.renku.logging.TestSparqlQueryTimeRecorder +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesstore.{InMemoryJenaForSpec, ProjectsDataset, SparqlQueryTimeRecorder, TSClient} +import org.scalatest.OptionValues +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.typelevel.log4cats.Logger + +class ProjectExistenceCheckerSpec + extends AsyncFlatSpec + with CustomAsyncIOSpec + with should.Matchers + with OptionValues + with InMemoryJenaForSpec + with ProjectsDataset + with SearchInfoDatasets { + + it should "return true if project exists in the TS" in { + + val project = anyProjectEntities.generateOne.to[entities.Project] + + provisionProject(project).assertNoException >> + checker.checkExists(project.slug).asserting(_ shouldBe true) + } + + it should "return false if project does not exist in the TS" in { + checker.checkExists(projectSlugs.generateOne).asserting(_ shouldBe false) + } + + private implicit val logger: TestLogger[IO] = TestLogger[IO]() + private implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() + private lazy val checker = new ProjectExistenceCheckerImpl[IO](TSClient[IO](projectsDSConnectionInfo)) + implicit override val ioLogger: Logger[IO] = TestLogger[IO]() +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectUpdaterSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectUpdaterSpec.scala new file mode 100644 index 0000000000..5d10a59679 --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/ProjectUpdaterSpec.scala @@ -0,0 +1,81 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.syntax.all._ +import io.renku.generators.CommonGraphGenerators.sparqlQueries +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.projects +import io.renku.triplesgenerator.api.Generators.projectUpdatesGen +import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesstore.{SparqlQuery, TSClient} +import org.scalamock.scalatest.MockFactory +import org.scalatest.TryValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should + +import scala.util.Try + +class ProjectUpdaterSpec extends AnyFlatSpec with should.Matchers with TryValues with MockFactory { + + it should "check if project exists and if not return NotExists" in { + + val slug = projectSlugs.generateOne + givenProjectExistenceChecking(slug, returning = false.pure[Try]) + + val updates = projectUpdatesGen.generateOne + + updater.updateProject(slug, updates).success.value shouldBe ProjectUpdater.Result.NotExists + } + + it should "check if project exists, prepare update queries, execute them and return Updated" in { + + val slug = projectSlugs.generateOne + givenProjectExistenceChecking(slug, returning = true.pure[Try]) + + val updates = projectUpdatesGen.generateOne + val queries = sparqlQueries.generateList() + givenUpdatesCalculation(slug, updates, returning = queries.pure[Try]) + + queries foreach givenRunningQuerySucceeds + + updater.updateProject(slug, updates).success.value shouldBe ProjectUpdater.Result.Updated + } + + private lazy val projectExistenceChecker = mock[ProjectExistenceChecker[Try]] + private lazy val updateQueriesCalculator = mock[UpdateQueriesCalculator[Try]] + private lazy val tsClient = mock[TSClient[Try]] + private lazy val updater = new ProjectUpdaterImpl[Try](projectExistenceChecker, updateQueriesCalculator, tsClient) + + private def givenProjectExistenceChecking(slug: projects.Slug, returning: Try[Boolean]) = + (projectExistenceChecker.checkExists _) + .expects(slug) + .returning(returning) + + private def givenUpdatesCalculation(slug: projects.Slug, updates: ProjectUpdates, returning: Try[List[SparqlQuery]]) = + (updateQueriesCalculator.calculateUpdateQueries _) + .expects(slug, updates) + .returning(returning) + + private def givenRunningQuerySucceeds(query: SparqlQuery) = + (tsClient.updateWithNoResult _) + .expects(query) + .returning(().pure[Try]) +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculatorSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculatorSpec.scala new file mode 100644 index 0000000000..10dd3f3b02 --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/UpdateQueriesCalculatorSpec.scala @@ -0,0 +1,353 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.projects.update + +import cats.effect.IO +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.entities.searchgraphs.SearchInfoDatasets +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.images.ImageUri +import io.renku.graph.model.testentities._ +import io.renku.graph.model.{GraphClass, entities, projects} +import io.renku.interpreters.TestLogger +import io.renku.jsonld.syntax._ +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore.client.syntax._ +import io.renku.triplesstore.{InMemoryJenaForSpec, ProjectsDataset, SparqlQuery} +import org.scalacheck.Gen +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.scalatest.{OptionValues, Succeeded} +import org.typelevel.log4cats.Logger + +class UpdateQueriesCalculatorSpec + extends AsyncFlatSpec + with CustomAsyncIOSpec + with should.Matchers + with OptionValues + with InMemoryJenaForSpec + with ProjectsDataset + with SearchInfoDatasets + with AsyncMockFactory { + + it should "create upsert queries when there's a new description" in { + + val project = anyProjectEntities.generateOne.to[entities.Project] + + val newValue = projectDescriptions.generateSome + val updates = ProjectUpdates.empty.copy(newDescription = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(maybeDescription = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create delete queries when description is removed" in { + + val project = anyProjectEntities + .map(replaceProjectDesc(projectDescriptions.generateSome)) + .generateOne + .to[entities.Project] + + val newValue = Option.empty[projects.Description] + val updates = ProjectUpdates.empty.copy(newDescription = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(maybeDescription = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create upsert queries when there are new keywords" in { + + val project = anyProjectEntities + .map(replaceProjectKeywords(projectKeywords.generateSet(min = 1))) + .generateOne + .to[entities.Project] + + val newValue = projectKeywords.generateSet(min = 1) + val updates = ProjectUpdates.empty.copy(newKeywords = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(keywords = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create queries that deletes keywords on removal" in { + + val project = anyProjectEntities + .map(replaceProjectKeywords(projectKeywords.generateSet(min = 1))) + .generateOne + .to[entities.Project] + + val newValue = Set.empty[projects.Keyword] + val updates = ProjectUpdates.empty.copy(newKeywords = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(keywords = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create upsert queries when there are new images" in { + + val project = anyProjectEntities.generateOne.to[entities.Project] + + val newValue = imageUris.generateList(min = 1) + val updates = ProjectUpdates.empty.copy(newImages = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(images = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create queries that deletes images on removal" in { + + val project = anyProjectEntities + .map(replaceImages(imageUris.generateList(min = 1))) + .generateOne + .to[entities.Project] + + val newValue = List.empty[ImageUri] + val updates = ProjectUpdates.empty.copy(newImages = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(images = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create queries that inserts keywords when there were none" in { + + val project = anyProjectEntities.map(replaceProjectKeywords(Set.empty)).generateOne.to[entities.Project] + + val newValue = projectKeywords.generateSet(min = 1) + val updates = ProjectUpdates.empty.copy(newKeywords = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(keywords = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create upsert queries when there's a visibility update" in { + + val project = anyProjectEntities.generateOne.to[entities.Project] + + val newValue = Gen.oneOf(projects.Visibility.all - project.visibility).generateOne + val updates = ProjectUpdates.empty.copy(newVisibility = newValue.some) + + for { + _ <- provisionProject(project).assertNoException + + beforeUpdate = TSData(project) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, updates) >>= execute).assertNoException + + afterUpdate = beforeUpdate.copy(visibility = newValue) + _ <- dataInProjectGraph(project).asserting(_.value shouldBe afterUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe afterUpdate) + } yield Succeeded + } + + it should "create no upsert queries if there are no new values" in { + + val project = anyProjectEntities.generateOne.to[entities.Project] + val beforeUpdate = TSData(project) + + for { + _ <- provisionProject(project).assertNoException + + _ <- (updatesCalculator.calculateUpdateQueries(project.slug, ProjectUpdates.empty) >>= execute).assertNoException + + _ <- dataInProjectGraph(project).asserting(_.value shouldBe beforeUpdate) + _ <- dataInProjectsGraph(project).asserting(_.value shouldBe beforeUpdate) + } yield Succeeded + } + + private implicit val logger: TestLogger[IO] = TestLogger[IO]() + private lazy val updatesCalculator = new UpdateQueriesCalculatorImpl[IO] + + private def execute(queries: List[SparqlQuery]) = + queries.traverse_(runUpdate(on = projectsDataset, _)) + + private case class TSData(maybeDescription: Option[projects.Description], + visibility: projects.Visibility, + images: List[ImageUri], + keywords: Set[projects.Keyword] + ) + + private object TSData { + def apply(project: entities.Project): TSData = + TSData(project.maybeDescription, project.visibility, project.images.map(_.uri), project.keywords) + } + + private def dataInProjectGraph(project: entities.Project): IO[Option[TSData]] = + runSelect( + on = projectsDataset, + SparqlQuery.ofUnsafe( + "UpdateQueriesCalculator Project fetch", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|SELECT DISTINCT ?maybeDesc ?visibility + | (GROUP_CONCAT(DISTINCT ?keyword; separator=',') AS ?keywords) + | (GROUP_CONCAT(?encodedImageUrl; separator=',') AS ?images) + |WHERE { + | BIND (${GraphClass.Project.id(project.resourceId)} AS ?id) + | GRAPH ?id { + | ?id renku:slug ?slug; + | renku:projectVisibility ?visibility. + | OPTIONAL { ?id schema:description ?maybeDesc } + | OPTIONAL { ?id schema:keywords ?keyword } + | OPTIONAL { + | ?id schema:image ?imageId. + | ?imageId schema:position ?imagePosition; + | schema:contentUrl ?imageUrl. + | BIND(CONCAT(STR(?imagePosition), STR(':'), STR(?imageUrl)) AS ?encodedImageUrl) + | } + | } + |} + |GROUP BY ?maybeDesc ?visibility + |""".stripMargin + ) + ).map(toDataExtract).flatMap(toOptionOrFail) + + private def dataInProjectsGraph(project: entities.Project): IO[Option[TSData]] = + runSelect( + on = projectsDataset, + SparqlQuery.ofUnsafe( + "UpdateQueriesCalculator Projects fetch", + Prefixes of (renku -> "renku", schema -> "schema"), + sparql"""|SELECT DISTINCT ?maybeDesc ?visibility + | (GROUP_CONCAT(DISTINCT ?keyword; separator=',') AS ?keywords) + | (GROUP_CONCAT(?encodedImageUrl; separator=',') AS ?images) + |WHERE { + | BIND (${project.resourceId.asEntityId} AS ?id) + | GRAPH ${GraphClass.Projects.id} { + | ?id renku:projectVisibility ?visibility. + | OPTIONAL { ?id schema:description ?maybeDesc } + | OPTIONAL { ?id schema:keywords ?keyword } + | OPTIONAL { + | ?id schema:image ?imageId. + | ?imageId schema:position ?imagePosition; + | schema:contentUrl ?imageUrl. + | BIND(CONCAT(STR(?imagePosition), STR(':'), STR(?imageUrl)) AS ?encodedImageUrl) + | } + | } + |} + |GROUP BY ?maybeDesc ?visibility + |""".stripMargin + ) + ).map(toDataExtract).flatMap(toOptionOrFail) + + private lazy val toDataExtract: List[Map[String, String]] => List[TSData] = + _.map { row => + TSData( + row.get("maybeDesc").map(projects.Description), + projects.Visibility(row("visibility")), + toListOfImages(row.get("images")), + toSetOfKeywords(row.get("keywords")) + ) + } + + private lazy val toSetOfKeywords: Option[String] => Set[projects.Keyword] = + _.map(_.split(',').toList.map(projects.Keyword(_)).toSet).getOrElse(Set.empty) + + private lazy val toListOfImages: Option[String] => List[ImageUri] = + _.map(ImageUri.fromSplitString(',')(_).fold(throw _, identity)).getOrElse(Nil) + + private lazy val toOptionOrFail: List[TSData] => IO[Option[TSData]] = { + case Nil => Option.empty[TSData].pure[IO] + case h :: Nil => h.some.pure[IO] + case l => new Exception(s"Found ${l.size} rows but expected not more than one").raiseError[IO, Nothing] + } + + implicit override val ioLogger: Logger[IO] = TestLogger[IO]() +} From 9c126e87551595e2315fcf2d03727f97cfb780c5 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 4 Aug 2023 19:36:48 +0200 Subject: [PATCH 02/49] feat: Project Update API on KG to update TS with a call to TG --- .../projects/update/Endpoint.scala | 21 ++++++++++++----- .../projects/update/EndpointSpec.scala | 23 ++++++++++++------- 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 65f1464f25..99262cde3a 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -29,7 +29,7 @@ import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry import io.renku.triplesgenerator -import io.renku.triplesgenerator.api.events.SyncRepoMetadata +import io.renku.triplesgenerator.api.ProjectUpdates import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl import org.http4s.{Request, Response} @@ -41,19 +41,18 @@ trait Endpoint[F[_]] { object Endpoint { def apply[F[_]: Async: Logger: MetricsRegistry: GitLabClient]: F[Endpoint[F]] = - triplesgenerator.api.events.Client[F].map(new EndpointImpl(GLProjectUpdater[F], _)) + triplesgenerator.api.Client[F].map(new EndpointImpl(GLProjectUpdater[F], _)) } private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdater[F], - tgClient: triplesgenerator.api.events.Client[F] + tgClient: triplesgenerator.api.Client[F] ) extends Http4sDsl[F] with Endpoint[F] { override def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = decodePayload(request) - .flatMap(updateGL(slug, authUser)) - .semiflatMap(_ => tgClient.send(SyncRepoMetadata(slug))) - .as(Response[F](Accepted).withEntity(Message.Info("Project update accepted"))) + .flatTap(updateGL(slug, authUser)) + .flatMap(updateTG(slug)) .merge .handleErrorWith(serverError(slug)(_)) @@ -75,6 +74,16 @@ private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdat private def updateGL(slug: projects.Slug, authUser: AuthUser)(newValues: NewValues): EitherT[F, Response[F], Unit] = glProjectUpdater.updateProject(slug, newValues, authUser.accessToken).leftMap(badRequest) + private def updateTG(slug: projects.Slug)(newValues: NewValues): EitherT[F, Response[F], Response[F]] = + EitherT { + tgClient + .updateProject(slug, ProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) + .map(_.toEither) + }.biSemiflatMap( + serverError(slug), + _ => Response[F](Accepted).withEntity(Message.Info("Project update accepted")).pure[F] + ) + private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = Logger[F] .error(_)(show"Updating project $slug failed") diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index f4c36532ad..6f6533cb67 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -37,7 +37,7 @@ import io.renku.http.server.EndpointTester._ import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator -import io.renku.triplesgenerator.api.events.SyncRepoMetadata +import io.renku.triplesgenerator.api.ProjectUpdates import org.http4s.{Request, Status} import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -46,7 +46,7 @@ import org.scalatest.matchers.should class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { it should "call the GL's Edit Project API with the new values extracted from the request, " + - "send a SYNC_REPO_METADATA event to TG " + + "send project update to TG " + "and return 202 Accepted" in { val authUser = authUsers.generateOne @@ -54,7 +54,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val newValues = newValuesGen.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) - givenSyncRepoMetadataSending(slug, returning = ().pure[IO]) + givenSyncRepoMetadataSending(slug, newValues, returning = triplesgenerator.api.Client.Result.success(()).pure[IO]) endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> @@ -107,7 +107,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc } } - it should "return 500 InternalServerError if sending event failed" in { + it should "return 500 InternalServerError if updating project in TG failed" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -115,7 +115,10 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) val exception = exceptions.generateOne - givenSyncRepoMetadataSending(slug, returning = exception.raiseError[IO, Nothing]) + givenSyncRepoMetadataSending(slug, + newValues, + returning = triplesgenerator.api.Client.Result.failure(exception.getMessage).pure[IO] + ) endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.InternalServerError) >> @@ -125,7 +128,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc private implicit val logger: TestLogger[IO] = TestLogger[IO]() private val glProjectUpdater = mock[GLProjectUpdater[IO]] - private val tgClient = mock[triplesgenerator.api.events.Client[IO]] + private val tgClient = mock[triplesgenerator.api.Client[IO]] private lazy val endpoint = new EndpointImpl[IO](glProjectUpdater, tgClient) private def givenUpdatingProjectInGL(slug: projects.Slug, @@ -136,8 +139,12 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc .expects(slug, newValues, at) .returning(returning) - private def givenSyncRepoMetadataSending(slug: projects.Slug, returning: IO[Unit]) = - (tgClient.send(_: SyncRepoMetadata)).expects(SyncRepoMetadata(slug)).returning(returning) + private def givenSyncRepoMetadataSending(slug: projects.Slug, + newValues: NewValues, + returning: IO[triplesgenerator.api.Client.Result[Unit]] + ) = (tgClient.updateProject _) + .expects(slug, ProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) + .returning(returning) private implicit lazy val payloadEncoder: Encoder[NewValues] = Encoder.instance { case NewValues(visibility) => json"""{"visibility": $visibility}""" From 34f1edbb3f7b3c7053bcace7b182264ed7543ce6 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 7 Aug 2023 11:22:57 +0200 Subject: [PATCH 03/49] refactor: changes to the shape of the payload on the TS Project Update API --- .../triplesgenerator/api/ProjectUpdates.scala | 46 ++++++++----------- .../api/ProjectUpdatesSpec.scala | 39 +++++++++------- triples-generator/README.md | 44 ++++++++++-------- 3 files changed, 66 insertions(+), 63 deletions(-) diff --git a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala index f649ab3adb..21c74589b1 100644 --- a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala +++ b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/ProjectUpdates.scala @@ -18,10 +18,10 @@ package io.renku.triplesgenerator.api +import cats.syntax.all._ import io.circe.literal._ import io.circe.syntax._ -import cats.syntax.all._ -import io.circe.{Decoder, Encoder, Json} +import io.circe.{Decoder, DecodingFailure, Encoder, Json} import io.renku.graph.model.images.ImageUri import io.renku.graph.model.projects import io.renku.tinytypes.json.TinyTypeDecoders._ @@ -39,35 +39,27 @@ object ProjectUpdates { implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { case ProjectUpdates(newDescription, newImages, newKeywords, newVisibility) => - def newOptionValue[T](option: Option[Option[T]])(implicit enc: Encoder[T]) = - option.map { - case Some(v) => Json.obj("newValue" -> v.asJson) - case None => Json.obj("newValue" -> Json.Null) - } - def newValue[T](option: Option[T])(implicit enc: Encoder[T]) = - option.map(v => Json.obj("newValue" -> v.asJson)) - - json"""{ - "description": ${newOptionValue(newDescription)}, - "images": ${newValue(newImages)}, - "keywords": ${newValue(newKeywords)}, - "visibility": ${newValue(newVisibility)} - }""" dropNullValues + Json.obj( + List( + newDescription.map(v => "description" -> v.fold(Json.Null)(_.asJson)), + newImages.map(v => "images" -> v.asJson), + newKeywords.map(v => "keywords" -> v.asJson), + newVisibility.map(v => "visibility" -> v.asJson) + ).flatten: _* + ) } implicit val decoder: Decoder[ProjectUpdates] = Decoder.instance { cur => for { - newDesc <- - cur - .downField("description") - .as[Option[Json]] - .flatMap(_.map(_.hcursor.downField("newValue").as[Option[projects.Description]]).sequence) - newImages <- - cur.downField("images").downField("newValue").as[Option[List[ImageUri]]] - newKeywords <- - cur.downField("keywords").downField("newValue").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) - newVisibility <- - cur.downField("visibility").downField("newValue").as[Option[projects.Visibility]] + newDesc <- cur + .downField("description") + .success + .fold(Option.empty[Option[projects.Description]].asRight[DecodingFailure]) { + _.as[Option[projects.Description]].map(_.some) + } + newImages <- cur.downField("images").as[Option[List[ImageUri]]] + newKeywords <- cur.downField("keywords").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) + newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] } yield ProjectUpdates(newDesc, newImages, newKeywords, newVisibility) } } diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala index 95ad5e3a2e..cb4e036ead 100644 --- a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ProjectUpdatesSpec.scala @@ -19,7 +19,7 @@ package io.renku.triplesgenerator.api import Generators._ -import io.circe.Json +import io.circe.literal._ import io.circe.syntax._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.blankStrings @@ -31,26 +31,33 @@ import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaCheckPropertyChecks with EitherValues { it should "encode/decode " in { - forAll(projectUpdatesGen) { values => - values.asJson.hcursor.as[ProjectUpdates].value shouldBe values + forAll(projectUpdatesGen) { updates => + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } } - it should "description.value = null to be considered as descriptions removals" in { - val values = projectUpdatesGen.generateOne.copy(newDescription = Some(None)) - values.asJson.deepDropNullValues - .deepMerge(Json.obj("description" -> Json.obj("value" -> null))) - .hcursor - .as[ProjectUpdates] - .value shouldBe values + it should "lack of description property to be considered as no-op for the property" in { + + val updates = ProjectUpdates.empty.copy(newDescription = None) + + updates.asJson shouldBe json"""{}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + it should "description = null to be considered as descriptions removals" in { + + val updates = ProjectUpdates.empty.copy(newDescription = Some(None)) + + updates.asJson shouldBe json"""{"description": null}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } it should "blank descriptions to be considered as description removals" in { - val values = projectUpdatesGen.generateOne.copy(newDescription = Some(None)) - values.asJson.deepDropNullValues - .deepMerge(Json.obj("description" -> Json.obj("value" -> blankStrings().generateOne.asJson))) - .hcursor - .as[ProjectUpdates] - .value shouldBe values + + val json = json"""{"description": ${blankStrings().generateOne}}""" + + json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newDescription = Some(None)) } } diff --git a/triples-generator/README.md b/triples-generator/README.md index db7a96c5e0..cf949f1096 100644 --- a/triples-generator/README.md +++ b/triples-generator/README.md @@ -283,10 +283,16 @@ Verifies service health. API to update project data in the Triples Store. -Each of the properties can be either set or be null (skipping the property means it's set to null). -In the case the value is null or specified, the value won't be changed in the TS. -In case it's set (even to an empty array) this value will be used in the update. -Blank descriptions are considered as Nones. +Each of the properties can be either set to a new value or omitted in case there's no new value. + +The properties that can be updated are: +* description - possible values are: + * `null` for removing the current description + * any non-blank String value +* images - an array of either relative or absolute links to the images; an empty array removes all the images +* keywords - an array of String values; an empty array removes all the keywords +* visibility - possible values are: `public`, `internal`, `private` + In case no properties are set, no data will be changed in the TS. **Request** @@ -295,28 +301,26 @@ In case no properties are set, no data will be changed in the TS. ```json { - "description": { - "newValue": "project description" - }, - "images": { - "newValue": ["image.png", "http://image.com/image.png"] - }, - "keywords":{ - "newValue": ["id1cb2f6f12ae50c46"] - }, - "visibility": { - "newValue": "public|internal|private" - } + "description": "a new project description", + "images": ["image.png", "http://image.com/image.png"], + "keywords": ["some keyword"], + "visibility": "public|internal|private" } ``` -* case when there's an update only for `description` that clears it up +* case when there's an update only for the `description` that removes it ```json { - "description": { - "newValue": null - } + "description": null +} +``` + +* case with the `description` set to a blank String means the same as `"description": null` + +```json +{ + "description": "" } ``` From 053a410b3eb6dc0b86f35394a3f0ba6a26672764 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 7 Aug 2023 12:04:55 +0200 Subject: [PATCH 04/49] refactor: TG Client renamed to TriplesGeneratorClient for consistency --- .../knowledgegraph/projects/update/Endpoint.scala | 7 +++---- .../projects/update/EndpointSpec.scala | 11 +++++------ .../{Client.scala => TriplesGeneratorClient.scala} | 14 +++++++------- ...Spec.scala => TriplesGeneratorClientSpec.scala} | 8 ++++---- 4 files changed, 19 insertions(+), 21 deletions(-) rename triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/{Client.scala => TriplesGeneratorClient.scala} (86%) rename triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/{ClientSpec.scala => TriplesGeneratorClientSpec.scala} (83%) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 99262cde3a..262f6df077 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -28,8 +28,7 @@ import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry -import io.renku.triplesgenerator -import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesgenerator.api.{ProjectUpdates, TriplesGeneratorClient} import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl import org.http4s.{Request, Response} @@ -41,11 +40,11 @@ trait Endpoint[F[_]] { object Endpoint { def apply[F[_]: Async: Logger: MetricsRegistry: GitLabClient]: F[Endpoint[F]] = - triplesgenerator.api.Client[F].map(new EndpointImpl(GLProjectUpdater[F], _)) + TriplesGeneratorClient[F].map(new EndpointImpl(GLProjectUpdater[F], _)) } private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdater[F], - tgClient: triplesgenerator.api.Client[F] + tgClient: TriplesGeneratorClient[F] ) extends Http4sDsl[F] with Endpoint[F] { diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 6f6533cb67..7b7fd92d7a 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -36,8 +36,7 @@ import io.renku.http.client.AccessToken import io.renku.http.server.EndpointTester._ import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec -import io.renku.triplesgenerator -import io.renku.triplesgenerator.api.ProjectUpdates +import io.renku.triplesgenerator.api.{ProjectUpdates, TriplesGeneratorClient} import org.http4s.{Request, Status} import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -54,7 +53,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val newValues = newValuesGen.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) - givenSyncRepoMetadataSending(slug, newValues, returning = triplesgenerator.api.Client.Result.success(()).pure[IO]) + givenSyncRepoMetadataSending(slug, newValues, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> @@ -117,7 +116,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val exception = exceptions.generateOne givenSyncRepoMetadataSending(slug, newValues, - returning = triplesgenerator.api.Client.Result.failure(exception.getMessage).pure[IO] + returning = TriplesGeneratorClient.Result.failure(exception.getMessage).pure[IO] ) endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => @@ -128,7 +127,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc private implicit val logger: TestLogger[IO] = TestLogger[IO]() private val glProjectUpdater = mock[GLProjectUpdater[IO]] - private val tgClient = mock[triplesgenerator.api.Client[IO]] + private val tgClient = mock[TriplesGeneratorClient[IO]] private lazy val endpoint = new EndpointImpl[IO](glProjectUpdater, tgClient) private def givenUpdatingProjectInGL(slug: projects.Slug, @@ -141,7 +140,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc private def givenSyncRepoMetadataSending(slug: projects.Slug, newValues: NewValues, - returning: IO[triplesgenerator.api.Client.Result[Unit]] + returning: IO[TriplesGeneratorClient.Result[Unit]] ) = (tgClient.updateProject _) .expects(slug, ProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) .returning(returning) diff --git a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala similarity index 86% rename from triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala rename to triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala index e85c360013..9f5fd7ea6f 100644 --- a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/Client.scala +++ b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala @@ -18,6 +18,7 @@ package io.renku.triplesgenerator.api +import TriplesGeneratorClient.Result import cats.effect.Async import cats.syntax.all._ import io.renku.control.Throttler @@ -25,21 +26,20 @@ import io.renku.graph.config.TriplesGeneratorUrl import io.renku.graph.model.projects import io.renku.http.client.RestClient import io.renku.metrics.MetricsRegistry -import io.renku.triplesgenerator.api.Client.Result import org.http4s.Uri import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger -trait Client[F[_]] { +trait TriplesGeneratorClient[F[_]] { def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result[Unit]] } -object Client { +object TriplesGeneratorClient { - def apply[F[_]: Async: Logger: MetricsRegistry]: F[Client[F]] = + def apply[F[_]: Async: Logger: MetricsRegistry]: F[TriplesGeneratorClient[F]] = TriplesGeneratorUrl[F]() - .map(tgUrl => new ClientImpl[F](Uri.unsafeFromString(tgUrl.value))) + .map(tgUrl => new TriplesGeneratorClientImpl[F](Uri.unsafeFromString(tgUrl.value))) sealed trait Result[+A] { def toEither: Either[Throwable, A] @@ -60,9 +60,9 @@ object Client { } } -private class ClientImpl[F[_]: Async: Logger](tgUri: Uri) +private class TriplesGeneratorClientImpl[F[_]: Async: Logger](tgUri: Uri) extends RestClient[F, Nothing](Throttler.noThrottling) - with Client[F] + with TriplesGeneratorClient[F] with Http4sDsl[F] with Http4sClientDsl[F] { diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala similarity index 83% rename from triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala rename to triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala index 85d585cdd7..2491736e2d 100644 --- a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/ClientSpec.scala +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala @@ -34,10 +34,10 @@ import org.scalatest.matchers.should import org.scalatest.wordspec.AsyncWordSpec import org.typelevel.log4cats.Logger -class ClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matchers with ExternalServiceStubbing { +class TriplesGeneratorClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matchers with ExternalServiceStubbing { private implicit val logger: Logger[IO] = TestLogger() - private lazy val client = new ClientImpl[IO](Uri.unsafeFromString(externalServiceBaseUrl)) + private lazy val client = new TriplesGeneratorClientImpl[IO](Uri.unsafeFromString(externalServiceBaseUrl)) "updateProject" should { @@ -52,7 +52,7 @@ class ClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matche .willReturn(ok()) } - client.updateProject(slug, updates).asserting(_ shouldBe Client.Result.success(())) + client.updateProject(slug, updates).asserting(_ shouldBe TriplesGeneratorClient.Result.success(())) } "failed if sending project update to the TG's Project Update API returned other status" in { @@ -67,7 +67,7 @@ class ClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matche client .updateProject(slug, updates) - .asserting(_ shouldBe Client.Result.failure("Project for update does not exist")) + .asserting(_ shouldBe TriplesGeneratorClient.Result.failure("Project for update does not exist")) } } } From 5139056b958b157e72644d4952a5dee4b77393e0 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 9 Aug 2023 19:12:04 +0200 Subject: [PATCH 05/49] feat: renku-core-client with the version APIs --- build.sbt | 9 + .../stubbing/ExternalServiceStubbing.scala | 29 +++ renku-core-client/README.md | 3 + .../src/main/resources/application.conf | 20 ++ .../io/renku/core/client/ClientTools.scala | 55 ++++++ .../io/renku/core/client/RenkuCoreUri.scala | 70 +++++++ .../core/client/RenkuCoreVersionClient.scala | 89 +++++++++ .../scala/io/renku/core/client/Result.scala | 76 ++++++++ .../renku/core/client/SchemaApiVersions.scala | 31 ++++ .../scala/io/renku/core/client/model.scala | 29 +++ .../renku/core/client/ClientToolsSpec.scala | 62 +++++++ .../io/renku/core/client/Generators.scala | 54 ++++++ .../io/renku/core/client/ModelEncoders.scala | 51 ++++++ .../renku/core/client/RenkuCoreUriSpec.scala | 138 ++++++++++++++ .../client/RenkuCoreVersionClientSpec.scala | 173 ++++++++++++++++++ .../core/client/SchemaApiVersionsSpec.scala | 37 ++++ 16 files changed, 926 insertions(+) create mode 100644 renku-core-client/README.md create mode 100644 renku-core-client/src/main/resources/application.conf create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/ClientTools.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/Result.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/SchemaApiVersions.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/model.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/ClientToolsSpec.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/Generators.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala diff --git a/build.sbt b/build.sbt index 8dee2571ac..ed44b0c634 100644 --- a/build.sbt +++ b/build.sbt @@ -51,6 +51,7 @@ lazy val root = project entitiesSearch, entitiesViewingsCollector, triplesGenerator, + renkuCoreClient, knowledgeGraph ) @@ -204,6 +205,13 @@ lazy val tokenRepository = project AutomateHeaderPlugin ) +lazy val renkuCoreClient = project + .in(file("renku-core-client")) + .withId("renku-core-client") + .settings(commonSettings) + .dependsOn(graphCommons % "compile->compile; test->test") + .enablePlugins(AutomateHeaderPlugin) + lazy val knowledgeGraph = project .in(file("knowledge-graph")) .withId("knowledge-graph") @@ -220,6 +228,7 @@ lazy val knowledgeGraph = project graphCommons % "compile->compile; test->test", entitiesSearch % "compile->compile; test->test", triplesGeneratorApi % "compile->compile; test->test", + renkuCoreClient % "compile->compile; test->test", entitiesViewingsCollector ) .enablePlugins( diff --git a/graph-commons/src/test/scala/io/renku/stubbing/ExternalServiceStubbing.scala b/graph-commons/src/test/scala/io/renku/stubbing/ExternalServiceStubbing.scala index aa96edda78..881c6fa9f9 100644 --- a/graph-commons/src/test/scala/io/renku/stubbing/ExternalServiceStubbing.scala +++ b/graph-commons/src/test/scala/io/renku/stubbing/ExternalServiceStubbing.scala @@ -18,6 +18,7 @@ package io.renku.stubbing +import cats.effect.{IO, Resource} import com.github.tomakehurst.wiremock.WireMockServer import com.github.tomakehurst.wiremock.client.WireMock.equalTo import com.github.tomakehurst.wiremock.client.{MappingBuilder, WireMock} @@ -26,6 +27,7 @@ import eu.timepit.refined.api.Refined import eu.timepit.refined.numeric.Positive import io.renku.http.client.AccessToken import io.renku.http.client.AccessToken._ +import org.http4s.Uri import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, Suite} trait ExternalServiceStubbing extends BeforeAndAfterEach with BeforeAndAfterAll { @@ -46,6 +48,7 @@ trait ExternalServiceStubbing extends BeforeAndAfterEach with BeforeAndAfterAll } lazy val externalServiceBaseUrl: String = s"http://localhost:${server.port()}" + lazy val externalServiceBaseUri: Uri = Uri.unsafeFromString(externalServiceBaseUrl) override def beforeEach(): Unit = server.resetAll() @@ -64,4 +67,30 @@ trait ExternalServiceStubbing extends BeforeAndAfterEach with BeforeAndAfterAll case None => mappingBuilder } } + + protected def otherWireMockResource = + Resource.make[IO, WireMockServer] { + IO { + val config = WireMockConfiguration.wireMockConfig().dynamicPort() + val server = new WireMockServer(config) + + server.start() + + server + } + }(server => + IO { + server.shutdownServer() + } + ) + + protected implicit class WireMockOps(server: WireMockServer) { + + lazy val baseUri: Uri = Uri.unsafeFromString(server.baseUrl()) + + def stubFor(mappingBuilder: MappingBuilder): Unit = + new WireMock(server.port()).register { + WireMock.stubFor(mappingBuilder) + } + } } diff --git a/renku-core-client/README.md b/renku-core-client/README.md new file mode 100644 index 0000000000..931fa2f41e --- /dev/null +++ b/renku-core-client/README.md @@ -0,0 +1,3 @@ +# renku-core-client + +This is a client of the Renku-Core service diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf new file mode 100644 index 0000000000..19812442f4 --- /dev/null +++ b/renku-core-client/src/main/resources/application.conf @@ -0,0 +1,20 @@ + +services { + + renku-core-current { + url = "http://localhost:80" + url = ${?RENKU_CORE_CURRENT_URL} + } + + renku-core-v10 { + schemaVersion = "10" + url = "http://localhost:80" + url = ${?RENKU_CORE_V10_URL} + } + + renku-core-v9 { + schemaVersion = "9" + url = "http://localhost:80" + url = ${?RENKU_CORE_V9_URL} + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/ClientTools.scala b/renku-core-client/src/main/scala/io/renku/core/client/ClientTools.scala new file mode 100644 index 0000000000..4cb7e7eea4 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/ClientTools.scala @@ -0,0 +1,55 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.effect.Concurrent +import cats.syntax.all._ +import io.circe.DecodingFailure.Reason.CustomReason +import io.circe.{Decoder, DecodingFailure} +import org.http4s.circe.jsonOf +import org.http4s.{EntityDecoder, Request, Response, Status} + +private object ClientTools { + def apply[F[_]: Concurrent]: ClientTools[F] = new ClientTools[F] +} + +private class ClientTools[F[_]: Concurrent] { + + def toResult[T](resp: Response[F])(implicit dec: Decoder[T]): F[Result[T]] = { + + val resultDecoder = Decoder.instance { cur => + val maybeSuccess = cur.downField("result").success.map(_.as[T].map(Result.success[T])) + val maybeError = cur.downField("error").success.map(_.as[Result.Failure.Detailed]) + val illegalPayload = + DecodingFailure(CustomReason(s"Cannot decode core's response ${cur.value.spaces2}"), cur).asLeft + + maybeSuccess orElse maybeError getOrElse illegalPayload + } + + implicit val entityDec: EntityDecoder[F, Result[T]] = jsonOf[F, Result[T]](implicitly[Concurrent[F]], resultDecoder) + + resp.as[Result[T]].handleError(ex => Result.failure[T](ex.getMessage)) + } + + def toFailure[T](message: String): ((Status, Request[F], Response[F])) => F[Result[T]] = { case (status, req, resp) => + Result + .failure[T](s"$message: ${req.method} ${req.pathInfo.renderString} responded with: $status, ${resp.as[String]}") + .pure[F] + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala new file mode 100644 index 0000000000..7213ff1e2a --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala @@ -0,0 +1,70 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.syntax.all._ +import cats.{MonadThrow, Show} +import com.typesafe.config.{Config, ConfigFactory} +import io.renku.config.ConfigLoader.find +import io.renku.graph.model.versions.SchemaVersion +import org.http4s.Uri + +sealed trait RenkuCoreUri { + val uri: Uri +} + +object RenkuCoreUri { + + final case class Current(uri: Uri) extends RenkuCoreUri + final case class ForSchema(uri: Uri, schemaVersion: SchemaVersion) extends RenkuCoreUri + final case class Versioned(baseUri: ForSchema, apiVersion: ApiVersion) extends RenkuCoreUri { + val uri: Uri = baseUri.uri / apiVersion.value + } + + object Current { + def loadFromConfig[F[_]: MonadThrow](config: Config = ConfigFactory.load): F[Current] = + loadUrlFromConfig("services.renku-core-current.url", config).map(Current(_)) + } + + trait ForSchemaLoader { + def loadFromConfig[F[_]: MonadThrow](schemaVersion: SchemaVersion, + config: Config = ConfigFactory.load + ): F[ForSchema] + } + object ForSchema extends ForSchemaLoader { + + override def loadFromConfig[F[_]: MonadThrow](schemaVersion: SchemaVersion, + config: Config = ConfigFactory.load + ): F[ForSchema] = + loadUrlFromConfig(s"services.renku-core-v$schemaVersion.url", config).map(ForSchema(_, schemaVersion)) + } + + implicit def show[U <: RenkuCoreUri]: Show[U] = Show.show(_.uri.renderString) + + private def loadUrlFromConfig[F[_]: MonadThrow](key: String, config: Config): F[Uri] = + find[F, String](key, config).flatMap(toUri[F](key, _)) + + private def toUri[F[_]: MonadThrow](key: String, uri: String): F[Uri] = + Uri + .fromString(uri) + .fold( + new Exception(s"'$uri' is not a valid '$key' uri", _).raiseError[F, Uri], + _.pure[F] + ) +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala new file mode 100644 index 0000000000..84084ecd58 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala @@ -0,0 +1,89 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.MonadThrow +import cats.effect.Async +import cats.syntax.all._ +import com.typesafe.config.{Config, ConfigFactory} +import io.circe.Decoder +import io.circe.Decoder.decodeList +import io.renku.control.Throttler +import io.renku.graph.model.versions.SchemaVersion +import io.renku.http.client.RestClient +import org.http4s.client.dsl.Http4sClientDsl +import org.http4s.dsl.Http4sDsl +import org.typelevel.log4cats.Logger + +private trait RenkuCoreVersionClient[F[_]] { + def findCoreUri(schemaVersion: SchemaVersion): F[RenkuCoreUri.Versioned] + def getVersions: F[Result[List[SchemaVersion]]] + def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] +} + +private object RenkuCoreVersionClient { + def apply[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, + config: Config = ConfigFactory.load + ): RenkuCoreVersionClient[F] = + new RenkuCoreVersionClientImpl[F](coreCurrentUri, RenkuCoreUri.ForSchema, config, ClientTools[F]) +} + +private class RenkuCoreVersionClientImpl[F[_]: Async: Logger](coreUri: RenkuCoreUri.Current, + coreUriForSchemaLoader: RenkuCoreUri.ForSchemaLoader, + config: Config, + clientTools: ClientTools[F] +) extends RestClient[F, Nothing](Throttler.noThrottling) + with RenkuCoreVersionClient[F] + with Http4sDsl[F] + with Http4sClientDsl[F] { + + import clientTools._ + + override def findCoreUri(schemaVersion: SchemaVersion): F[RenkuCoreUri.Versioned] = + for { + uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) + apiVersionsRes <- getApiVersion(uriForSchema) + versionedUriRes = apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) + versionedUri <- MonadThrow[F].fromEither(versionedUriRes.toEither) + } yield versionedUri + + override def getVersions: F[Result[List[SchemaVersion]]] = { + val decoder = Decoder.instance[List[SchemaVersion]] { res => + val singleVersionDecoder = + Decoder.instance(_.downField("data").downField("metadata_version").as[SchemaVersion]) + + res.downField("versions").as(decodeList(singleVersionDecoder)) + } + + send(GET(coreUri.uri / "renku" / "versions")) { + case (Ok, _, resp) => toResult[List[SchemaVersion]](resp)(decoder) + case reqInfo => toFailure[List[SchemaVersion]](s"Version info cannot be found")(reqInfo) + } + } + + override def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] = + send(GET(uri.uri / "renku" / "apiversion")) { + case (Ok, _, resp) => + toResult[SchemaApiVersions](resp) + case reqInfo @ (NotFound, _, _) => + toFailure[SchemaApiVersions](s"Api version info for ${uri.uri} does not exist")(reqInfo) + case reqInfo => + toFailure[SchemaApiVersions](s"Finding api version info for ${uri.uri} failed")(reqInfo) + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/Result.scala b/renku-core-client/src/main/scala/io/renku/core/client/Result.scala new file mode 100644 index 0000000000..2441f673b4 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/Result.scala @@ -0,0 +1,76 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.FlatMap +import io.circe.Decoder + +sealed trait Result[+A] { + def toEither: Either[Throwable, A] +} + +object Result { + final case class Success[+A](value: A) extends Result[A] { + def toEither: Either[Throwable, A] = Right(value) + } + + sealed trait Failure extends RuntimeException with Result[Nothing] { + def toEither: Either[Throwable, Nothing] = Left(this) + } + + object Failure { + + final case class Simple(error: String) extends RuntimeException(error) with Failure + final case class Detailed(code: Int, userMessage: String) + extends RuntimeException(s"$userMessage: $code") + with Failure + + object Detailed { + implicit val decoder: Decoder[Detailed] = + Decoder.forProduct2("code", "userMessage")(Detailed.apply) + } + } + + def success[A](value: A): Result[A] = Success(value) + + def failure[A](error: String): Result[A] = Failure.Simple(error) + + def failure[A](code: Int, userMessage: String): Result[A] = Failure.Detailed(code, userMessage) + + implicit lazy val flatMapOps: FlatMap[Result] = new FlatMap[Result] { + + override def map[A, B](fa: Result[A])(f: A => B): Result[B] = + fa match { + case Success(a) => success(f(a)) + case f: Failure => f + } + + override def flatMap[A, B](fa: Result[A])(f: A => Result[B]): Result[B] = + fa match { + case Success(a) => f(a) + case f: Failure => f + } + + override def tailRecM[A, B](a: A)(f: A => Result[Either[A, B]]): Result[B] = + flatMap(f(a)) { + case Left(v) => tailRecM[A, B](v)(f) + case Right(b) => success(b) + } + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/SchemaApiVersions.scala b/renku-core-client/src/main/scala/io/renku/core/client/SchemaApiVersions.scala new file mode 100644 index 0000000000..4e1c5435c9 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/SchemaApiVersions.scala @@ -0,0 +1,31 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import io.circe.Decoder +import io.renku.graph.model.versions.CliVersion + +final case class SchemaApiVersions(min: ApiVersion, max: ApiVersion, cliVersion: CliVersion) + +object SchemaApiVersions { + + implicit val decoder: Decoder[SchemaApiVersions] = + Decoder + .forProduct3("minimum_api_version", "maximum_api_version", "latest_version")(SchemaApiVersions.apply) +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/model.scala b/renku-core-client/src/main/scala/io/renku/core/client/model.scala new file mode 100644 index 0000000000..6ab5a93cf2 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/model.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import io.circe.Decoder +import io.renku.tinytypes.constraints.NonBlank +import io.renku.tinytypes.json.TinyTypeDecoders +import io.renku.tinytypes.{StringTinyType, TinyTypeFactory} + +final class ApiVersion private (val value: String) extends AnyVal with StringTinyType +object ApiVersion extends TinyTypeFactory[ApiVersion](new ApiVersion(_)) with NonBlank[ApiVersion] { + implicit val decoder: Decoder[ApiVersion] = TinyTypeDecoders.stringDecoder(ApiVersion) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ClientToolsSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/ClientToolsSpec.scala new file mode 100644 index 0000000000..874e2f54b2 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/ClientToolsSpec.scala @@ -0,0 +1,62 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators.resultDetailedFailures +import cats.effect.IO +import io.circe.literal._ +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.nonEmptyStrings +import io.renku.testtools.CustomAsyncIOSpec +import org.http4s.Response +import org.http4s.Status.Ok +import org.http4s.circe._ +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec + +class ClientToolsSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matchers { + + "toResult" should { + + "decode the object from the 'result' into Result.success" in { + + val result = nonEmptyStrings().generateOne + val response = Response[IO](Ok).withEntity(json"""{"result": $result}""") + + ClientTools[IO].toResult[String](response).asserting(_ shouldBe Result.success(result)) + } + + "decode the failure into Result.Failure.Detailed" in { + + val failure = resultDetailedFailures.generateOne + + val response = Response[IO](Ok) + .withEntity { + json"""{ + "error": { + "code": ${failure.code}, + "userMessage": ${failure.userMessage} + } + }""" + } + + ClientTools[IO].toResult[String](response).asserting(_ shouldBe failure) + } + } +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala new file mode 100644 index 0000000000..cf24d6aa6d --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -0,0 +1,54 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.syntax.all._ +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators._ +import io.renku.graph.model.RenkuTinyTypeGenerators.{cliVersions, projectSchemaVersions} +import org.http4s.Uri +import org.scalacheck.Gen + +object Generators { + + def resultSuccesses[T](payloadGen: Gen[T]): Gen[Result[T]] = + payloadGen.map(Result.success) + + implicit lazy val resultDetailedFailures: Gen[Result.Failure.Detailed] = + (positiveInts().map(_.value) -> sentences().map(_.value)) + .mapN(Result.Failure.Detailed(_, _)) + + implicit lazy val apiVersions: Gen[ApiVersion] = + (positiveInts(), positiveInts()).mapN((major, minor) => ApiVersion(s"$major.$minor")) + + implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = + for { + baseUri <- httpUrls() + schema <- projectSchemaVersions + } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(baseUri), schema) + + implicit lazy val coreUrisVersioned: Gen[RenkuCoreUri.Versioned] = + for { + baseUri <- coreUrisForSchema + apiVersion <- apiVersions + } yield RenkuCoreUri.Versioned(baseUri, apiVersion) + + implicit lazy val schemaApiVersions: Gen[SchemaApiVersions] = + (apiVersions, apiVersions, cliVersions).mapN(SchemaApiVersions.apply) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala new file mode 100644 index 0000000000..dae964324d --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala @@ -0,0 +1,51 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import io.circe.Encoder +import io.circe.literal._ +import io.renku.graph.model.versions.{CliVersion, SchemaVersion} + +private object ModelEncoders { + + implicit val versionsEnc: Encoder[List[(SchemaVersion, CliVersion)]] = Encoder.instance { versions => + val versionJsons = versions.map { case (schema, cli) => + val cliValue = s"v$cli" + json"""{ + "data": { + "metadata_version": $schema + }, + "version": $cliValue + }""" + } + + json"""{ + "name": "renku-core", + "versions": $versionJsons + }""" + } + + implicit val schemaApiVersionsEnc: Encoder[SchemaApiVersions] = Encoder.instance { + case SchemaApiVersions(min, max, cliVersion) => json"""{ + "minimum_api_version": $min, + "maximum_api_version": $max, + "latest_version": $cliVersion + }""" + } +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala new file mode 100644 index 0000000000..e9c733d450 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala @@ -0,0 +1,138 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators._ +import com.typesafe.config.ConfigFactory +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.httpUrls +import io.renku.graph.model.GraphModelGenerators.projectSchemaVersions +import io.renku.graph.model.versions.SchemaVersion +import org.http4s.Uri +import org.scalatest.TryValues +import org.scalatest.matchers.should +import org.scalatest.wordspec.AnyWordSpec + +import scala.util.Try + +class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { + + "Current.loadFromConfig" should { + + "read the 'services.renku-core-current.url' from the Config" in { + + val url = httpUrls().generateOne + val config = configForCurrent(url) + + RenkuCoreUri.Current.loadFromConfig[Try](config).success.value shouldBe + RenkuCoreUri.Current(Uri.unsafeFromString(url)) + } + + "fail if the url does not exist" in { + RenkuCoreUri.Current.loadFromConfig[Try](ConfigFactory.empty()).failure.exception.getMessage should include( + "Key not found: 'services'" + ) + } + + "fail if the url is invalid" in { + + val illegalUrl = "?ddkf !&&" + + val config = configForCurrent(illegalUrl) + + RenkuCoreUri.Current.loadFromConfig[Try](config).failure.exception.getMessage should include( + s"'$illegalUrl' is not a valid 'services.renku-core-current.url' uri" + ) + } + } + + "ForSchema.loadFromConfig" should { + + "read the 'services.renku-core-vXXX.url' from the Config" in { + + val uri = coreUrisForSchema.generateOne + val config = configForSchema(uri.schemaVersion, uri.uri.renderString) + + RenkuCoreUri.ForSchema.loadFromConfig[Try](uri.schemaVersion, config).success.value shouldBe uri + } + + "fail if the url does not exist" in { + RenkuCoreUri.ForSchema + .loadFromConfig[Try](projectSchemaVersions.generateOne, ConfigFactory.empty()) + .failure + .exception + .getMessage should include("Key not found: 'services'") + } + + "fail if the url is invalid" in { + + val illegalUrl = "?ddkf !&&" + val schemaVersion = projectSchemaVersions.generateOne + + val config = configForSchema(schemaVersion, illegalUrl) + + RenkuCoreUri.ForSchema + .loadFromConfig[Try](schemaVersion, config) + .failure + .exception + .getMessage should include(s"'$illegalUrl' is not a valid 'services.renku-core-v$schemaVersion.url' uri") + } + } + + "ForSchema.uri" should { + + "return the same uri that was used on instantiation" in { + + val uri = Uri.unsafeFromString(httpUrls().generateOne) + val schema = projectSchemaVersions.generateOne + + RenkuCoreUri.ForSchema(uri, schema).uri shouldBe uri + } + } + + "Versioned.uri" should { + + "return a value that is composed from the baseUri and the apiVersion" in { + + val baseUri = coreUrisForSchema.generateOne + val apiVersion = apiVersions.generateOne + + RenkuCoreUri.Versioned(baseUri, apiVersion).uri shouldBe baseUri.uri / apiVersion.value + } + } + + private def configForCurrent(url: String) = + ConfigFactory.parseString( + s"""services { + renku-core-current { + url = "$url" + } + }""" + ) + + private def configForSchema(schema: SchemaVersion, url: String) = + ConfigFactory.parseString( + s"""services { + renku-core-v$schema { + schemaVersion = "$schema" + url = "$url" + } + }""" + ) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala new file mode 100644 index 0000000000..96be509ca4 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala @@ -0,0 +1,173 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators._ +import ModelEncoders._ +import cats.MonadThrow +import cats.effect.IO +import cats.syntax.all._ +import com.github.tomakehurst.wiremock.client.WireMock._ +import com.typesafe.config.Config +import io.circe.Encoder +import io.circe.literal._ +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.exceptions +import io.renku.graph.model.GraphModelGenerators.{cliVersions, projectSchemaVersions} +import io.renku.graph.model.versions.SchemaVersion +import io.renku.interpreters.TestLogger +import io.renku.stubbing.ExternalServiceStubbing +import io.renku.testtools.CustomAsyncIOSpec +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec +import org.scalatest.{EitherValues, OptionValues} +import org.typelevel.log4cats.Logger + +class RenkuCoreVersionClientSpec + extends AsyncWordSpec + with CustomAsyncIOSpec + with should.Matchers + with OptionValues + with EitherValues + with ExternalServiceStubbing + with AsyncMockFactory { + + "findCoreUri" should { + + "find the uri of the core for the given schema in the config, " + + "fetch the api version using the uri and " + + "return the CoreUri relevant for the given schema" in { + + val schemaVersion = projectSchemaVersions.generateOne + val apiVersions = schemaApiVersions.generateOne + + givenApiVersionFetching(schemaVersion, returning = Result.success(apiVersions)).use { + case (coreUriForSchema, _) => + givenCoreUriForSchemaInConfig(returning = coreUriForSchema) + + client + .findCoreUri(schemaVersion) + .asserting(_ shouldBe RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max)) + } + } + + "fail if finding the uri of the core for the given schema in the config fails " in { + + val exception = exceptions.generateOne + givenCoreUriForSchemaInConfig(failsWith = exception) + + val schemaVersion = projectSchemaVersions.generateOne + + client + .findCoreUri(schemaVersion) + .assertThrowsError[Exception](_ shouldBe exception) + } + + "fail if fetching the api version fails" in { + + val schemaVersion = projectSchemaVersions.generateOne + val failure = resultDetailedFailures.generateOne + + givenApiVersionFetching(schemaVersion, returning = failure).use { case (coreUriForSchema, _) => + givenCoreUriForSchemaInConfig(returning = coreUriForSchema) + + client + .findCoreUri(schemaVersion) + .assertThrowsError[Exception](_ shouldBe failure) + } + } + } + + "getVersions" should { + + "return info about available versions" in { + + val versionTuples = (projectSchemaVersions -> cliVersions).mapN(_ -> _).generateList() + + stubFor { + get(s"/renku/versions") + .willReturn(ok(Result.success(versionTuples).asJson.spaces2)) + } + + client.getVersions.asserting(_ shouldBe Result.success(versionTuples.map(_._1))) + } + } + + "getApiVersion" should { + + "return info about API versions" in { + + val successResult = resultSuccesses(schemaApiVersions).generateOne + + givenApiVersionFetching(projectSchemaVersions.generateOne, returning = successResult).map { + case (_, actualApiVersions) => actualApiVersions shouldBe successResult + }.use_ + } + } + + private val coreUriForSchemaLoader = mock[RenkuCoreUri.ForSchemaLoader] + private lazy val config = mock[Config] + private implicit val logger: Logger[IO] = TestLogger() + private lazy val client = + new RenkuCoreVersionClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), + coreUriForSchemaLoader, + config, + ClientTools[IO] + ) + + private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = + (coreUriForSchemaLoader + .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) + .expects(returning.schemaVersion, config, *) + .returning(returning.pure[IO]) + + private def givenCoreUriForSchemaInConfig(failsWith: Throwable) = + (coreUriForSchemaLoader + .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) + .expects(*, config, *) + .returning(failsWith.raiseError[IO, Nothing]) + + private def givenApiVersionFetching(schemaVersion: SchemaVersion, returning: Result[SchemaApiVersions]) = + otherWireMockResource.evalMap { server => + server.stubFor { + get(s"/renku/apiversion") + .willReturn(ok(returning.asJson.spaces2)) + } + + val uriForSchema = RenkuCoreUri.ForSchema(server.baseUri, schemaVersion) + + client.getApiVersion(uriForSchema).map(uriForSchema -> _) + } + + private implicit def resultEncoder[T](implicit enc: Encoder[T]): Encoder[Result[T]] = + Encoder.instance { + case Result.Success(obj) => json"""{ + "result": ${obj.asJson} + }""" + case Result.Failure.Detailed(code, userMessage) => json"""{ + "error": { + "code": $code, + "userMessage": $userMessage + } + }""" + case result => fail(s"$result shouldn't be in the core API response payload") + } +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala new file mode 100644 index 0000000000..8fee3df7c1 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators._ +import ModelEncoders.schemaApiVersionsEnc +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import org.scalatest.EitherValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should +import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks + +class SchemaApiVersionsSpec extends AnyFlatSpec with should.Matchers with EitherValues with ScalaCheckPropertyChecks { + + it should "decode from JSON" in { + forAll { apiVersions: SchemaApiVersions => + apiVersions.asJson.hcursor.as[SchemaApiVersions].value shouldBe apiVersions + } + } +} From f0e83e47a49ceca0bff0be3f563987a504e941e8 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 10 Aug 2023 21:03:05 +0200 Subject: [PATCH 06/49] refactor: VersionClient to return Result from findCoreUri --- .../renku/core/client/RenkuCoreClient.scala | 49 +++++++++++++++++++ .../core/client/RenkuCoreVersionClient.scala | 9 ++-- .../core/client/RenkuCoreClientSpec.scala | 49 +++++++++++++++++++ .../client/RenkuCoreVersionClientSpec.scala | 8 ++- 4 files changed, 104 insertions(+), 11 deletions(-) create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala new file mode 100644 index 0000000000..8d10613341 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -0,0 +1,49 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.effect.Async +import cats.syntax.all._ +import com.typesafe.config.{Config, ConfigFactory} +import io.renku.control.Throttler +import io.renku.http.client.RestClient +import org.http4s.client.dsl.Http4sClientDsl +import org.http4s.dsl.Http4sDsl +import org.typelevel.log4cats.Logger + +trait RenkuCoreClient[F[_]] {} + +object RenkuCoreClient { + def apply[F[_]: Async: Logger](config: Config = ConfigFactory.load): F[RenkuCoreClient[F]] = + for { + coreCurrentUri <- RenkuCoreUri.Current.loadFromConfig[F](config) + versionClient = RenkuCoreVersionClient[F](coreCurrentUri, config) + } yield new RenkuCoreClientImpl[F](coreCurrentUri, versionClient) +} + +private class RenkuCoreClientImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, + versionClient: RenkuCoreVersionClient[F] +) extends RestClient[F, Nothing](Throttler.noThrottling) + with RenkuCoreClient[F] + with Http4sDsl[F] + with Http4sClientDsl[F] { + + println(coreCurrentUri) + println(versionClient) +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala index 84084ecd58..443907fc3c 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala @@ -18,7 +18,6 @@ package io.renku.core.client -import cats.MonadThrow import cats.effect.Async import cats.syntax.all._ import com.typesafe.config.{Config, ConfigFactory} @@ -32,7 +31,7 @@ import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger private trait RenkuCoreVersionClient[F[_]] { - def findCoreUri(schemaVersion: SchemaVersion): F[RenkuCoreUri.Versioned] + def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] def getVersions: F[Result[List[SchemaVersion]]] def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] } @@ -55,13 +54,11 @@ private class RenkuCoreVersionClientImpl[F[_]: Async: Logger](coreUri: RenkuCore import clientTools._ - override def findCoreUri(schemaVersion: SchemaVersion): F[RenkuCoreUri.Versioned] = + override def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] = for { uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) apiVersionsRes <- getApiVersion(uriForSchema) - versionedUriRes = apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) - versionedUri <- MonadThrow[F].fromEither(versionedUriRes.toEither) - } yield versionedUri + } yield apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) override def getVersions: F[Result[List[SchemaVersion]]] = { val decoder = Decoder.instance[List[SchemaVersion]] { res => diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala new file mode 100644 index 0000000000..ec1112fac8 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -0,0 +1,49 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.effect.IO +import io.renku.interpreters.TestLogger +import io.renku.stubbing.ExternalServiceStubbing +import io.renku.testtools.CustomAsyncIOSpec +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec +import org.scalatest.{EitherValues, OptionValues} +import org.typelevel.log4cats.Logger + +class RenkuCoreClientSpec + extends AsyncWordSpec + with CustomAsyncIOSpec + with should.Matchers + with OptionValues + with EitherValues + with ExternalServiceStubbing + with AsyncMockFactory { + + private implicit val logger: Logger[IO] = TestLogger() + private val coreVersionClient = mock[RenkuCoreVersionClient[IO]] + private lazy val client = new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), coreVersionClient) + +// private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = +// (coreUriForSchemaLoader +// .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) +// .expects(returning.schemaVersion, config, *) +// .returning(returning.pure[IO]) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala index 96be509ca4..823b0162e0 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala @@ -65,7 +65,7 @@ class RenkuCoreVersionClientSpec client .findCoreUri(schemaVersion) - .asserting(_ shouldBe RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max)) + .asserting(_ shouldBe Result.success(RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max))) } } @@ -81,7 +81,7 @@ class RenkuCoreVersionClientSpec .assertThrowsError[Exception](_ shouldBe exception) } - "fail if fetching the api version fails" in { + "return a failure if fetching the api version fails" in { val schemaVersion = projectSchemaVersions.generateOne val failure = resultDetailedFailures.generateOne @@ -89,9 +89,7 @@ class RenkuCoreVersionClientSpec givenApiVersionFetching(schemaVersion, returning = failure).use { case (coreUriForSchema, _) => givenCoreUriForSchemaInConfig(returning = coreUriForSchema) - client - .findCoreUri(schemaVersion) - .assertThrowsError[Exception](_ shouldBe failure) + client.findCoreUri(schemaVersion).asserting(_ shouldBe failure) } } } From cda8ba7cf4f40518eda4cf4d72f556b37fe9df3d Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 10 Aug 2023 21:07:06 +0200 Subject: [PATCH 07/49] refactor: Result encoder moved to ModelEncoders --- .../io/renku/core/client/ModelEncoders.scala | 15 +++++++++++++++ .../core/client/RenkuCoreVersionClientSpec.scala | 16 ---------------- 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala index dae964324d..cda6ecb0ee 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala @@ -20,6 +20,7 @@ package io.renku.core.client import io.circe.Encoder import io.circe.literal._ +import io.circe.syntax._ import io.renku.graph.model.versions.{CliVersion, SchemaVersion} private object ModelEncoders { @@ -48,4 +49,18 @@ private object ModelEncoders { "latest_version": $cliVersion }""" } + + implicit def resultEncoder[T](implicit enc: Encoder[T]): Encoder[Result[T]] = + Encoder.instance { + case Result.Success(obj) => json"""{ + "result": ${obj.asJson} + }""" + case Result.Failure.Detailed(code, userMessage) => json"""{ + "error": { + "code": $code, + "userMessage": $userMessage + } + }""" + case result => throw new Exception(s"$result shouldn't be in the core API response payload") + } } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala index 823b0162e0..02e440e5c8 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala @@ -25,8 +25,6 @@ import cats.effect.IO import cats.syntax.all._ import com.github.tomakehurst.wiremock.client.WireMock._ import com.typesafe.config.Config -import io.circe.Encoder -import io.circe.literal._ import io.circe.syntax._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.exceptions @@ -154,18 +152,4 @@ class RenkuCoreVersionClientSpec client.getApiVersion(uriForSchema).map(uriForSchema -> _) } - - private implicit def resultEncoder[T](implicit enc: Encoder[T]): Encoder[Result[T]] = - Encoder.instance { - case Result.Success(obj) => json"""{ - "result": ${obj.asJson} - }""" - case Result.Failure.Detailed(code, userMessage) => json"""{ - "error": { - "code": $code, - "userMessage": $userMessage - } - }""" - case result => fail(s"$result shouldn't be in the core API response payload") - } } From d08a83c68e2d854e55d645285c5fe80fa9d3457a Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 11 Aug 2023 11:53:48 +0200 Subject: [PATCH 08/49] feat: Renku Core Client --- .../renku/core/client/RenkuCoreClientSpec.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index ec1112fac8..392df2bdfd 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -19,10 +19,13 @@ package io.renku.core.client import cats.effect.IO +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.GraphModelGenerators.{projectSchemaVersions, projectSlugs} import io.renku.interpreters.TestLogger import io.renku.stubbing.ExternalServiceStubbing import io.renku.testtools.CustomAsyncIOSpec import org.scalamock.scalatest.AsyncMockFactory +import com.github.tomakehurst.wiremock.client.WireMock._ import org.scalatest.matchers.should import org.scalatest.wordspec.AsyncWordSpec import org.scalatest.{EitherValues, OptionValues} @@ -37,6 +40,20 @@ class RenkuCoreClientSpec with ExternalServiceStubbing with AsyncMockFactory { + "getMigrationCheck" should { + + "return info about current project schema version" in { + + val schemaVersion = projectSchemaVersions.generateOne + val projectSlug = projectSlugs.generateOne + + stubFor { + get(s"/renku/cache.migrations_check").withQueryParam("git_url", ) + .willReturn(ok(Result.success(versionTuples).asJson.spaces2)) + } + } + } + private implicit val logger: Logger[IO] = TestLogger() private val coreVersionClient = mock[RenkuCoreVersionClient[IO]] private lazy val client = new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), coreVersionClient) From 22b7255aac4dff718193a8347913983026227039 Mon Sep 17 00:00:00 2001 From: RenkuBot <53332360+RenkuBot@users.noreply.github.com> Date: Thu, 10 Aug 2023 09:55:36 +0200 Subject: [PATCH 09/49] chore: Update logback-classic from 1.4.9 to 1.4.11 (#1653) --- project/Dependencies.scala | 2 +- token-repository/build.sbt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 44629e1df0..6803f8e41d 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -19,7 +19,7 @@ object Dependencies { val jsonld4s = "0.12.0" val log4cats = "2.6.0" val log4jCore = "2.20.0" - val logback = "1.4.9" + val logback = "1.4.11" val luceneQueryParser = "9.7.0" val monocle = "2.1.0" val owlapi = "5.5.0" diff --git a/token-repository/build.sbt b/token-repository/build.sbt index 3072493510..0e0f6784f1 100644 --- a/token-repository/build.sbt +++ b/token-repository/build.sbt @@ -20,4 +20,4 @@ name := "token-repository" Test / fork := true -libraryDependencies += "ch.qos.logback" % "logback-classic" % "1.4.9" +libraryDependencies += "ch.qos.logback" % "logback-classic" % "1.4.11" From 93932dd9424b50631fb5e6746c569bf721679b00 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 11 Aug 2023 17:27:31 +0200 Subject: [PATCH 10/49] fix: Cross-Entity Search to fetch creator name from the Persons graph (#1656) * fix: Cross-Entity Search to fetch creator name from the Persons graph * fix: Recently Viewed Entities to fetch creator name from the Persons graph * fix: Search graphs provisioning not to contain creator name * feat: TS migrations that remove Person entities from the Search graphs --- .../renku/entities/search/DatasetsQuery.scala | 11 +- .../renku/entities/search/ProjectsQuery.scala | 7 +- .../entities/searchgraphs/PersonInfo.scala | 34 ------ .../datasets/DatasetSearchInfo.scala | 5 +- .../datasets/SearchInfoExtractor.scala | 3 +- .../datasets/commands/Encoders.scala | 18 +-- .../datasets/commands/SearchInfoFetcher.scala | 28 ++--- .../searchgraphs/datasets/ontology.scala | 2 +- .../projects/ProjectSearchInfo.scala | 5 +- .../projects/SearchInfoExtractor.scala | 3 +- .../projects/commands/Encoders.scala | 16 +-- .../searchgraphs/projects/ontology.scala | 13 +-- .../entities/searchgraphs/Generators.scala | 6 - .../searchgraphs/PersonInfoSpec.scala | 37 ------ .../searchgraphs/datasets/Generators.scala | 3 +- .../datasets/SearchInfoExtractorSpec.scala | 5 +- .../datasets/commands/EncodersSpec.scala | 23 +--- .../commands/SearchInfoFetcherSpec.scala | 14 +-- .../searchgraphs/projects/Generators.scala | 3 +- .../projects/SearchInfoExtractorSpec.scala | 3 +- .../projects/commands/EncodersSpec.scala | 24 +--- .../viewings/search/DatasetQuery.scala | 5 +- .../io/renku/triplesstore/InMemoryJena.scala | 3 + .../DatasetsGraphPersonRemover.scala | 57 ++++++++++ .../migrations/Migrations.scala | 6 +- .../ProjectsGraphPersonRemover.scala | 57 ++++++++++ .../RegisteredUpdateQueryMigration.scala | 55 +++++++++ .../DatasetsGraphPersonRemoverSpec.scala | 106 ++++++++++++++++++ .../ProjectsGraphPersonRemoverSpec.scala | 104 +++++++++++++++++ .../RegisteredUpdateQueryMigrationSpec.scala | 88 +++++++++++++++ 30 files changed, 529 insertions(+), 215 deletions(-) delete mode 100644 entities-search/src/main/scala/io/renku/entities/searchgraphs/PersonInfo.scala delete mode 100644 entities-search/src/test/scala/io/renku/entities/searchgraphs/PersonInfoSpec.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemover.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemover.scala create mode 100644 triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigration.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemoverSpec.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemoverSpec.scala create mode 100644 triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigrationSpec.scala diff --git a/entities-search/src/main/scala/io/renku/entities/search/DatasetsQuery.scala b/entities-search/src/main/scala/io/renku/entities/search/DatasetsQuery.scala index f6d28789ff..9d6f2eb248 100644 --- a/entities-search/src/main/scala/io/renku/entities/search/DatasetsQuery.scala +++ b/entities-search/src/main/scala/io/renku/entities/search/DatasetsQuery.scala @@ -91,9 +91,12 @@ object DatasetsQuery extends EntityQuery[Entity.Dataset] { | # textQuery | ${textQueryPart(criteria.filters.maybeQuery)} | - | Graph schema:Dataset { + | GRAPH schema:Dataset { | #creator - | $sameAsVar schema:creator / schema:name ?creatorName. + | $sameAsVar schema:creator ?creatorId. + | GRAPH ${GraphClass.Persons.id} { + | ?creatorId schema:name ?creatorName + | } | | #keywords | $keywords @@ -105,7 +108,7 @@ object DatasetsQuery extends EntityQuery[Entity.Dataset] { | $images | } | - | Graph ?projId { + | GRAPH ?projId { | # project namespaces | ${namespacesPart(criteria.filters.namespaces)} | @@ -121,7 +124,7 @@ object DatasetsQuery extends EntityQuery[Entity.Dataset] { | | ${creatorsPart(criteria.filters.creators)} | - | Graph schema:Dataset { + | GRAPH schema:Dataset { | # name | $sameAsVar renku:slug $nameVar | diff --git a/entities-search/src/main/scala/io/renku/entities/search/ProjectsQuery.scala b/entities-search/src/main/scala/io/renku/entities/search/ProjectsQuery.scala index 65f1d2b4cd..1e6c11a4be 100644 --- a/entities-search/src/main/scala/io/renku/entities/search/ProjectsQuery.scala +++ b/entities-search/src/main/scala/io/renku/entities/search/ProjectsQuery.scala @@ -97,7 +97,12 @@ private case object ProjectsQuery extends EntityQuery[model.Entity.Project] { | ${namespacesPart(criteria.filters.namespaces)} | } | - | OPTIONAL { $projectIdVar schema:creator/schema:name $someCreatorNameVar } + | OPTIONAL { + | $projectIdVar schema:creator ?creatorId. + | GRAPH ${GraphClass.Persons.id} { + | ?creatorId schema:name $someCreatorNameVar + | } + | } | ${filters.maybeOnCreatorName(someCreatorNameVar)} | | OPTIONAL { $projectIdVar schema:description $maybeDescriptionVar } diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/PersonInfo.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/PersonInfo.scala deleted file mode 100644 index 6ed09be392..0000000000 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/PersonInfo.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright 2023 Swiss Data Science Center (SDSC) - * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and - * Eidgenössische Technische Hochschule Zürich (ETHZ). - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.renku.entities.searchgraphs - -import cats.Show -import cats.syntax.all._ -import io.renku.graph.model.entities.Person -import io.renku.graph.model.persons - -private final case class PersonInfo(resourceId: persons.ResourceId, name: persons.Name) - -private object PersonInfo { - - lazy val toPersonInfo: Person => PersonInfo = p => PersonInfo(p.resourceId, p.name) - implicit lazy val show: Show[PersonInfo] = Show.show { case PersonInfo(resourceId, name) => - show"id = $resourceId, name = $name" - } -} diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/DatasetSearchInfo.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/DatasetSearchInfo.scala index 3f23a501f2..1d17e9058e 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/DatasetSearchInfo.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/DatasetSearchInfo.scala @@ -21,16 +21,15 @@ package io.renku.entities.searchgraphs.datasets import cats.Show import cats.data.NonEmptyList import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo import io.renku.graph.model.images.Image -import io.renku.graph.model.{datasets, projects} +import io.renku.graph.model.{datasets, persons, projects} private final case class DatasetSearchInfo(topmostSameAs: datasets.TopmostSameAs, name: datasets.Name, visibility: projects.Visibility, createdOrPublished: datasets.CreatedOrPublished, maybeDateModified: Option[datasets.DateModified], - creators: NonEmptyList[PersonInfo], + creators: NonEmptyList[persons.ResourceId], keywords: List[datasets.Keyword], maybeDescription: Option[datasets.Description], images: List[Image], diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractor.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractor.scala index a972c4a530..f2545dae02 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractor.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractor.scala @@ -21,7 +21,6 @@ package io.renku.entities.searchgraphs.datasets import cats.MonadThrow import cats.data.NonEmptyList import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo.toPersonInfo import io.renku.graph.model.datasets import io.renku.graph.model.entities.{Dataset, Project} @@ -65,7 +64,7 @@ private object SearchInfoExtractor { project.visibility, createdOrPublished, maybeDateModified, - ds.provenance.creators.map(toPersonInfo), + ds.provenance.creators.map(_.resourceId), ds.additionalInfo.keywords, ds.additionalInfo.maybeDescription, ds.additionalInfo.images, diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/Encoders.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/Encoders.scala index 401e80e71b..b0b8a9766b 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/Encoders.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/Encoders.scala @@ -19,13 +19,11 @@ package io.renku.entities.searchgraphs.datasets.commands import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo import io.renku.entities.searchgraphs.datasets.Link.{ImportedDataset, OriginalDataset} import io.renku.entities.searchgraphs.datasets.{DatasetSearchInfo, DatasetSearchInfoOntology, Link, LinkOntology} import io.renku.graph.model.Schemas.{rdf, renku} -import io.renku.graph.model.datasets -import io.renku.graph.model.entities.Person import io.renku.graph.model.images.Image +import io.renku.graph.model.{datasets, persons} import io.renku.jsonld.Property import io.renku.jsonld.syntax._ import io.renku.triplesstore.client.model.{Quad, QuadsEncoder, TripleObject} @@ -33,14 +31,6 @@ import io.renku.triplesstore.client.syntax._ private object Encoders { - implicit val personInfoEncoder: QuadsEncoder[PersonInfo] = QuadsEncoder.instance { - case PersonInfo(resourceId, name) => - Set( - DatasetsQuad(resourceId, rdf / "type", Person.Ontology.typeClass.id), - DatasetsQuad(resourceId, Person.Ontology.nameProperty.id, name.asObject) - ) - } - implicit val imageEncoder: QuadsEncoder[Image] = QuadsEncoder.instance { case Image(resourceId, uri, position) => Set( DatasetsQuad(resourceId, rdf / "type", Image.Ontology.typeClass.id), @@ -84,9 +74,8 @@ private object Encoders { searchInfoQuad(DatasetSearchInfoOntology.descriptionProperty.id, d.asObject) } - val creatorsQuads = info.creators.toList.toSet.flatMap { (pi: PersonInfo) => - pi.asQuads + - searchInfoQuad(DatasetSearchInfoOntology.creatorProperty, pi.resourceId.asEntityId) + val creatorsQuads = info.creators.toList.toSet.map { (resourceId: persons.ResourceId) => + searchInfoQuad(DatasetSearchInfoOntology.creatorProperty, resourceId.asEntityId) } val keywordsQuads = info.keywords.toSet.map { (k: datasets.Keyword) => @@ -107,6 +96,7 @@ private object Encoders { searchInfoQuad(rdf / "type", DatasetSearchInfoOntology.typeDef.clazz.id).some, searchInfoQuad(DatasetSearchInfoOntology.slugProperty.id, info.name.asObject).some, searchInfoQuad(DatasetSearchInfoOntology.visibilityProperty.id, info.visibility.asObject).some, + searchInfoQuad(DatasetSearchInfoOntology.visibilityProperty.id, info.visibility.asObject).some, createdOrPublishedQuad.some, maybeDateModifiedQuad, maybeDescriptionQuad diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcher.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcher.scala index de6b49db60..10f13c7a46 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcher.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcher.scala @@ -21,7 +21,6 @@ package io.renku.entities.searchgraphs.datasets.commands import cats.data.NonEmptyList import cats.effect.Async import io.renku.entities.searchgraphs -import io.renku.entities.searchgraphs.PersonInfo import io.renku.entities.searchgraphs.datasets.{DatasetSearchInfo, Link, links} import io.renku.graph.model.datasets.TopmostSameAs import io.renku.graph.model.{GraphClass, projects} @@ -78,16 +77,12 @@ private class SearchInfoFetcherImpl[F[_]: Async: Logger: SparqlQueryTimeRecorder | } { | GRAPH ${GraphClass.Datasets.id} { | ?topSameAs renku:slug ?name; - | renku:projectVisibility ?visibility. + | renku:projectVisibility ?visibility; + | schema:creator ?creator. | OPTIONAL { ?topSameAs schema:description ?maybeDescription } | OPTIONAL { ?topSameAs schema:dateCreated ?maybeDateCreated } | OPTIONAL { ?topSameAs schema:datePublished ?maybeDatePublished } | OPTIONAL { ?topSameAs schema:dateModified ?maybeDateModified } - | { - | ?topSameAs schema:creator ?creatorId. - | ?creatorId schema:name ?creatorName. - | BIND (CONCAT(STR(?creatorId), STR(';;'), STR(?creatorName)) AS ?creator) - | } | OPTIONAL { ?topSameAs schema:keywords ?keyword } | OPTIONAL { | ?topSameAs schema:image ?imageId. @@ -138,18 +133,15 @@ private class SearchInfoFetcherImpl[F[_]: Async: Logger: SparqlQueryTimeRecorder _.map(decode[datasets.Keyword](map = toKeyword, sort = _.sorted)) .getOrElse(List.empty.asRight[DecodingFailure]) - val toCreator: String => Decoder.Result[PersonInfo] = { - case s"$id;;$name" => - (persons.ResourceId.from(id) -> persons.Name.from(name)) - .mapN(PersonInfo(_, _)) - .leftMap(ex => DecodingFailure(DecodingFailure.Reason.CustomReason(ex.getMessage), cur)) - case other => - DecodingFailure(DecodingFailure.Reason.CustomReason(s"'$other' not a valid creator record"), cur) - .asLeft[PersonInfo] - } + val toCreatorId: String => Decoder.Result[persons.ResourceId] = + persons.ResourceId + .from(_) + .leftMap(ex => DecodingFailure(DecodingFailure.Reason.CustomReason(ex.getMessage), cur)) - def toListOfCreators(implicit topmostSameAs: TopmostSameAs): String => Decoder.Result[NonEmptyList[PersonInfo]] = - decode[PersonInfo](map = toCreator, sort = _.sortBy(_.name))(_) + def toListOfCreators(implicit + topmostSameAs: TopmostSameAs + ): String => Decoder.Result[NonEmptyList[persons.ResourceId]] = + decode[persons.ResourceId](map = toCreatorId, sort = _.sorted)(_) .flatMap(toNonEmptyList(s"No creators found for $topmostSameAs")) val toImage: String => Decoder.Result[images.Image] = { diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/ontology.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/ontology.scala index b423a6a68f..27e0aa3905 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/ontology.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/datasets/ontology.scala @@ -40,7 +40,7 @@ object DatasetSearchInfoOntology { lazy val typeDef: Type = Type.Def( Class(renku / "DiscoverableDataset"), ObjectProperties( - ObjectProperty(creatorProperty, PersonInfoOntology.typeDef), + ObjectProperty(creatorProperty, Person.Ontology.typeDef), ObjectProperty(imageProperty, Image.Ontology.typeDef), ObjectProperty(linkProperty, LinkOntology.typeDef) ), diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ProjectSearchInfo.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ProjectSearchInfo.scala index 12c3f41e07..a5dfa84bf1 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ProjectSearchInfo.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ProjectSearchInfo.scala @@ -20,9 +20,8 @@ package io.renku.entities.searchgraphs.projects import cats.Show import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo import io.renku.graph.model.images.Image -import io.renku.graph.model.projects +import io.renku.graph.model.{persons, projects} private final case class ProjectSearchInfo(id: projects.ResourceId, name: projects.Name, @@ -30,7 +29,7 @@ private final case class ProjectSearchInfo(id: projects.ResourceId visibility: projects.Visibility, dateCreated: projects.DateCreated, dateModified: projects.DateModified, - maybeCreator: Option[PersonInfo], + maybeCreator: Option[persons.ResourceId], keywords: List[projects.Keyword], maybeDescription: Option[projects.Description], images: List[Image] diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractor.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractor.scala index 42caf138e2..96791dcb5e 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractor.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractor.scala @@ -18,7 +18,6 @@ package io.renku.entities.searchgraphs.projects -import io.renku.entities.searchgraphs.PersonInfo.toPersonInfo import io.renku.graph.model.entities.Project private object SearchInfoExtractor { @@ -31,7 +30,7 @@ private object SearchInfoExtractor { project.visibility, project.dateCreated, project.dateModified, - project.maybeCreator.map(toPersonInfo), + project.maybeCreator.map(_.resourceId), project.keywords.toList, project.maybeDescription, project.images diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/commands/Encoders.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/commands/Encoders.scala index e093783ee8..62b382fe0d 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/commands/Encoders.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/commands/Encoders.scala @@ -20,11 +20,9 @@ package io.renku.entities.searchgraphs.projects package commands import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo import io.renku.graph.model.Schemas.rdf -import io.renku.graph.model.entities.Person import io.renku.graph.model.images.Image -import io.renku.graph.model.projects +import io.renku.graph.model.{persons, projects} import io.renku.jsonld.Property import io.renku.jsonld.syntax._ import io.renku.triplesstore.client.model.{Quad, QuadsEncoder, TripleObject} @@ -32,14 +30,6 @@ import io.renku.triplesstore.client.syntax._ private object Encoders { - implicit val personInfoEncoder: QuadsEncoder[PersonInfo] = QuadsEncoder.instance { - case PersonInfo(resourceId, name) => - Set( - ProjectsQuad(resourceId, rdf / "type", Person.Ontology.typeClass.id), - ProjectsQuad(resourceId, Person.Ontology.nameProperty.id, name.asObject) - ) - } - implicit val imageEncoder: QuadsEncoder[Image] = QuadsEncoder.instance { case Image(resourceId, uri, position) => Set( ProjectsQuad(resourceId, rdf / "type", Image.Ontology.typeClass.id), @@ -56,8 +46,8 @@ private object Encoders { searchInfoQuad(ProjectSearchInfoOntology.descriptionProperty.id, d.asObject) } - val creatorQuads = info.maybeCreator.toSet.flatMap { (c: PersonInfo) => - c.asQuads + searchInfoQuad(ProjectSearchInfoOntology.creatorProperty, c.resourceId.asEntityId) + val creatorQuads = info.maybeCreator.toSet.map { (resourceId: persons.ResourceId) => + searchInfoQuad(ProjectSearchInfoOntology.creatorProperty, resourceId.asEntityId) } val keywordsQuads = info.keywords.toSet.map { (k: projects.Keyword) => diff --git a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ontology.scala b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ontology.scala index b6c64656b5..2062cbb487 100644 --- a/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ontology.scala +++ b/entities-search/src/main/scala/io/renku/entities/searchgraphs/projects/ontology.scala @@ -40,7 +40,7 @@ object ProjectSearchInfoOntology { lazy val typeDef: Type = Type.Def( Class(renku / "DiscoverableProject"), ObjectProperties( - ObjectProperty(creatorProperty, PersonInfoOntology.typeDef), + ObjectProperty(creatorProperty, Person.Ontology.typeDef), ObjectProperty(imageProperty, Image.Ontology.typeDef) ), DataProperties( @@ -56,17 +56,6 @@ object ProjectSearchInfoOntology { ) } -object PersonInfoOntology { - - val nameProperty: DataProperty.Def = Person.Ontology.nameProperty - - lazy val typeDef: Type = Type.Def( - Class(renku / "DiscoverableDatasetPerson"), - ObjectProperties(), - DataProperties(nameProperty) - ) -} - object LinkOntology { val project: Property = renku / "project" diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/Generators.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/Generators.scala index 02b3367c33..d1909a0f79 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/Generators.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/Generators.scala @@ -18,9 +18,6 @@ package io.renku.entities.searchgraphs -import io.renku.entities.searchgraphs.PersonInfo.toPersonInfo -import io.renku.graph.model.entities -import io.renku.graph.model.testentities._ import io.renku.triplesstore.SparqlQuery import io.renku.triplesstore.client.TriplesStoreGenerators.quads import io.renku.triplesstore.client.syntax._ @@ -28,9 +25,6 @@ import org.scalacheck.Gen private object Generators { - lazy val personInfos: Gen[PersonInfo] = - personEntities.map(_.to[entities.Person]).map(toPersonInfo) - val insertUpdateCommands: Gen[UpdateCommand] = quads.map(UpdateCommand.Insert) val deleteUpdateCommands: Gen[UpdateCommand] = diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/PersonInfoSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/PersonInfoSpec.scala deleted file mode 100644 index efdc9408ca..0000000000 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/PersonInfoSpec.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Copyright 2023 Swiss Data Science Center (SDSC) - * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and - * Eidgenössische Technische Hochschule Zürich (ETHZ). - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.renku.entities.searchgraphs - -import cats.syntax.all._ -import io.renku.entities.searchgraphs.Generators.personInfos -import org.scalatest.matchers.should -import org.scalatest.wordspec.AnyWordSpec -import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks - -class PersonInfoSpec extends AnyWordSpec with should.Matchers with ScalaCheckPropertyChecks { - - "show" should { - - "returns String representation" in { - forAll(personInfos) { case info @ PersonInfo(id, name) => - info.show shouldBe show"id = $id, name = $name" - } - } - } -} diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/Generators.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/Generators.scala index e835edaebe..40b4171a60 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/Generators.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/Generators.scala @@ -20,7 +20,6 @@ package io.renku.entities.searchgraphs.datasets import cats.data.NonEmptyList import cats.syntax.all._ -import io.renku.entities.searchgraphs import io.renku.entities.searchgraphs.datasets.SearchInfoLens._ import io.renku.generators.Generators.Implicits._ import io.renku.graph.model.datasets.TopmostSameAs @@ -37,7 +36,7 @@ private object Generators { createdOrPublished <- datasetCreatedOrPublished visibility <- projectVisibilities maybeDateModified <- datasetModifiedDates(notYoungerThan = createdOrPublished).toGeneratorOfOptions - creators <- searchgraphs.Generators.personInfos.toGeneratorOfNonEmptyList(max = 2) + creators <- personResourceIds.toGeneratorOfNonEmptyList(max = 2) keywords <- datasetKeywords.toGeneratorOfList(max = 2) maybeDesc <- datasetDescriptions.toGeneratorOfOptions images <- imageUris.toGeneratorOfList(max = 2).map(_.toEntitiesImages(datasetResourceIds.generateOne)) diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractorSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractorSpec.scala index 6bd1d1cf51..0739d1da0f 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractorSpec.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/SearchInfoExtractorSpec.scala @@ -20,7 +20,6 @@ package io.renku.entities.searchgraphs.datasets import cats.data.NonEmptyList import cats.syntax.all._ -import io.renku.entities.searchgraphs.PersonInfo._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators._ import io.renku.graph.model.testentities._ @@ -51,7 +50,7 @@ class SearchInfoExtractorSpec extends AnyWordSpec with should.Matchers { project.visibility, ds.provenance.date, maybeDateModified = None, - ds.provenance.creators.map(toPersonInfo), + ds.provenance.creators.map(_.resourceId), ds.additionalInfo.keywords, ds.additionalInfo.maybeDescription, ds.additionalInfo.images, @@ -80,7 +79,7 @@ class SearchInfoExtractorSpec extends AnyWordSpec with should.Matchers { project.visibility, originalDataset.provenance.date, datasets.DateModified(lastModification.provenance.date).some, - lastModification.provenance.creators.map(toPersonInfo), + lastModification.provenance.creators.map(_.resourceId), lastModification.additionalInfo.keywords, lastModification.additionalInfo.maybeDescription, lastModification.additionalInfo.images, diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/EncodersSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/EncodersSpec.scala index 60a5f4d754..65520d2981 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/EncodersSpec.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/EncodersSpec.scala @@ -19,7 +19,6 @@ package io.renku.entities.searchgraphs.datasets package commands -import io.renku.entities.searchgraphs.Generators.personInfos import io.renku.entities.searchgraphs.datasets.Generators._ import io.renku.entities.searchgraphs.datasets.{DatasetSearchInfo, DatasetSearchInfoOntology, LinkOntology} import io.renku.generators.Generators.Implicits._ @@ -28,7 +27,6 @@ import io.renku.generators.jsonld.JsonLDGenerators.entityIds import io.renku.graph.model.GraphModelGenerators.{datasetTopmostSameAs, imageUris} import io.renku.graph.model.Schemas.{rdf, renku, schema} import io.renku.graph.model.datasets -import io.renku.graph.model.entities.Person import io.renku.graph.model.images.{Image, ImagePosition, ImageResourceId} import io.renku.jsonld.syntax._ import io.renku.triplesstore.client.model.Quad @@ -40,19 +38,6 @@ class EncodersSpec extends AnyWordSpec with should.Matchers { import io.renku.entities.searchgraphs.datasets.commands.Encoders._ - "personInfoEncoder" should { - - "turn a PersonInfo object into a Set of relevant Quads" in { - - val personInfo = personInfos.generateOne - - personInfo.asQuads shouldBe Set( - DatasetsQuad(personInfo.resourceId, rdf / "type", Person.Ontology.typeClass.id), - DatasetsQuad(personInfo.resourceId, Person.Ontology.nameProperty.id, personInfo.name.asObject) - ) - } - } - "imageEncoder" should { "turn an Image object into a Set of relevant Quads" in { @@ -139,15 +124,11 @@ class EncodersSpec extends AnyWordSpec with should.Matchers { private def creatorsToQuads(searchInfo: DatasetSearchInfo): Set[Quad] = searchInfo.creators - .map(pi => - pi.asQuads + DatasetsQuad(searchInfo.topmostSameAs, - DatasetSearchInfoOntology.creatorProperty, - pi.resourceId.asEntityId - ) + .map(creatorId => + DatasetsQuad(searchInfo.topmostSameAs, DatasetSearchInfoOntology.creatorProperty, creatorId.asEntityId) ) .toList .toSet - .flatten private def keywordsToQuads(searchInfo: DatasetSearchInfo): Set[Quad] = searchInfo.keywords diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcherSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcherSpec.scala index ba84029fcd..b551087db2 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcherSpec.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/datasets/commands/SearchInfoFetcherSpec.scala @@ -20,7 +20,6 @@ package io.renku.entities.searchgraphs.datasets package commands import cats.effect.IO -import io.renku.entities.searchgraphs.Generators.personInfos import io.renku.entities.searchgraphs.datasets.DatasetSearchInfo import io.renku.entities.searchgraphs.datasets.Generators._ import io.renku.entities.searchgraphs.datasets.commands.Encoders._ @@ -55,17 +54,6 @@ class SearchInfoFetcherSpec fetcher.fetchTSSearchInfos(projectId).unsafeRunSync() shouldBe infos.sortBy(_.name).map(orderValues) } - "work if there are ',' in names" in new TestCase { - - val infos = datasetSearchInfoObjects(withLinkTo = projectId) - .map(_.copy(creators = personInfos.map(_.copy(name = "name, surname")).generateNonEmptyList(max = 1))) - .generateFixedSizeList(ofSize = 1) - - insert(projectsDataset, infos.map(_.asQuads).toSet.flatten) - - fetcher.fetchTSSearchInfos(projectId).unsafeRunSync() shouldBe infos.sortBy(_.name).map(orderValues) - } - "return nothing if no Datasets for the Project" in new TestCase { insert(projectsDataset, datasetSearchInfoObjects.generateOne.asQuads) @@ -83,7 +71,7 @@ class SearchInfoFetcherSpec val fetcher = new SearchInfoFetcherImpl[IO](projectsDSConnectionInfo) } - private def orderValues(info: DatasetSearchInfo) = info.copy(creators = info.creators.sortBy(_.name), + private def orderValues(info: DatasetSearchInfo) = info.copy(creators = info.creators.sorted, keywords = info.keywords.sorted, images = info.images.sortBy(_.position), links = info.links.sortBy(_.projectId) diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/Generators.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/Generators.scala index 6a81f2cf90..972b79be51 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/Generators.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/Generators.scala @@ -18,7 +18,6 @@ package io.renku.entities.searchgraphs.projects -import io.renku.entities.searchgraphs import io.renku.generators.Generators.Implicits._ import io.renku.graph.model.testentities._ import io.renku.jsonld.syntax._ @@ -33,7 +32,7 @@ private object Generators { createdDate <- projectCreatedDates() modifiedDate <- projectModifiedDates(createdDate.value) visibility <- projectVisibilities - maybeCreator <- searchgraphs.Generators.personInfos.toGeneratorOfOptions + maybeCreator <- personResourceIds.toGeneratorOfOptions keywords <- projectKeywords.toGeneratorOfList(max = 2) maybeDesc <- projectDescriptions.toGeneratorOfOptions images <- imageUris.toGeneratorOfList(max = 2).map(convertImageUris(id.asEntityId)) diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractorSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractorSpec.scala index 3771793b99..3f63b991f3 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractorSpec.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/SearchInfoExtractorSpec.scala @@ -18,7 +18,6 @@ package io.renku.entities.searchgraphs.projects -import io.renku.entities.searchgraphs.PersonInfo import io.renku.generators.Generators.Implicits._ import io.renku.graph.model.entities import io.renku.graph.model.testentities._ @@ -38,7 +37,7 @@ class SearchInfoExtractorSpec extends AnyFlatSpec with should.Matchers { project.visibility, project.dateCreated, project.dateModified, - project.maybeCreator.map(PersonInfo.toPersonInfo), + project.maybeCreator.map(_.resourceId), project.keywords.toList, project.maybeDescription, project.images diff --git a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/commands/EncodersSpec.scala b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/commands/EncodersSpec.scala index fc131a8d00..36a5a6b67b 100644 --- a/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/commands/EncodersSpec.scala +++ b/entities-search/src/test/scala/io/renku/entities/searchgraphs/projects/commands/EncodersSpec.scala @@ -20,16 +20,13 @@ package io.renku.entities.searchgraphs.projects package commands import Generators._ -import io.renku.entities.searchgraphs.Generators.personInfos -import io.renku.entities.searchgraphs.PersonInfo import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.positiveInts import io.renku.generators.jsonld.JsonLDGenerators.entityIds import io.renku.graph.model.GraphModelGenerators.imageUris import io.renku.graph.model.Schemas.{rdf, renku, schema} -import io.renku.graph.model.entities.Person import io.renku.graph.model.images.{Image, ImagePosition, ImageResourceId} -import io.renku.graph.model.projects +import io.renku.graph.model.{persons, projects} import io.renku.jsonld.syntax._ import io.renku.triplesstore.client.model.Quad import io.renku.triplesstore.client.syntax._ @@ -40,19 +37,6 @@ class EncodersSpec extends AnyWordSpec with should.Matchers { import Encoders._ - "personInfoEncoder" should { - - "turn a PersonInfo object into a Set of relevant Quads" in { - - val personInfo = personInfos.generateOne - - personInfo.asQuads shouldBe Set( - ProjectsQuad(personInfo.resourceId, rdf / "type", Person.Ontology.typeClass.id), - ProjectsQuad(personInfo.resourceId, Person.Ontology.nameProperty.id, personInfo.name.asObject) - ) - } - } - "imageEncoder" should { "turn an Image object into a Set of relevant Quads" in { @@ -96,9 +80,9 @@ class EncodersSpec extends AnyWordSpec with should.Matchers { } private def creatorToQuads(searchInfo: ProjectSearchInfo): Set[Quad] = - searchInfo.maybeCreator.toSet.flatMap((pi: PersonInfo) => - pi.asQuads + ProjectsQuad(searchInfo.id, ProjectSearchInfoOntology.creatorProperty, pi.resourceId.asEntityId) - ) + searchInfo.maybeCreator.toSet.map { (resourceId: persons.ResourceId) => + ProjectsQuad(searchInfo.id, ProjectSearchInfoOntology.creatorProperty, resourceId.asEntityId) + } private def keywordsToQuads(searchInfo: ProjectSearchInfo): Set[Quad] = searchInfo.keywords diff --git a/entities-viewings-collector/src/main/scala/io/renku/entities/viewings/search/DatasetQuery.scala b/entities-viewings-collector/src/main/scala/io/renku/entities/viewings/search/DatasetQuery.scala index 0340627c3f..0e424a5374 100644 --- a/entities-viewings-collector/src/main/scala/io/renku/entities/viewings/search/DatasetQuery.scala +++ b/entities-viewings-collector/src/main/scala/io/renku/entities/viewings/search/DatasetQuery.scala @@ -83,7 +83,10 @@ object DatasetQuery extends (Criteria => Option[SparqlQuery]) { | } | | Optional { - | ${v.datasetSameAs} schema:creator / schema:name ?creatorName. + | ${v.datasetSameAs} schema:creator ?creatorId. + | GRAPH ${GraphClass.Persons.id} { + | ?creatorId schema:name ?creatorName + | } | } | Optional { | ${v.datasetSameAs} schema:keywords ?keyword diff --git a/graph-commons/src/test/scala/io/renku/triplesstore/InMemoryJena.scala b/graph-commons/src/test/scala/io/renku/triplesstore/InMemoryJena.scala index a6e7fabdff..83aa0e8c98 100644 --- a/graph-commons/src/test/scala/io/renku/triplesstore/InMemoryJena.scala +++ b/graph-commons/src/test/scala/io/renku/triplesstore/InMemoryJena.scala @@ -238,6 +238,9 @@ sealed trait NamedGraphDataset { }) .unsafeRunSync() + def insertIO(to: DatasetName, quads: List[Quad]): IO[Unit] = + quads.map(insertIO(to, _)).sequence.void + def insertIO(to: DatasetName, quad: Quad): IO[Unit] = queryRunnerFor(to) >>= (_.runUpdate { SparqlQuery.of("insert quad", show"INSERT DATA { ${quad.asSparql.sparql} }") diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemover.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemover.scala new file mode 100644 index 0000000000..64dfd7f492 --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemover.scala @@ -0,0 +1,57 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest +package migrations + +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.graph.model.GraphClass +import io.renku.graph.model.Schemas.schema +import io.renku.metrics.MetricsRegistry +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore.client.syntax._ +import io.renku.triplesstore.{SparqlQuery, SparqlQueryTimeRecorder} +import migrations.tooling.RegisteredUpdateQueryMigration +import org.typelevel.log4cats.Logger + +private object DatasetsGraphPersonRemover { + + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder: MetricsRegistry]: F[Migration[F]] = + RegisteredUpdateQueryMigration[F](name, query).widen + + private lazy val name = Migration.Name("DiscoverableDatasetPerson remover") + + private[migrations] lazy val query = SparqlQuery.of( + name.asRefined, + Prefixes of schema -> "schema", + sparql"""|DELETE { + | GRAPH ${GraphClass.Datasets.id} { + | ?id ?p ?o + | } + |} + |WHERE { + | GRAPH ${GraphClass.Datasets.id} { + | ?id a schema:Person; + | ?p ?o + | } + |} + |""".stripMargin + ) +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/Migrations.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/Migrations.scala index ea5465bb23..1c4ce07993 100644 --- a/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/Migrations.scala +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/Migrations.scala @@ -48,6 +48,8 @@ private[tsmigrationrequest] object Migrations { addProjectDateModified <- datemodified.AddProjectDateModified[F] fixMultipleProjectVersions <- FixMultipleProjectVersions[F] addProjectSlug <- projectslug.AddProjectSlug[F] + datasetsGraphPersonRemover <- DatasetsGraphPersonRemover[F] + projectsGraphPersonRemover <- ProjectsGraphPersonRemover[F] migrations <- validateNames( datasetsCreator, datasetsRemover, @@ -63,7 +65,9 @@ private[tsmigrationrequest] object Migrations { provisionProjectsGraph, addProjectDateModified, fixMultipleProjectVersions, - addProjectSlug + addProjectSlug, + datasetsGraphPersonRemover, + projectsGraphPersonRemover ) } yield migrations diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemover.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemover.scala new file mode 100644 index 0000000000..b2603c9b43 --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemover.scala @@ -0,0 +1,57 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest +package migrations + +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.graph.model.GraphClass +import io.renku.graph.model.Schemas.schema +import io.renku.metrics.MetricsRegistry +import migrations.tooling.RegisteredUpdateQueryMigration +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore.client.syntax._ +import io.renku.triplesstore.{SparqlQuery, SparqlQueryTimeRecorder} +import org.typelevel.log4cats.Logger + +private object ProjectsGraphPersonRemover { + + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder: MetricsRegistry]: F[Migration[F]] = + RegisteredUpdateQueryMigration[F](name, query).widen + + private lazy val name = Migration.Name("Projects graph Person remover") + + private[migrations] lazy val query = SparqlQuery.of( + name.asRefined, + Prefixes of schema -> "schema", + sparql"""|DELETE { + | GRAPH ${GraphClass.Projects.id} { + | ?id ?p ?o + | } + |} + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ?id a schema:Person; + | ?p ?o + | } + |} + |""".stripMargin + ) +} diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigration.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigration.scala new file mode 100644 index 0000000000..23ed08b527 --- /dev/null +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigration.scala @@ -0,0 +1,55 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest +package migrations.tooling + +import cats.MonadThrow +import cats.data.EitherT +import cats.effect.Async +import cats.syntax.all._ +import io.renku.triplesgenerator.events.consumers.ProcessingRecoverableError +import io.renku.triplesstore.{SparqlQuery, SparqlQueryTimeRecorder} +import org.typelevel.log4cats.Logger + +private[migrations] class RegisteredUpdateQueryMigration[F[_]: MonadThrow: Logger]( + override val name: Migration.Name, + updateQuery: SparqlQuery, + executionRegister: MigrationExecutionRegister[F], + updateQueryRunner: UpdateQueryRunner[F], + recoveryStrategy: RecoverableErrorsRecovery = RecoverableErrorsRecovery +) extends RegisteredMigration[F](name, executionRegister, recoveryStrategy) { + import recoveryStrategy._ + + protected[tooling] override def migrate(): EitherT[F, ProcessingRecoverableError, Unit] = EitherT { + updateQueryRunner + .run(updateQuery) + .map(_.asRight[ProcessingRecoverableError]) + .recoverWith(maybeRecoverableError[F, Unit]) + } +} + +private[migrations] object RegisteredUpdateQueryMigration { + + def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]( + name: Migration.Name, + updateQuery: SparqlQuery + ): F[RegisteredUpdateQueryMigration[F]] = + (MigrationExecutionRegister[F], UpdateQueryRunner[F]) + .mapN(new RegisteredUpdateQueryMigration[F](name, updateQuery, _, _)) +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemoverSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemoverSpec.scala new file mode 100644 index 0000000000..ef5a5baac5 --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/DatasetsGraphPersonRemoverSpec.scala @@ -0,0 +1,106 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest.migrations + +import cats.data.NonEmptyList +import cats.effect.IO +import eu.timepit.refined.auto._ +import io.renku.entities.searchgraphs.SearchInfoDatasets +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model._ +import io.renku.graph.model.testentities._ +import io.renku.interpreters.TestLogger +import io.renku.jsonld.syntax._ +import io.renku.logging.TestSparqlQueryTimeRecorder +import io.renku.metrics.TestMetricsRegistry +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore._ +import io.renku.triplesstore.client.model.Quad +import io.renku.triplesstore.client.syntax._ +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.typelevel.log4cats.Logger +import tooling.RegisteredUpdateQueryMigration + +class DatasetsGraphPersonRemoverSpec + extends AsyncFlatSpec + with CustomAsyncIOSpec + with should.Matchers + with InMemoryJenaForSpec + with ProjectsDataset + with SearchInfoDatasets + with AsyncMockFactory { + + it should "be a RegisteredUpdateQueryMigration" in { + implicit val metricsRegistry: TestMetricsRegistry[IO] = TestMetricsRegistry[IO] + implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() + + DatasetsGraphPersonRemover[IO].asserting( + _.getClass shouldBe classOf[RegisteredUpdateQueryMigration[IO]] + ) + } + + it should "remove Person entities from the Datasets graph" in { + + val ds1 -> project1 = anyRenkuProjectEntities.addDataset(datasetEntities(provenanceInternal)).generateOne + val ds2 -> project2 = anyRenkuProjectEntities.addDataset(datasetEntities(provenanceInternal)).generateOne + + val allNames = (ds1.provenance.creators.toList ::: ds2.provenance.creators.toList).map(_.name).toSet + + provisionTestProjects(project1, project2).assertNoException >> + insertIO(projectsDataset, toQuads(ds1.provenance.creators)).assertNoException >> + insertIO(projectsDataset, toQuads(ds2.provenance.creators)).assertNoException >> + fetchCreatorsNames.asserting(_.toSet shouldBe allNames) >> + runUpdate(projectsDataset, DatasetsGraphPersonRemover.query).assertNoException >> + fetchCreatorsNames.asserting(_ shouldBe List.empty) + } + + private def toQuads(persons: NonEmptyList[Person]): List[Quad] = + persons.toList.flatMap(toSinglePersonQuads) + + private def toSinglePersonQuads(person: Person): List[Quad] = List( + Quad(GraphClass.Datasets.id, person.resourceId.asEntityId, rdf / "type", entities.Person.Ontology.typeClass.id), + Quad(GraphClass.Datasets.id, + person.resourceId.asEntityId, + entities.Person.Ontology.nameProperty.id, + person.name.asObject + ) + ) + + private def fetchCreatorsNames: IO[List[persons.Name]] = + runSelect( + on = projectsDataset, + SparqlQuery.ofUnsafe( + "test creator name", + Prefixes of schema -> "schema", + sparql"""|SELECT ?name + |WHERE { + | GRAPH ${GraphClass.Datasets.id} { + | ?id a schema:Person; + | schema:name ?name + | } + |} + |""".stripMargin + ) + ).map(_.flatMap(_.get("name").map(persons.Name))) + + implicit override lazy val ioLogger: Logger[IO] = TestLogger[IO]() +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemoverSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemoverSpec.scala new file mode 100644 index 0000000000..bcf31f806d --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/ProjectsGraphPersonRemoverSpec.scala @@ -0,0 +1,104 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest.migrations + +import cats.effect.IO +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.entities.searchgraphs.SearchInfoDatasets +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model._ +import io.renku.graph.model.testentities._ +import io.renku.interpreters.TestLogger +import io.renku.jsonld.syntax._ +import io.renku.logging.TestSparqlQueryTimeRecorder +import io.renku.metrics.TestMetricsRegistry +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesstore.SparqlQuery.Prefixes +import io.renku.triplesstore._ +import io.renku.triplesstore.client.model.Quad +import io.renku.triplesstore.client.syntax._ +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.typelevel.log4cats.Logger +import tooling.RegisteredUpdateQueryMigration + +class ProjectsGraphPersonRemoverSpec + extends AsyncFlatSpec + with CustomAsyncIOSpec + with should.Matchers + with InMemoryJenaForSpec + with ProjectsDataset + with SearchInfoDatasets + with AsyncMockFactory { + + it should "be a RegisteredUpdateQueryMigration" in { + implicit val metricsRegistry: TestMetricsRegistry[IO] = TestMetricsRegistry[IO] + implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() + + ProjectsGraphPersonRemover[IO].asserting( + _.getClass shouldBe classOf[RegisteredUpdateQueryMigration[IO]] + ) + } + + it should "remove Person entities from the Projects graph" in { + + val project1Creator = personEntities(withGitLabId).generateOne + val project1 = anyProjectEntities.map(replaceProjectCreator(project1Creator.some)).generateOne.to[entities.Project] + val project2Creator = personEntities(withGitLabId).generateOne + val project2 = anyProjectEntities.map(replaceProjectCreator(project2Creator.some)).generateOne.to[entities.Project] + + provisionProjects(project1, project2).assertNoException >> + insertIO(projectsDataset, toQuads(project1Creator)).assertNoException >> + insertIO(projectsDataset, toQuads(project2Creator)).assertNoException >> + fetchCreatorName(project1Creator.resourceId).asserting(_ shouldBe List(project1Creator.name)) >> + fetchCreatorName(project2Creator.resourceId).asserting(_ shouldBe List(project2Creator.name)) >> + runUpdate(projectsDataset, ProjectsGraphPersonRemover.query).assertNoException >> + fetchCreatorName(project1Creator.resourceId).asserting(_ shouldBe List.empty) >> + fetchCreatorName(project2Creator.resourceId).asserting(_ shouldBe List.empty) + } + + private def toQuads(person: Person): List[Quad] = List( + Quad(GraphClass.Projects.id, person.resourceId.asEntityId, rdf / "type", entities.Person.Ontology.typeClass.id), + Quad(GraphClass.Projects.id, + person.resourceId.asEntityId, + entities.Person.Ontology.nameProperty.id, + person.name.asObject + ) + ) + + private def fetchCreatorName(personId: persons.ResourceId): IO[List[persons.Name]] = + runSelect( + on = projectsDataset, + SparqlQuery.ofUnsafe( + "test creator name", + Prefixes of schema -> "schema", + sparql"""|SELECT ?name + |WHERE { + | GRAPH ${GraphClass.Projects.id} { + | ${personId.asEntityId} schema:name ?name + | } + |} + |""".stripMargin + ) + ).map(_.flatMap(_.get("name").map(persons.Name))) + + implicit override lazy val ioLogger: Logger[IO] = TestLogger[IO]() +} diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigrationSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigrationSpec.scala new file mode 100644 index 0000000000..e8f8bebc4b --- /dev/null +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/events/consumers/tsmigrationrequest/migrations/tooling/RegisteredUpdateQueryMigrationSpec.scala @@ -0,0 +1,88 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.triplesgenerator.events.consumers.tsmigrationrequest +package migrations.tooling + +import Generators.migrationNames +import cats.MonadThrow +import cats.effect.IO +import cats.syntax.all._ +import io.renku.generators.CommonGraphGenerators.sparqlQueries +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.exceptions +import io.renku.interpreters.TestLogger +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesgenerator.generators.ErrorGenerators.processingRecoverableErrors +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.EitherValues +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec +import org.typelevel.log4cats.Logger + +class RegisteredUpdateQueryMigrationSpec + extends AsyncWordSpec + with CustomAsyncIOSpec + with AsyncMockFactory + with should.Matchers + with EitherValues { + + "RegisteredUpdateQueryMigration" should { + + "be the RegisteredMigration" in { + migration.getClass.getSuperclass shouldBe classOf[RegisteredMigration[IO]] + } + } + + "migrate" should { + + "execute the defined query" in { + + (queryRunner.run _).expects(query).returning(().pure[IO]) + + migration.migrate().value.asserting(_.value shouldBe ()) + } + + "return a Recoverable Error if in case of an exception the given strategy returns one" in { + + val exception = exceptions.generateOne + (queryRunner.run _) + .expects(query) + .returning(exception.raiseError[IO, Unit]) + + migration.migrate().value.asserting(_.left.value shouldBe recoverableError) + } + } + + private lazy val query = sparqlQueries.generateOne + private lazy val queryRunner = mock[UpdateQueryRunner[IO]] + private lazy val executionRegister = mock[MigrationExecutionRegister[IO]] + private lazy val recoverableError = processingRecoverableErrors.generateOne + private lazy val recoveryStrategy = new RecoverableErrorsRecovery { + override def maybeRecoverableError[F[_]: MonadThrow, OUT]: RecoveryStrategy[F, OUT] = { _ => + recoverableError.asLeft[OUT].pure[F] + } + } + private implicit lazy val logger: Logger[IO] = TestLogger[IO]() + private lazy val migration = new RegisteredUpdateQueryMigration[IO](migrationNames.generateOne, + query, + executionRegister, + queryRunner, + recoveryStrategy + ) +} From 4e9d4181c17e0328c6bfad6ef04ebf7c8ea08e6c Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 11 Aug 2023 19:29:51 +0200 Subject: [PATCH 11/49] feat: RenkuCoreClient.getMigrationCheck; HttpUrl moved to projects file --- .../projects/details/EndpointDocs.scala | 4 +- .../details/GitLabProjectFinder.scala | 8 ++-- .../projects/details/model.scala | 18 +------- .../details/ProjectJsonEncoderSpec.scala | 2 +- .../projects/details/ProjectsGenerators.scala | 9 +--- .../projects/details/modelSpec.scala | 21 +--------- .../core/client/ProjectMigrationCheck.scala | 38 +++++++++++++++++ .../renku/core/client/RenkuCoreClient.scala | 34 ++++++++++++--- .../scala/io/renku/core/client/model.scala | 9 +++- .../io/renku/core/client/Generators.scala | 6 +++ .../io/renku/core/client/ModelEncoders.scala | 9 ++++ .../client/ProjectMigrationCheckSpec.scala | 41 +++++++++++++++++++ .../core/client/RenkuCoreClientSpec.scala | 24 +++++++---- .../scala/io/renku/graph/model/projects.scala | 15 ++++++- .../graph/model/RenkuTinyTypeGenerators.scala | 5 +++ .../io/renku/graph/model/projectsSpec.scala | 25 +++++++++++ .../tinytypes/json/TinyTypeDecoders.scala | 5 +++ 17 files changed, 208 insertions(+), 65 deletions(-) create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/ProjectMigrationCheck.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/EndpointDocs.scala index 2f45dfab81..cf24ae0a4d 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/EndpointDocs.scala @@ -33,7 +33,7 @@ import model.Forking.ForksCount import model.Permissions.{AccessLevel, GroupAccessLevel} import model.Project.StarsCount import model.Statistics.{CommitsCount, JobArtifactsSize, LsfObjectsSize, RepositorySize, StorageSize} -import model.Urls.{HttpUrl, ReadmeUrl, SshUrl, WebUrl} +import model.Urls.{ReadmeUrl, SshUrl, WebUrl} import model._ import java.time.Instant @@ -104,7 +104,7 @@ private class EndpointDocsImpl(projectJsonEncoder: ProjectJsonEncoder, projectJs projects.DateModified(Instant.parse("2012-11-16T10:00:00.000Z")), Urls( SshUrl("git@github.com:namespace/name.git"), - HttpUrl("https://github.com/namespace/name.git"), + projects.GitHttpUrl("https://github.com/namespace/name.git"), WebUrl("https://github.com/namespace/name"), ReadmeUrl("https://github.com/namespace/name/README.md").some ), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/GitLabProjectFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/GitLabProjectFinder.scala index dc86b5ec21..38f2a91f85 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/GitLabProjectFinder.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/GitLabProjectFinder.scala @@ -23,7 +23,7 @@ import cats.effect.kernel.Async import cats.syntax.all._ import eu.timepit.refined.auto._ import io.renku.graph.model.projects -import io.renku.graph.model.projects.{GitLabId, Visibility} +import io.renku.graph.model.projects.Visibility import io.renku.http.client.{AccessToken, GitLabClient} import model.Forking.ForksCount import model.Project.StarsCount @@ -103,9 +103,9 @@ private class GitLabProjectFinderImpl[F[_]: Async: GitLabClient: Logger] extends implicit val decoder: Decoder[GitLabProject] = cursor => for { - id <- cursor.downField("id").as[GitLabId] + id <- cursor.downField("id").as[projects.GitLabId] sshUrl <- cursor.downField("ssh_url_to_repo").as[SshUrl] - httpUrl <- cursor.downField("http_url_to_repo").as[HttpUrl] + httpUrl <- cursor.downField("http_url_to_repo").as[projects.GitHttpUrl] webUrl <- cursor.downField("web_url").as[WebUrl] maybeReadmeUrl <- cursor.downField("readme_url").as[Option[ReadmeUrl]] forksCount <- cursor.downField("forks_count").as[ForksCount] @@ -130,7 +130,7 @@ private class GitLabProjectFinderImpl[F[_]: Async: GitLabClient: Logger] extends private object GitLabProjectFinder { final case class GitLabProject( - id: GitLabId, + id: projects.GitLabId, visibility: Visibility, urls: Urls, forksCount: ForksCount, diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/model.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/model.scala index 6f1d275847..2c4a96b0ee 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/model.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/details/model.scala @@ -18,7 +18,6 @@ package io.renku.knowledgegraph.projects.details -import cats.data.Validated import cats.syntax.all._ import eu.timepit.refined.api.Refined import eu.timepit.refined.auto._ @@ -27,14 +26,12 @@ import eu.timepit.refined.numeric.Positive import io.circe.Decoder import io.renku.graph.model.images.ImageUri import io.renku.graph.model.persons -import io.renku.graph.model.projects.{DateCreated, DateModified, Description, GitLabId, Keyword, Name, Slug, ResourceId, Visibility} +import io.renku.graph.model.projects.{DateCreated, DateModified, Description, GitLabId, GitHttpUrl, Keyword, Name, ResourceId, Slug, Visibility} import io.renku.graph.model.versions.SchemaVersion import io.renku.tinytypes._ import io.renku.tinytypes.constraints._ import model.Statistics._ -import java.net.{MalformedURLException, URL} - private object model { import Forking.ForksCount import Project._ @@ -170,7 +167,7 @@ private object model { with NonNegativeLong[JobArtifactsSize] } - final case class Urls(ssh: SshUrl, http: HttpUrl, web: WebUrl, maybeReadme: Option[ReadmeUrl]) + final case class Urls(ssh: SshUrl, http: GitHttpUrl, web: WebUrl, maybeReadme: Option[ReadmeUrl]) object Urls { @@ -182,17 +179,6 @@ private object model { ) } - final class HttpUrl private (val value: String) extends AnyVal with StringTinyType - implicit object HttpUrl extends TinyTypeFactory[HttpUrl](new HttpUrl(_)) with NonBlank[HttpUrl] { - addConstraint( - check = url => - (url endsWith ".git") && Validated - .catchOnly[MalformedURLException](new URL(url)) - .isValid, - message = url => s"$url is not a valid repository http url" - ) - } - final class WebUrl private (val value: String) extends AnyVal with StringTinyType implicit object WebUrl extends TinyTypeFactory[WebUrl](new WebUrl(_)) with Url[WebUrl] diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala index 3cd5c33239..35ba64d7c2 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala @@ -139,7 +139,7 @@ class ProjectJsonEncoderSpec extends AnyWordSpec with should.Matchers with Scala private implicit lazy val urlsDecoder: Decoder[Urls] = cursor => for { ssh <- cursor.downField("ssh").as[SshUrl] - http <- cursor.downField("http").as[HttpUrl] + http <- cursor.downField("http").as[GitHttpUrl] web <- cursor.downField("web").as[WebUrl] maybeReadme <- cursor.downField("readme").as[Option[ReadmeUrl]] } yield Urls(ssh, http, web, maybeReadme) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectsGenerators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectsGenerators.scala index dc176f7b4c..20779f8f3c 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectsGenerators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectsGenerators.scala @@ -28,7 +28,7 @@ import model.Forking.ForksCount import model.Permissions._ import model.Project.StarsCount import model.Statistics.{CommitsCount, JobArtifactsSize, LsfObjectsSize, RepositorySize, StorageSize} -import model.Urls.{HttpUrl, ReadmeUrl, SshUrl, WebUrl} +import model.Urls.{ReadmeUrl, SshUrl, WebUrl} import model._ import org.scalacheck.Gen @@ -84,7 +84,7 @@ private object ProjectsGenerators { implicit lazy val urlsObjects: Gen[Urls] = for { sshUrl <- sshUrls - httpUrl <- httpUrls + httpUrl <- projectGitHttpUrls webUrl <- webUrls maybeReadmeUrl <- readmeUrls.toGeneratorOfOptions } yield Urls(sshUrl, httpUrl, webUrl, maybeReadmeUrl) @@ -97,11 +97,6 @@ private object ProjectsGenerators { projectSlug <- projectSlugs } yield SshUrl(s"git@${hostParts.toList.mkString(".")}:$projectSlug.git") - private implicit lazy val httpUrls: Gen[HttpUrl] = for { - url <- urls() - projectSlug <- projectSlugs - } yield HttpUrl(s"$url/$projectSlug.git") - private implicit lazy val readmeUrls: Gen[ReadmeUrl] = for { url <- urls() projectSlug <- projectSlugs diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/modelSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/modelSpec.scala index 791bff9c83..eda516c239 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/modelSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/modelSpec.scala @@ -23,32 +23,13 @@ import io.renku.generators.Generators._ import io.renku.graph.model.GraphModelGenerators._ import model.Permissions.AccessLevel import model.Permissions.AccessLevel._ -import model.Urls.{HttpUrl, SshUrl} +import model.Urls.SshUrl import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpec import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks class modelSpec extends AnyWordSpec with ScalaCheckPropertyChecks with should.Matchers { - "HttpUrl" should { - - "instantiate for valid absolute git urls" in { - forAll(httpUrls(), projectSlugs) { (httpUrl, projectSlug) => - val url = s"$httpUrl/$projectSlug.git" - HttpUrl.from(url).map(_.value) shouldBe Right(url) - } - } - - "fail instantiation for non-absolute urls" in { - val url = s"${relativePaths().generateOne}/${projectSlugs.generateOne}.git" - - val Left(exception) = HttpUrl.from(url) - - exception shouldBe an[IllegalArgumentException] - exception.getMessage shouldBe s"$url is not a valid repository http url" - } - } - "SshUrl" should { "instantiate for valid absolute ssh urls" in { diff --git a/renku-core-client/src/main/scala/io/renku/core/client/ProjectMigrationCheck.scala b/renku-core-client/src/main/scala/io/renku/core/client/ProjectMigrationCheck.scala new file mode 100644 index 0000000000..b615ccb73c --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/ProjectMigrationCheck.scala @@ -0,0 +1,38 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.syntax.all._ +import io.circe.Decoder +import io.renku.graph.model.versions.SchemaVersion + +final case class ProjectMigrationCheck(schemaVersion: SchemaVersion, migrationRequired: MigrationRequired) + +object ProjectMigrationCheck { + + implicit val decoder: Decoder[ProjectMigrationCheck] = + Decoder.instance { cur => + val statusCur = cur.downField("core_compatibility_status") + + ( + statusCur.downField("project_metadata_version").as[SchemaVersion], + statusCur.downField("migration_required").as[MigrationRequired] + ).mapN(ProjectMigrationCheck.apply) + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index 8d10613341..de1e4f7439 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -22,28 +22,50 @@ import cats.effect.Async import cats.syntax.all._ import com.typesafe.config.{Config, ConfigFactory} import io.renku.control.Throttler -import io.renku.http.client.RestClient +import io.renku.graph.model.projects +import io.renku.http.client.{AccessToken, RestClient} +import org.http4s.Header import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger +import org.typelevel.ci._ -trait RenkuCoreClient[F[_]] {} +trait RenkuCoreClient[F[_]] { + def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, accessToken: AccessToken): F[Result[ProjectMigrationCheck]] +} object RenkuCoreClient { def apply[F[_]: Async: Logger](config: Config = ConfigFactory.load): F[RenkuCoreClient[F]] = for { coreCurrentUri <- RenkuCoreUri.Current.loadFromConfig[F](config) versionClient = RenkuCoreVersionClient[F](coreCurrentUri, config) - } yield new RenkuCoreClientImpl[F](coreCurrentUri, versionClient) + } yield new RenkuCoreClientImpl[F](coreCurrentUri, versionClient, ClientTools[F]) } -private class RenkuCoreClientImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, - versionClient: RenkuCoreVersionClient[F] +private class RenkuCoreClientImpl[F[_]: Async: Logger](currentUri: RenkuCoreUri.Current, + versionClient: RenkuCoreVersionClient[F], + clientTools: ClientTools[F] ) extends RestClient[F, Nothing](Throttler.noThrottling) with RenkuCoreClient[F] with Http4sDsl[F] with Http4sClientDsl[F] { - println(coreCurrentUri) + import clientTools._ + println(versionClient) + + override def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): F[Result[ProjectMigrationCheck]] = { + + val uri = (currentUri.uri / "renku" / "cache.migrations_check") + .withQueryParam("git_url", projectGitHttpUrl.value) + + send(GET(uri).withHeaders(Header.Raw(ci"gitlab-token", accessToken.value))) { + case (Ok, _, resp) => + toResult[ProjectMigrationCheck](resp) + case reqInfo => + toFailure[ProjectMigrationCheck](s"Migration check for $projectGitHttpUrl failed")(reqInfo) + } + } } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/model.scala b/renku-core-client/src/main/scala/io/renku/core/client/model.scala index 6ab5a93cf2..b375b5d53f 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/model.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/model.scala @@ -21,9 +21,16 @@ package io.renku.core.client import io.circe.Decoder import io.renku.tinytypes.constraints.NonBlank import io.renku.tinytypes.json.TinyTypeDecoders -import io.renku.tinytypes.{StringTinyType, TinyTypeFactory} +import io.renku.tinytypes.{BooleanTinyType, StringTinyType, TinyTypeFactory} final class ApiVersion private (val value: String) extends AnyVal with StringTinyType object ApiVersion extends TinyTypeFactory[ApiVersion](new ApiVersion(_)) with NonBlank[ApiVersion] { implicit val decoder: Decoder[ApiVersion] = TinyTypeDecoders.stringDecoder(ApiVersion) } + +final class MigrationRequired private (val value: Boolean) extends AnyVal with BooleanTinyType +object MigrationRequired extends TinyTypeFactory[MigrationRequired](new MigrationRequired(_)) { + lazy val yes: MigrationRequired = MigrationRequired(true) + lazy val no: MigrationRequired = MigrationRequired(false) + implicit val decoder: Decoder[MigrationRequired] = TinyTypeDecoders.booleanDecoder(MigrationRequired) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index cf24d6aa6d..f603a28514 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -37,6 +37,9 @@ object Generators { implicit lazy val apiVersions: Gen[ApiVersion] = (positiveInts(), positiveInts()).mapN((major, minor) => ApiVersion(s"$major.$minor")) + implicit lazy val migrationRequiredGen: Gen[MigrationRequired] = + Gen.oneOf(MigrationRequired.yes, MigrationRequired.no) + implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = for { baseUri <- httpUrls() @@ -51,4 +54,7 @@ object Generators { implicit lazy val schemaApiVersions: Gen[SchemaApiVersions] = (apiVersions, apiVersions, cliVersions).mapN(SchemaApiVersions.apply) + + implicit lazy val projectMigrationChecks: Gen[ProjectMigrationCheck] = + (projectSchemaVersions, migrationRequiredGen).mapN(ProjectMigrationCheck.apply) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala index cda6ecb0ee..36545ddbeb 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala @@ -50,6 +50,15 @@ private object ModelEncoders { }""" } + implicit val projectMigrationCheckEnc: Encoder[ProjectMigrationCheck] = Encoder.instance { + case ProjectMigrationCheck(schemaVersion, migrationRequired) => json"""{ + "core_compatibility_status": { + "project_metadata_version": $schemaVersion, + "migration_required": $migrationRequired + } + }""" + } + implicit def resultEncoder[T](implicit enc: Encoder[T]): Encoder[Result[T]] = Encoder.instance { case Result.Success(obj) => json"""{ diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala new file mode 100644 index 0000000000..5c2536397d --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala @@ -0,0 +1,41 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators._ +import ModelEncoders.projectMigrationCheckEnc +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import org.scalatest.EitherValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should +import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks + +class ProjectMigrationCheckSpec + extends AnyFlatSpec + with should.Matchers + with EitherValues + with ScalaCheckPropertyChecks { + + it should "decode from JSON" in { + forAll { migrationCheck: ProjectMigrationCheck => + migrationCheck.asJson.hcursor.as[ProjectMigrationCheck].value shouldBe migrationCheck + } + } +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index 392df2bdfd..4738e05b1d 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -18,14 +18,18 @@ package io.renku.core.client +import Generators.projectMigrationChecks +import ModelEncoders._ import cats.effect.IO +import com.github.tomakehurst.wiremock.client.WireMock._ +import io.circe.syntax._ +import io.renku.generators.CommonGraphGenerators.accessTokens import io.renku.generators.Generators.Implicits._ -import io.renku.graph.model.GraphModelGenerators.{projectSchemaVersions, projectSlugs} +import io.renku.graph.model.GraphModelGenerators.projectGitHttpUrls import io.renku.interpreters.TestLogger import io.renku.stubbing.ExternalServiceStubbing import io.renku.testtools.CustomAsyncIOSpec import org.scalamock.scalatest.AsyncMockFactory -import com.github.tomakehurst.wiremock.client.WireMock._ import org.scalatest.matchers.should import org.scalatest.wordspec.AsyncWordSpec import org.scalatest.{EitherValues, OptionValues} @@ -44,19 +48,25 @@ class RenkuCoreClientSpec "return info about current project schema version" in { - val schemaVersion = projectSchemaVersions.generateOne - val projectSlug = projectSlugs.generateOne + val accessToken = accessTokens.generateOne + val projectGitHttpUrl = projectGitHttpUrls.generateOne + val migrationCheck = projectMigrationChecks.generateOne stubFor { - get(s"/renku/cache.migrations_check").withQueryParam("git_url", ) - .willReturn(ok(Result.success(versionTuples).asJson.spaces2)) + get(urlPathEqualTo("/renku/cache.migrations_check")) + .withQueryParam("git_url", equalTo(projectGitHttpUrl.value)) + .withHeader("gitlab-token", equalTo(accessToken.value)) + .willReturn(ok(Result.success(migrationCheck).asJson.spaces2)) } + + client.getMigrationCheck(projectGitHttpUrl, accessToken).asserting(_ shouldBe Result.success(migrationCheck)) } } private implicit val logger: Logger[IO] = TestLogger() private val coreVersionClient = mock[RenkuCoreVersionClient[IO]] - private lazy val client = new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), coreVersionClient) + private lazy val client = + new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), coreVersionClient, ClientTools[IO]) // private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = // (coreUriForSchemaLoader diff --git a/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala b/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala index 9413de0a58..5628c36bda 100644 --- a/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala +++ b/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala @@ -18,14 +18,16 @@ package io.renku.graph.model +import cats.data.Validated import cats.syntax.all._ import eu.timepit.refined.api.Refined import eu.timepit.refined.numeric.Positive import io.renku.graph.model.views.{EntityIdJsonLDOps, NonBlankTTJsonLDOps, TinyTypeJsonLDOps, UrlResourceRenderer} import io.renku.jsonld.{EntityId, JsonLDDecoder, JsonLDEncoder} -import io.renku.tinytypes.constraints._ import io.renku.tinytypes._ +import io.renku.tinytypes.constraints._ +import java.net.{MalformedURLException, URL} import java.time.Instant import java.time.temporal.ChronoUnit @@ -206,4 +208,15 @@ object projects { extends TinyTypeFactory[Keyword](new Keyword(_)) with NonBlank[Keyword] with NonBlankTTJsonLDOps[Keyword] + + final class GitHttpUrl private(val value: String) extends AnyVal with StringTinyType + implicit object GitHttpUrl extends TinyTypeFactory[GitHttpUrl](new GitHttpUrl(_)) with NonBlank[GitHttpUrl] { + addConstraint( + check = url => + (url endsWith ".git") && Validated + .catchOnly[MalformedURLException](new URL(url)) + .isValid, + message = url => s"$url is not a valid repository http url" + ) + } } diff --git a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/RenkuTinyTypeGenerators.scala b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/RenkuTinyTypeGenerators.scala index 1b62cdb9e8..f329abcbb4 100644 --- a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/RenkuTinyTypeGenerators.scala +++ b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/RenkuTinyTypeGenerators.scala @@ -151,6 +151,11 @@ trait RenkuTinyTypeGenerators { implicit val projectKeywords: Gen[projects.Keyword] = Generators.nonBlankStrings(minLength = 5).map(v => projects.Keyword(v.value)) + implicit lazy val projectGitHttpUrls: Gen[projects.GitHttpUrl] = for { + url <- httpUrls() + projectSlug <- projectSlugs + } yield projects.GitHttpUrl(s"$url/$projectSlug.git") + implicit val filePaths: Gen[projects.FilePath] = Generators.relativePaths() map projects.FilePath.apply implicit val datasetIdentifiers: Gen[datasets.Identifier] = Generators.noDashUuid.toGeneratorOf(datasets.Identifier) diff --git a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/projectsSpec.scala b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/projectsSpec.scala index b8d015111e..992b4ac41c 100644 --- a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/projectsSpec.scala +++ b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/projectsSpec.scala @@ -18,6 +18,7 @@ package io.renku.graph.model +import RenkuTinyTypeGenerators.projectSlugs import cats.syntax.all._ import io.circe.{DecodingFailure, Json} import io.renku.generators.Generators.Implicits._ @@ -28,6 +29,7 @@ import io.renku.tinytypes.constraints.{RelativePath, Url} import org.apache.jena.util.URIref import org.scalacheck.Gen import org.scalacheck.Gen.{alphaChar, const, frequency, numChar} +import org.scalatest.EitherValues import org.scalatest.matchers.should import org.scalatest.prop.TableDrivenPropertyChecks import org.scalatest.wordspec.AnyWordSpec @@ -228,3 +230,26 @@ class ProjectResourceIdSpec private lazy val pathGenerator = projectSlugs.map(slug => s"projects/$slug") } + +class GitHttpUrlSpec extends AnyWordSpec with should.Matchers with EitherValues with ScalaCheckPropertyChecks { + + "HttpUrl" should { + + "instantiate for valid absolute git urls" in { + forAll(httpUrls(), projectSlugs) { (httpUrl, projectSlug) => + val url = s"$httpUrl/$projectSlug.git" + GitHttpUrl.from(url).map(_.value) shouldBe Right(url) + } + } + + "fail instantiation for non-absolute urls" in { + + val url = s"${relativePaths().generateOne}/${projectSlugs.generateOne}.git" + + val exception = GitHttpUrl.from(url).left.value + + exception shouldBe an[IllegalArgumentException] + exception.getMessage shouldBe s"$url is not a valid repository http url" + } + } +} diff --git a/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala b/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala index 4012cbf76f..734eaf2eda 100644 --- a/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala +++ b/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala @@ -32,6 +32,11 @@ object TinyTypeDecoders { private type NonBlank = String Refined NonEmpty + implicit def booleanDecoder[TT <: BooleanTinyType](implicit tinyTypeFactory: From[TT]): Decoder[TT] = + decodeBoolean.emap { value => + tinyTypeFactory.from(value).leftMap(_.getMessage) + } + implicit def stringDecoder[TT <: StringTinyType](implicit tinyTypeFactory: From[TT]): Decoder[TT] = decodeString.emap { value => tinyTypeFactory.from(value).leftMap(_.getMessage) From 417269791ea5458ef9f8fecd9f548591b3cac298 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 11 Aug 2023 20:04:42 +0200 Subject: [PATCH 12/49] fix: imports and reformat --- .../projects/details/ProjectJsonEncoderSpec.scala | 2 +- .../main/scala/io/renku/core/client/RenkuCoreClient.scala | 4 +++- .../src/main/scala/io/renku/graph/model/projects.scala | 2 +- .../triplesgenerator/api/TriplesGeneratorClientSpec.scala | 6 +++++- 4 files changed, 10 insertions(+), 4 deletions(-) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala index 35ba64d7c2..47a5da16c5 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/details/ProjectJsonEncoderSpec.scala @@ -38,7 +38,7 @@ import model.Forking.ForksCount import model.Permissions.{AccessLevel, GroupAccessLevel, ProjectAccessLevel} import model.Project.{ImageLinks, StarsCount} import model.Statistics.{CommitsCount, JobArtifactsSize, LsfObjectsSize, RepositorySize, StorageSize} -import model.Urls.{HttpUrl, ReadmeUrl, SshUrl, WebUrl} +import model.Urls.{ReadmeUrl, SshUrl, WebUrl} import model._ import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpec diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index de1e4f7439..33d521d28d 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -31,7 +31,9 @@ import org.typelevel.log4cats.Logger import org.typelevel.ci._ trait RenkuCoreClient[F[_]] { - def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, accessToken: AccessToken): F[Result[ProjectMigrationCheck]] + def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): F[Result[ProjectMigrationCheck]] } object RenkuCoreClient { diff --git a/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala b/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala index 5628c36bda..e38a59916c 100644 --- a/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala +++ b/renku-model-tiny-types/src/main/scala/io/renku/graph/model/projects.scala @@ -209,7 +209,7 @@ object projects { with NonBlank[Keyword] with NonBlankTTJsonLDOps[Keyword] - final class GitHttpUrl private(val value: String) extends AnyVal with StringTinyType + final class GitHttpUrl private (val value: String) extends AnyVal with StringTinyType implicit object GitHttpUrl extends TinyTypeFactory[GitHttpUrl](new GitHttpUrl(_)) with NonBlank[GitHttpUrl] { addConstraint( check = url => diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala index 2491736e2d..78f3f1ed1b 100644 --- a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala @@ -34,7 +34,11 @@ import org.scalatest.matchers.should import org.scalatest.wordspec.AsyncWordSpec import org.typelevel.log4cats.Logger -class TriplesGeneratorClientSpec extends AsyncWordSpec with CustomAsyncIOSpec with should.Matchers with ExternalServiceStubbing { +class TriplesGeneratorClientSpec + extends AsyncWordSpec + with CustomAsyncIOSpec + with should.Matchers + with ExternalServiceStubbing { private implicit val logger: Logger[IO] = TestLogger() private lazy val client = new TriplesGeneratorClientImpl[IO](Uri.unsafeFromString(externalServiceBaseUrl)) From e59f10e2cc4a2fce618b5e269b3830dcd89df86a Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 14 Aug 2023 13:20:54 +0200 Subject: [PATCH 13/49] reformat: LowLevelApis and RenkuCoreClient --- ...VersionClient.scala => LowLevelApis.scala} | 75 +++++---- .../renku/core/client/RenkuCoreClient.scala | 44 ++--- .../renku/core/client/LowLevelApisSpec.scala | 110 +++++++++++++ .../core/client/RenkuCoreClientSpec.scala | 91 +++++++--- .../client/RenkuCoreVersionClientSpec.scala | 155 ------------------ 5 files changed, 232 insertions(+), 243 deletions(-) rename renku-core-client/src/main/scala/io/renku/core/client/{RenkuCoreVersionClient.scala => LowLevelApis.scala} (61%) create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala delete mode 100644 renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala similarity index 61% rename from renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala rename to renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala index 443907fc3c..7aaa2587bc 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreVersionClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala @@ -19,46 +19,65 @@ package io.renku.core.client import cats.effect.Async -import cats.syntax.all._ -import com.typesafe.config.{Config, ConfigFactory} import io.circe.Decoder import io.circe.Decoder.decodeList import io.renku.control.Throttler +import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion -import io.renku.http.client.RestClient +import io.renku.http.client.{AccessToken, RestClient} +import org.http4s.Header import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger +import org.typelevel.ci._ -private trait RenkuCoreVersionClient[F[_]] { - def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] - def getVersions: F[Result[List[SchemaVersion]]] +private trait LowLevelApis[F[_]] { def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] + def getMigrationCheck(coreUri: RenkuCoreUri, + projectGitHttpUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): F[Result[ProjectMigrationCheck]] + def getVersions: F[Result[List[SchemaVersion]]] } -private object RenkuCoreVersionClient { - def apply[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, - config: Config = ConfigFactory.load - ): RenkuCoreVersionClient[F] = - new RenkuCoreVersionClientImpl[F](coreCurrentUri, RenkuCoreUri.ForSchema, config, ClientTools[F]) +private object LowLevelApis { + def apply[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current): LowLevelApis[F] = + new LowLevelApisImpl[F](coreCurrentUri, ClientTools[F]) } -private class RenkuCoreVersionClientImpl[F[_]: Async: Logger](coreUri: RenkuCoreUri.Current, - coreUriForSchemaLoader: RenkuCoreUri.ForSchemaLoader, - config: Config, - clientTools: ClientTools[F] -) extends RestClient[F, Nothing](Throttler.noThrottling) - with RenkuCoreVersionClient[F] +private class LowLevelApisImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, clientTools: ClientTools[F]) + extends RestClient[F, Nothing](Throttler.noThrottling) + with LowLevelApis[F] with Http4sDsl[F] with Http4sClientDsl[F] { import clientTools._ - override def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] = - for { - uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) - apiVersionsRes <- getApiVersion(uriForSchema) - } yield apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) + override def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] = + send(GET(uri.uri / "renku" / "apiversion")) { + case (Ok, _, resp) => + toResult[SchemaApiVersions](resp) + case reqInfo @ (NotFound, _, _) => + toFailure[SchemaApiVersions](s"Api version info for ${uri.uri} does not exist")(reqInfo) + case reqInfo => + toFailure[SchemaApiVersions](s"Finding api version info for ${uri.uri} failed")(reqInfo) + } + + override def getMigrationCheck(coreUri: RenkuCoreUri, + projectGitHttpUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): F[Result[ProjectMigrationCheck]] = { + + val uri = (coreUri.uri / "renku" / "cache.migrations_check") + .withQueryParam("git_url", projectGitHttpUrl.value) + + send(GET(uri).withHeaders(Header.Raw(ci"gitlab-token", accessToken.value))) { + case (Ok, _, resp) => + toResult[ProjectMigrationCheck](resp) + case reqInfo => + toFailure[ProjectMigrationCheck](s"Migration check for $projectGitHttpUrl failed")(reqInfo) + } + } override def getVersions: F[Result[List[SchemaVersion]]] = { val decoder = Decoder.instance[List[SchemaVersion]] { res => @@ -68,19 +87,9 @@ private class RenkuCoreVersionClientImpl[F[_]: Async: Logger](coreUri: RenkuCore res.downField("versions").as(decodeList(singleVersionDecoder)) } - send(GET(coreUri.uri / "renku" / "versions")) { + send(GET(coreCurrentUri.uri / "renku" / "versions")) { case (Ok, _, resp) => toResult[List[SchemaVersion]](resp)(decoder) case reqInfo => toFailure[List[SchemaVersion]](s"Version info cannot be found")(reqInfo) } } - - override def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] = - send(GET(uri.uri / "renku" / "apiversion")) { - case (Ok, _, resp) => - toResult[SchemaApiVersions](resp) - case reqInfo @ (NotFound, _, _) => - toFailure[SchemaApiVersions](s"Api version info for ${uri.uri} does not exist")(reqInfo) - case reqInfo => - toFailure[SchemaApiVersions](s"Finding api version info for ${uri.uri} failed")(reqInfo) - } } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index 33d521d28d..b615597ed5 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -22,52 +22,38 @@ import cats.effect.Async import cats.syntax.all._ import com.typesafe.config.{Config, ConfigFactory} import io.renku.control.Throttler -import io.renku.graph.model.projects -import io.renku.http.client.{AccessToken, RestClient} -import org.http4s.Header +import io.renku.graph.model.versions.SchemaVersion +import io.renku.http.client.RestClient import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger -import org.typelevel.ci._ trait RenkuCoreClient[F[_]] { - def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, - accessToken: AccessToken - ): F[Result[ProjectMigrationCheck]] + def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] } object RenkuCoreClient { def apply[F[_]: Async: Logger](config: Config = ConfigFactory.load): F[RenkuCoreClient[F]] = - for { - coreCurrentUri <- RenkuCoreUri.Current.loadFromConfig[F](config) - versionClient = RenkuCoreVersionClient[F](coreCurrentUri, config) - } yield new RenkuCoreClientImpl[F](coreCurrentUri, versionClient, ClientTools[F]) + RenkuCoreUri.Current.loadFromConfig[F](config).map { coreCurrentUri => + new RenkuCoreClientImpl[F](coreCurrentUri, RenkuCoreUri.ForSchema, LowLevelApis[F](coreCurrentUri), config) + } } private class RenkuCoreClientImpl[F[_]: Async: Logger](currentUri: RenkuCoreUri.Current, - versionClient: RenkuCoreVersionClient[F], - clientTools: ClientTools[F] + coreUriForSchemaLoader: RenkuCoreUri.ForSchemaLoader, + lowLevelApis: LowLevelApis[F], + config: Config ) extends RestClient[F, Nothing](Throttler.noThrottling) with RenkuCoreClient[F] with Http4sDsl[F] with Http4sClientDsl[F] { - import clientTools._ - - println(versionClient) + println(currentUri) - override def getMigrationCheck(projectGitHttpUrl: projects.GitHttpUrl, - accessToken: AccessToken - ): F[Result[ProjectMigrationCheck]] = { - - val uri = (currentUri.uri / "renku" / "cache.migrations_check") - .withQueryParam("git_url", projectGitHttpUrl.value) + override def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] = + for { + uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) + apiVersionsRes <- lowLevelApis.getApiVersion(uriForSchema) + } yield apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) - send(GET(uri).withHeaders(Header.Raw(ci"gitlab-token", accessToken.value))) { - case (Ok, _, resp) => - toResult[ProjectMigrationCheck](resp) - case reqInfo => - toFailure[ProjectMigrationCheck](s"Migration check for $projectGitHttpUrl failed")(reqInfo) - } - } } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala new file mode 100644 index 0000000000..624f8a1ae6 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala @@ -0,0 +1,110 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators._ +import ModelEncoders._ +import cats.effect.IO +import cats.syntax.all._ +import com.github.tomakehurst.wiremock.client.WireMock._ +import io.circe.syntax._ +import io.renku.generators.CommonGraphGenerators.accessTokens +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.GraphModelGenerators.{cliVersions, projectSchemaVersions} +import io.renku.graph.model.RenkuTinyTypeGenerators.projectGitHttpUrls +import io.renku.interpreters.TestLogger +import io.renku.stubbing.ExternalServiceStubbing +import io.renku.testtools.CustomAsyncIOSpec +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.matchers.should +import org.scalatest.wordspec.AsyncWordSpec +import org.scalatest.{EitherValues, OptionValues} +import org.typelevel.log4cats.Logger + +class LowLevelApisSpec + extends AsyncWordSpec + with CustomAsyncIOSpec + with should.Matchers + with OptionValues + with EitherValues + with ExternalServiceStubbing + with AsyncMockFactory { + + "getApiVersion" should { + + "return info about API versions" in { + + val successResult = resultSuccesses(schemaApiVersions).generateOne + + otherWireMockResource.evalMap { server => + server.stubFor { + get(s"/renku/apiversion") + .willReturn(ok(successResult.asJson.spaces2)) + } + + val uriForSchema = RenkuCoreUri.ForSchema(server.baseUri, projectSchemaVersions.generateOne) + + client.getApiVersion(uriForSchema).asserting(_ shouldBe successResult) + }.use_ + } + } + + "getMigrationCheck" should { + + "return info about migration status of the given project on the given Core API" in { + + val accessToken = accessTokens.generateOne + val projectGitHttpUrl = projectGitHttpUrls.generateOne + val migrationCheck = projectMigrationChecks.generateOne + + otherWireMockResource.evalMap { server => + server.stubFor { + get(urlPathEqualTo("/renku/cache.migrations_check")) + .withQueryParam("git_url", equalTo(projectGitHttpUrl.value)) + .withHeader("gitlab-token", equalTo(accessToken.value)) + .willReturn(ok(Result.success(migrationCheck).asJson.spaces2)) + } + + val uriForSchema = RenkuCoreUri.ForSchema(server.baseUri, projectSchemaVersions.generateOne) + + client + .getMigrationCheck(uriForSchema, projectGitHttpUrl, accessToken) + .asserting(_ shouldBe Result.success(migrationCheck)) + }.use_ + } + } + + "getVersions" should { + + "return info about available versions" in { + + val versionTuples = (projectSchemaVersions -> cliVersions).mapN(_ -> _).generateList() + + stubFor { + get(s"/renku/versions") + .willReturn(ok(Result.success(versionTuples).asJson.spaces2)) + } + + client.getVersions.asserting(_ shouldBe Result.success(versionTuples.map(_._1))) + } + } + + private implicit val logger: Logger[IO] = TestLogger() + private lazy val client = new LowLevelApisImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), ClientTools[IO]) +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index 4738e05b1d..eafb63a1af 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -18,14 +18,15 @@ package io.renku.core.client -import Generators.projectMigrationChecks -import ModelEncoders._ +import Generators._ +import cats.MonadThrow import cats.effect.IO -import com.github.tomakehurst.wiremock.client.WireMock._ -import io.circe.syntax._ -import io.renku.generators.CommonGraphGenerators.accessTokens +import cats.syntax.all._ +import com.typesafe.config.Config import io.renku.generators.Generators.Implicits._ -import io.renku.graph.model.GraphModelGenerators.projectGitHttpUrls +import io.renku.generators.Generators.exceptions +import io.renku.graph.model.GraphModelGenerators.projectSchemaVersions +import io.renku.graph.model.versions.SchemaVersion import io.renku.interpreters.TestLogger import io.renku.stubbing.ExternalServiceStubbing import io.renku.testtools.CustomAsyncIOSpec @@ -44,33 +45,71 @@ class RenkuCoreClientSpec with ExternalServiceStubbing with AsyncMockFactory { - "getMigrationCheck" should { + "findCoreUri" should { - "return info about current project schema version" in { + "find the uri of the core for the given schema in the config, " + + "fetch the api version using the uri and " + + "return the CoreUri relevant for the given schema" in { - val accessToken = accessTokens.generateOne - val projectGitHttpUrl = projectGitHttpUrls.generateOne - val migrationCheck = projectMigrationChecks.generateOne + val coreUriForSchema = coreUrisForSchema.generateOne + val apiVersions = schemaApiVersions.generateOne - stubFor { - get(urlPathEqualTo("/renku/cache.migrations_check")) - .withQueryParam("git_url", equalTo(projectGitHttpUrl.value)) - .withHeader("gitlab-token", equalTo(accessToken.value)) - .willReturn(ok(Result.success(migrationCheck).asJson.spaces2)) + givenCoreUriForSchemaInConfig(returning = coreUriForSchema) + givenApiVersionFetching(coreUriForSchema, returning = Result.success(apiVersions)) + + client + .findCoreUri(coreUriForSchema.schemaVersion) + .asserting(_ shouldBe Result.success(RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max))) } - client.getMigrationCheck(projectGitHttpUrl, accessToken).asserting(_ shouldBe Result.success(migrationCheck)) + "fail if finding the uri of the core for the given schema in the config fails " in { + + val exception = exceptions.generateOne + givenCoreUriForSchemaInConfig(failsWith = exception) + + val schemaVersion = projectSchemaVersions.generateOne + + client + .findCoreUri(schemaVersion) + .assertThrowsError[Exception](_ shouldBe exception) + } + + "return a failure if fetching the api version fails" in { + + val coreUriForSchema = coreUrisForSchema.generateOne + val failure = resultDetailedFailures.generateOne + + givenCoreUriForSchemaInConfig(returning = coreUriForSchema) + givenApiVersionFetching(coreUriForSchema, returning = failure) + + client.findCoreUri(coreUriForSchema.schemaVersion).asserting(_ shouldBe failure) } } private implicit val logger: Logger[IO] = TestLogger() - private val coreVersionClient = mock[RenkuCoreVersionClient[IO]] - private lazy val client = - new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), coreVersionClient, ClientTools[IO]) - -// private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = -// (coreUriForSchemaLoader -// .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) -// .expects(returning.schemaVersion, config, *) -// .returning(returning.pure[IO]) + private val coreUriForSchemaLoader = mock[RenkuCoreUri.ForSchemaLoader] + private val lowLevelApis = mock[LowLevelApis[IO]] + private lazy val config = mock[Config] + private lazy val client = new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), + coreUriForSchemaLoader, + lowLevelApis, + config + ) + + private def givenApiVersionFetching(coreUri: RenkuCoreUri.ForSchema, returning: Result[SchemaApiVersions]) = + (lowLevelApis.getApiVersion _) + .expects(coreUri) + .returning(returning.pure[IO]) + + private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = + (coreUriForSchemaLoader + .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) + .expects(returning.schemaVersion, config, *) + .returning(returning.pure[IO]) + + private def givenCoreUriForSchemaInConfig(failsWith: Throwable) = + (coreUriForSchemaLoader + .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) + .expects(*, config, *) + .returning(failsWith.raiseError[IO, Nothing]) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala deleted file mode 100644 index 02e440e5c8..0000000000 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreVersionClientSpec.scala +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Copyright 2023 Swiss Data Science Center (SDSC) - * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and - * Eidgenössische Technische Hochschule Zürich (ETHZ). - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.renku.core.client - -import Generators._ -import ModelEncoders._ -import cats.MonadThrow -import cats.effect.IO -import cats.syntax.all._ -import com.github.tomakehurst.wiremock.client.WireMock._ -import com.typesafe.config.Config -import io.circe.syntax._ -import io.renku.generators.Generators.Implicits._ -import io.renku.generators.Generators.exceptions -import io.renku.graph.model.GraphModelGenerators.{cliVersions, projectSchemaVersions} -import io.renku.graph.model.versions.SchemaVersion -import io.renku.interpreters.TestLogger -import io.renku.stubbing.ExternalServiceStubbing -import io.renku.testtools.CustomAsyncIOSpec -import org.scalamock.scalatest.AsyncMockFactory -import org.scalatest.matchers.should -import org.scalatest.wordspec.AsyncWordSpec -import org.scalatest.{EitherValues, OptionValues} -import org.typelevel.log4cats.Logger - -class RenkuCoreVersionClientSpec - extends AsyncWordSpec - with CustomAsyncIOSpec - with should.Matchers - with OptionValues - with EitherValues - with ExternalServiceStubbing - with AsyncMockFactory { - - "findCoreUri" should { - - "find the uri of the core for the given schema in the config, " + - "fetch the api version using the uri and " + - "return the CoreUri relevant for the given schema" in { - - val schemaVersion = projectSchemaVersions.generateOne - val apiVersions = schemaApiVersions.generateOne - - givenApiVersionFetching(schemaVersion, returning = Result.success(apiVersions)).use { - case (coreUriForSchema, _) => - givenCoreUriForSchemaInConfig(returning = coreUriForSchema) - - client - .findCoreUri(schemaVersion) - .asserting(_ shouldBe Result.success(RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max))) - } - } - - "fail if finding the uri of the core for the given schema in the config fails " in { - - val exception = exceptions.generateOne - givenCoreUriForSchemaInConfig(failsWith = exception) - - val schemaVersion = projectSchemaVersions.generateOne - - client - .findCoreUri(schemaVersion) - .assertThrowsError[Exception](_ shouldBe exception) - } - - "return a failure if fetching the api version fails" in { - - val schemaVersion = projectSchemaVersions.generateOne - val failure = resultDetailedFailures.generateOne - - givenApiVersionFetching(schemaVersion, returning = failure).use { case (coreUriForSchema, _) => - givenCoreUriForSchemaInConfig(returning = coreUriForSchema) - - client.findCoreUri(schemaVersion).asserting(_ shouldBe failure) - } - } - } - - "getVersions" should { - - "return info about available versions" in { - - val versionTuples = (projectSchemaVersions -> cliVersions).mapN(_ -> _).generateList() - - stubFor { - get(s"/renku/versions") - .willReturn(ok(Result.success(versionTuples).asJson.spaces2)) - } - - client.getVersions.asserting(_ shouldBe Result.success(versionTuples.map(_._1))) - } - } - - "getApiVersion" should { - - "return info about API versions" in { - - val successResult = resultSuccesses(schemaApiVersions).generateOne - - givenApiVersionFetching(projectSchemaVersions.generateOne, returning = successResult).map { - case (_, actualApiVersions) => actualApiVersions shouldBe successResult - }.use_ - } - } - - private val coreUriForSchemaLoader = mock[RenkuCoreUri.ForSchemaLoader] - private lazy val config = mock[Config] - private implicit val logger: Logger[IO] = TestLogger() - private lazy val client = - new RenkuCoreVersionClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), - coreUriForSchemaLoader, - config, - ClientTools[IO] - ) - - private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = - (coreUriForSchemaLoader - .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) - .expects(returning.schemaVersion, config, *) - .returning(returning.pure[IO]) - - private def givenCoreUriForSchemaInConfig(failsWith: Throwable) = - (coreUriForSchemaLoader - .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) - .expects(*, config, *) - .returning(failsWith.raiseError[IO, Nothing]) - - private def givenApiVersionFetching(schemaVersion: SchemaVersion, returning: Result[SchemaApiVersions]) = - otherWireMockResource.evalMap { server => - server.stubFor { - get(s"/renku/apiversion") - .willReturn(ok(returning.asJson.spaces2)) - } - - val uriForSchema = RenkuCoreUri.ForSchema(server.baseUri, schemaVersion) - - client.getApiVersion(uriForSchema).map(uriForSchema -> _) - } -} From b1b9a9c6e2b7b5f5566b2d458f01ba43029795a4 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 14 Aug 2023 20:02:20 +0200 Subject: [PATCH 14/49] feat: RenkuCoreClient.findCoreUri(projects.GitHttpUrl) --- .../scala/io/renku/core/client/NestedF.scala | 70 +++++++ .../renku/core/client/RenkuCoreClient.scala | 42 ++++- .../io/renku/core/client/Generators.scala | 3 + .../core/client/RenkuCoreClientSpec.scala | 172 ++++++++++++++++-- 4 files changed, 264 insertions(+), 23 deletions(-) create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala diff --git a/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala b/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala new file mode 100644 index 0000000000..09960f5cff --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala @@ -0,0 +1,70 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.Monad +import cats.data.Nested +import cats.syntax.all._ + +private object NestedF { + def apply[F[_]: Monad]: NestedF[F] = new NestedF[F] +} + +private class NestedF[F[_]: Monad] { + + type NestedF[A] = Nested[F, Result, A] + + implicit lazy val nestedResultMonad: Monad[NestedF] = new Monad[NestedF] { + + override def pure[A](a: A): NestedF[A] = + Nested { + Result.success(a).pure[F] + } + + override def flatMap[A, B](fa: NestedF[A])(f: A => NestedF[B]): NestedF[B] = + Nested { + fa.value >>= { + case Result.Success(a) => f(a).value + case r: Result.Failure => r.pure[F].widen + } + } + + override def map[A, B](fa: NestedF[A])(f: A => B): NestedF[B] = + Nested { + fa.value.map(_.map(f)) + } + + override def tailRecM[A, B](a: A)(f: A => NestedF[Either[A, B]]): NestedF[B] = Nested { + f(a).value >>= { + case Result.Success(Right(vb)) => Result.success(vb).pure[F] + case Result.Success(Left(va)) => tailRecM(va)(f).value + case f: Result.Failure => f.pure[F].widen + } + } + } + + implicit class NestedFOps[A](nestedF: NestedF[A]) { + + def flatMapF[B](f: A => F[Result[B]]): NestedF[B] = + nestedF.flatMap(a => Nested(f(a))) + + def subflatMap[B](f: A => Result[B]): NestedF[B] = + nestedF.flatMap(a => Nested(f(a).pure[F])) + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index b615597ed5..99158b6309 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -18,42 +18,68 @@ package io.renku.core.client +import cats.data.Nested import cats.effect.Async import cats.syntax.all._ import com.typesafe.config.{Config, ConfigFactory} import io.renku.control.Throttler +import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion -import io.renku.http.client.RestClient +import io.renku.http.client.{AccessToken, RestClient} import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger trait RenkuCoreClient[F[_]] { + def findCoreUri(projectUrl: projects.GitHttpUrl, accessToken: AccessToken): F[Result[RenkuCoreUri.Versioned]] def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] } object RenkuCoreClient { def apply[F[_]: Async: Logger](config: Config = ConfigFactory.load): F[RenkuCoreClient[F]] = RenkuCoreUri.Current.loadFromConfig[F](config).map { coreCurrentUri => - new RenkuCoreClientImpl[F](coreCurrentUri, RenkuCoreUri.ForSchema, LowLevelApis[F](coreCurrentUri), config) + new RenkuCoreClientImpl[F](RenkuCoreUri.ForSchema, LowLevelApis[F](coreCurrentUri), config) } } -private class RenkuCoreClientImpl[F[_]: Async: Logger](currentUri: RenkuCoreUri.Current, - coreUriForSchemaLoader: RenkuCoreUri.ForSchemaLoader, - lowLevelApis: LowLevelApis[F], - config: Config +private class RenkuCoreClientImpl[F[_]: Async: Logger](coreUriForSchemaLoader: RenkuCoreUri.ForSchemaLoader, + lowLevelApis: LowLevelApis[F], + config: Config ) extends RestClient[F, Nothing](Throttler.noThrottling) with RenkuCoreClient[F] with Http4sDsl[F] with Http4sClientDsl[F] { - println(currentUri) + private val nestedF = NestedF[F] + import nestedF._ + + override def findCoreUri(projectUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): F[Result[RenkuCoreUri.Versioned]] = + Nested(lowLevelApis.getVersions) + .flatMap(_.findM(migratedAndMatchingSchema(projectUrl, accessToken))) + .flatMapF[RenkuCoreUri.Versioned] { + case Some(sv) => findCoreUri(sv) + case None => Result.failure(show"No API for $projectUrl. Quite likely migration required").pure[F].widen + } + .value + + private def migratedAndMatchingSchema(projectUrl: projects.GitHttpUrl, + accessToken: AccessToken + ): SchemaVersion => Nested[F, Result, Boolean] = + schemaVersion => + Nested { + coreUriForSchemaLoader + .loadFromConfig[F](schemaVersion, config) + .flatMap(lowLevelApis.getMigrationCheck(_, projectUrl, accessToken)) + }.subflatMap { + case ProjectMigrationCheck(`schemaVersion`, MigrationRequired.no) => Result.success(true) + case _ => Result.success(false) + } override def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] = for { uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) apiVersionsRes <- lowLevelApis.getApiVersion(uriForSchema) } yield apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) - } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index f603a28514..5b4ae08649 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -40,6 +40,9 @@ object Generators { implicit lazy val migrationRequiredGen: Gen[MigrationRequired] = Gen.oneOf(MigrationRequired.yes, MigrationRequired.no) + implicit lazy val coreCurrentUris: Gen[RenkuCoreUri.Current] = + httpUrls().map(uri => RenkuCoreUri.Current(Uri.unsafeFromString(uri))) + implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = for { baseUri <- httpUrls() diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index eafb63a1af..a810d21f0e 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -23,12 +23,14 @@ import cats.MonadThrow import cats.effect.IO import cats.syntax.all._ import com.typesafe.config.Config +import io.renku.generators.CommonGraphGenerators.accessTokens import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.exceptions -import io.renku.graph.model.GraphModelGenerators.projectSchemaVersions +import io.renku.graph.model.GraphModelGenerators.{projectGitHttpUrls, projectSchemaVersions} +import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion +import io.renku.http.client.AccessToken import io.renku.interpreters.TestLogger -import io.renku.stubbing.ExternalServiceStubbing import io.renku.testtools.CustomAsyncIOSpec import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.matchers.should @@ -42,24 +44,144 @@ class RenkuCoreClientSpec with should.Matchers with OptionValues with EitherValues - with ExternalServiceStubbing with AsyncMockFactory { - "findCoreUri" should { + "findCoreUri(projects.GitHttpUrl)" should { + + "call the getVersions and " + + "for the first version find the CoreUri in the config and " + + "call the getMigrationCheck on the found uri and " + + "if the schema versions match and migration is not required " + + "return uri from the call to findCoreUri(SchemaVersion)" in { + + val schemaVersion = projectSchemaVersions.generateOne + givenVersionsFinding(returning = Result.success(List(schemaVersion, projectSchemaVersions.generateOne))) + + val coreUriForSchema = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion) + givenCoreUriForSchemaInConfig(coreUriForSchema) + + val projectUrl = projectGitHttpUrls.generateOne + val accessToken = accessTokens.generateOne + val migrationCheck = ProjectMigrationCheck(schemaVersion, MigrationRequired.no) + + givenMigrationCheckFetching(coreUriForSchema, + projectUrl, + accessToken, + returning = Result.success(migrationCheck) + ) + + val coreUriVersioned = givenFindCoreUri(migrationCheck.schemaVersion) + + client + .findCoreUri(projectUrl, accessToken) + .asserting(_ shouldBe Result.success(coreUriVersioned)) + } + + "call the getVersions and " + + "call the getMigrationCheck for each schema and uri " + + "until the schema versions match and migration is not required and " + + "return the uri from the call to findCoreUri(SchemaVersion)" in { + + val schemaVersion1 = projectSchemaVersions.generateOne + val schemaVersion2 = projectSchemaVersions.generateOne + val schemaVersion3 = projectSchemaVersions.generateOne + givenVersionsFinding(returning = Result.success(List(schemaVersion1, schemaVersion2, schemaVersion3))) + + val coreUriForSchema1 = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion1) + givenCoreUriForSchemaInConfig(coreUriForSchema1) + val coreUriForSchema2 = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion2) + givenCoreUriForSchemaInConfig(coreUriForSchema2) + val coreUriForSchema3 = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion3) + givenCoreUriForSchemaInConfig(coreUriForSchema3) + + val projectUrl = projectGitHttpUrls.generateOne + val accessToken = accessTokens.generateOne + + // case when Migration not required by different schema version - a weird case + val migrationCheck1 = ProjectMigrationCheck(projectSchemaVersions.generateOne, MigrationRequired.no) + givenMigrationCheckFetching(coreUriForSchema1, + projectUrl, + accessToken, + returning = Result.success(migrationCheck1) + ) + + // case when Migration required for the schema version + val migrationCheck2 = ProjectMigrationCheck(schemaVersion2, MigrationRequired.yes) + givenMigrationCheckFetching(coreUriForSchema2, + projectUrl, + accessToken, + returning = Result.success(migrationCheck2) + ) + // case when Migration not required for the schema version + val migrationCheck3 = ProjectMigrationCheck(schemaVersion3, MigrationRequired.no) + givenMigrationCheckFetching(coreUriForSchema3, + projectUrl, + accessToken, + returning = Result.success(migrationCheck3) + ) + + val coreUriVersioned = givenFindCoreUri(migrationCheck3.schemaVersion) + + client + .findCoreUri(projectUrl, accessToken) + .asserting(_ shouldBe Result.success(coreUriVersioned)) + } + + "fail when no getMigrationCheck returning matching schemaVersion and no migration requirements" in { + + val schemaVersions = projectSchemaVersions.generateList() + givenVersionsFinding(returning = Result.success(schemaVersions)) + + val projectUrl = projectGitHttpUrls.generateOne + val accessToken = accessTokens.generateOne + + schemaVersions foreach { sv => + val coreUriForSchema = coreUrisForSchema.generateOne.copy(schemaVersion = sv) + givenCoreUriForSchemaInConfig(coreUriForSchema) + + givenMigrationCheckFetching(coreUriForSchema, + projectUrl, + accessToken, + returning = Result.success(projectMigrationChecks.generateOne) + ) + } + + client + .findCoreUri(projectUrl, accessToken) + .asserting(_ shouldBe Result.failure(show"No API for $projectUrl. Quite likely migration required")) + } + + "fail if any of the calls towards the Core API fails" in { + + val schemaVersion = projectSchemaVersions.generateOne + givenVersionsFinding(returning = Result.success(List(schemaVersion))) + + val coreUriForSchema = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion) + givenCoreUriForSchemaInConfig(coreUriForSchema) + + val projectUrl = projectGitHttpUrls.generateOne + val accessToken = accessTokens.generateOne + + val failure = resultDetailedFailures.generateOne + givenMigrationCheckFetching(coreUriForSchema, projectUrl, accessToken, returning = failure) + + client.findCoreUri(projectUrl, accessToken).asserting(_ shouldBe failure) + } + } + + "findCoreUri(SchemaVersion)" should { "find the uri of the core for the given schema in the config, " + "fetch the api version using the uri and " + "return the CoreUri relevant for the given schema" in { - val coreUriForSchema = coreUrisForSchema.generateOne - val apiVersions = schemaApiVersions.generateOne + val schemaVersion = projectSchemaVersions.generateOne - givenCoreUriForSchemaInConfig(returning = coreUriForSchema) - givenApiVersionFetching(coreUriForSchema, returning = Result.success(apiVersions)) + val expectedUriVersioned = givenFindCoreUri(schemaVersion) client - .findCoreUri(coreUriForSchema.schemaVersion) - .asserting(_ shouldBe Result.success(RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max))) + .findCoreUri(schemaVersion) + .asserting(_ shouldBe Result.success(expectedUriVersioned)) } "fail if finding the uri of the core for the given schema in the config fails " in { @@ -90,17 +212,37 @@ class RenkuCoreClientSpec private val coreUriForSchemaLoader = mock[RenkuCoreUri.ForSchemaLoader] private val lowLevelApis = mock[LowLevelApis[IO]] private lazy val config = mock[Config] - private lazy val client = new RenkuCoreClientImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), - coreUriForSchemaLoader, - lowLevelApis, - config - ) + private lazy val client = new RenkuCoreClientImpl[IO](coreUriForSchemaLoader, lowLevelApis, config) + + private def givenFindCoreUri(schemaVersion: SchemaVersion) = { + + val coreUriForSchema = coreUrisForSchema.generateOne.copy(schemaVersion = schemaVersion) + givenCoreUriForSchemaInConfig(returning = coreUriForSchema) + + val apiVersions = schemaApiVersions.generateOne + givenApiVersionFetching(coreUriForSchema, returning = Result.success(apiVersions)) + + RenkuCoreUri.Versioned(coreUriForSchema, apiVersions.max) + } private def givenApiVersionFetching(coreUri: RenkuCoreUri.ForSchema, returning: Result[SchemaApiVersions]) = (lowLevelApis.getApiVersion _) .expects(coreUri) .returning(returning.pure[IO]) + private def givenMigrationCheckFetching(coreUri: RenkuCoreUri, + projectUrl: projects.GitHttpUrl, + accessToken: AccessToken, + returning: Result[ProjectMigrationCheck] + ) = (lowLevelApis.getMigrationCheck _) + .expects(coreUri, projectUrl, accessToken) + .returning(returning.pure[IO]) + + private def givenVersionsFinding(returning: Result[List[SchemaVersion]]) = + (() => lowLevelApis.getVersions) + .expects() + .returning(returning.pure[IO]) + private def givenCoreUriForSchemaInConfig(returning: RenkuCoreUri.ForSchema) = (coreUriForSchemaLoader .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) From 4c5a12204f7ccf5c69c55124ee47d9b6e3671934 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Tue, 15 Aug 2023 20:12:55 +0200 Subject: [PATCH 15/49] feat: LowLevelApis.postProjectUpdate --- .../io/renku/core/client/LowLevelApis.scala | 38 +++++++- .../io/renku/core/client/ProjectUpdates.scala | 45 ++++++++++ .../scala/io/renku/core/client/UserInfo.scala | 23 +++++ .../io/renku/core/client/Generators.scala | 18 +++- .../renku/core/client/LowLevelApisSpec.scala | 33 ++++++- .../client/ProjectMigrationCheckSpec.scala | 2 +- .../core/client/ProjectUpdatesSpec.scala | 90 +++++++++++++++++++ .../core/client/SchemaApiVersionsSpec.scala | 2 +- ...elEncoders.scala => TestModelCodecs.scala} | 2 +- 9 files changed, 242 insertions(+), 11 deletions(-) create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala create mode 100644 renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala create mode 100644 renku-core-client/src/test/scala/io/renku/core/client/ProjectUpdatesSpec.scala rename renku-core-client/src/test/scala/io/renku/core/client/{ModelEncoders.scala => TestModelCodecs.scala} (98%) diff --git a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala index 7aaa2587bc..ff2ef3edc7 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala @@ -19,17 +19,21 @@ package io.renku.core.client import cats.effect.Async -import io.circe.Decoder +import cats.syntax.all._ import io.circe.Decoder.decodeList +import io.circe.DecodingFailure.Reason.CustomReason +import io.circe.syntax._ +import io.circe.{Decoder, DecodingFailure} import io.renku.control.Throttler import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion -import io.renku.http.client.{AccessToken, RestClient} +import io.renku.http.client.{AccessToken, RestClient, UserAccessToken} import org.http4s.Header +import org.http4s.circe._ import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl -import org.typelevel.log4cats.Logger import org.typelevel.ci._ +import org.typelevel.log4cats.Logger private trait LowLevelApis[F[_]] { def getApiVersion(uri: RenkuCoreUri.ForSchema): F[Result[SchemaApiVersions]] @@ -38,6 +42,10 @@ private trait LowLevelApis[F[_]] { accessToken: AccessToken ): F[Result[ProjectMigrationCheck]] def getVersions: F[Result[List[SchemaVersion]]] + def postProjectUpdate(coreUri: RenkuCoreUri.Versioned, + updates: ProjectUpdates, + accessToken: UserAccessToken + ): F[Result[Unit]] } private object LowLevelApis { @@ -89,7 +97,29 @@ private class LowLevelApisImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri send(GET(coreCurrentUri.uri / "renku" / "versions")) { case (Ok, _, resp) => toResult[List[SchemaVersion]](resp)(decoder) - case reqInfo => toFailure[List[SchemaVersion]](s"Version info cannot be found")(reqInfo) + case reqInfo => toFailure[List[SchemaVersion]]("Version info cannot be found")(reqInfo) + } + } + + override def postProjectUpdate(uri: RenkuCoreUri.Versioned, + updates: ProjectUpdates, + accessToken: UserAccessToken + ): F[Result[Unit]] = + send( + request(POST, uri.uri / "renku" / "project.edit", accessToken) + .withEntity(updates.asJson) + .putHeaders(Header.Raw(ci"renku-user-email", updates.userInfo.email.value)) + .putHeaders(Header.Raw(ci"renku-user-fullname", updates.userInfo.name.value)) + ) { + case (Ok, _, resp) => toResult[Unit](resp)(toSuccessfulEdit) + case reqInfo => toFailure[Unit]("Submitting Project Edit payload failed")(reqInfo) } + + private lazy val toSuccessfulEdit = Decoder.instance { cur => + cur + .downField("edited") + .success + .as(().asRight) + .getOrElse(DecodingFailure(CustomReason("Submitting Project Edit payload did not succeed"), cur).asLeft[Unit]) } } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala b/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala new file mode 100644 index 0000000000..b67f6e21cb --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala @@ -0,0 +1,45 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import cats.syntax.all._ +import io.circe.syntax._ +import io.circe.{Encoder, Json} +import io.renku.graph.model.projects + +final case class ProjectUpdates(projectUrl: projects.GitHttpUrl, + userInfo: UserInfo, + maybeDescription: Option[Option[projects.Description]], + maybeKeywords: Option[Set[projects.Keyword]] +) + +object ProjectUpdates { + implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { + case ProjectUpdates(projectUrl, _, maybeDescription, maybeKeywords) => + Json.obj( + List( + ("git_url" -> projectUrl.asJson).some, + ("is_delayed" -> false.asJson).some, + ("migrate_project" -> false.asJson).some, + maybeDescription.map("description" -> _.fold("")(_.value).asJson), + maybeKeywords.map("keywords" -> _.asJson) + ).flatten: _* + ) + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala b/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala new file mode 100644 index 0000000000..748ff1f267 --- /dev/null +++ b/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala @@ -0,0 +1,23 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import io.renku.graph.model.persons + +final case class UserInfo(name: persons.Name, email: persons.Email) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index 5b4ae08649..74af1f91e7 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -21,7 +21,7 @@ package io.renku.core.client import cats.syntax.all._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators._ -import io.renku.graph.model.RenkuTinyTypeGenerators.{cliVersions, projectSchemaVersions} +import io.renku.graph.model.RenkuTinyTypeGenerators.{cliVersions, personEmails, personNames, projectDescriptions, projectGitHttpUrls, projectKeywords, projectSchemaVersions} import org.http4s.Uri import org.scalacheck.Gen @@ -49,15 +49,31 @@ object Generators { schema <- projectSchemaVersions } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(baseUri), schema) + def coreUrisForSchema(baseUri: Uri): Gen[RenkuCoreUri.ForSchema] = + projectSchemaVersions.map(RenkuCoreUri.ForSchema(baseUri, _)) + implicit lazy val coreUrisVersioned: Gen[RenkuCoreUri.Versioned] = for { baseUri <- coreUrisForSchema apiVersion <- apiVersions } yield RenkuCoreUri.Versioned(baseUri, apiVersion) + def coreUrisVersioned(baseUri: Uri): Gen[RenkuCoreUri.Versioned] = + (coreUrisForSchema(baseUri), apiVersions).mapN(RenkuCoreUri.Versioned) + implicit lazy val schemaApiVersions: Gen[SchemaApiVersions] = (apiVersions, apiVersions, cliVersions).mapN(SchemaApiVersions.apply) implicit lazy val projectMigrationChecks: Gen[ProjectMigrationCheck] = (projectSchemaVersions, migrationRequiredGen).mapN(ProjectMigrationCheck.apply) + + implicit lazy val userInfos: Gen[UserInfo] = + (personNames, personEmails).mapN(UserInfo.apply) + + implicit lazy val projectUpdatesGen: Gen[ProjectUpdates] = + (projectGitHttpUrls, + userInfos, + projectDescriptions.toGeneratorOfOptions.toGeneratorOfOptions, + projectKeywords.toGeneratorOfSet().toGeneratorOfOptions + ).mapN(ProjectUpdates.apply) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala index 624f8a1ae6..4841ad43bd 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala @@ -19,12 +19,13 @@ package io.renku.core.client import Generators._ -import ModelEncoders._ +import TestModelCodecs._ import cats.effect.IO import cats.syntax.all._ import com.github.tomakehurst.wiremock.client.WireMock._ +import io.circe.literal._ import io.circe.syntax._ -import io.renku.generators.CommonGraphGenerators.accessTokens +import io.renku.generators.CommonGraphGenerators.userAccessTokens import io.renku.generators.Generators.Implicits._ import io.renku.graph.model.GraphModelGenerators.{cliVersions, projectSchemaVersions} import io.renku.graph.model.RenkuTinyTypeGenerators.projectGitHttpUrls @@ -69,7 +70,7 @@ class LowLevelApisSpec "return info about migration status of the given project on the given Core API" in { - val accessToken = accessTokens.generateOne + val accessToken = userAccessTokens.generateOne val projectGitHttpUrl = projectGitHttpUrls.generateOne val migrationCheck = projectMigrationChecks.generateOne @@ -105,6 +106,32 @@ class LowLevelApisSpec } } + "postProjectUpdate" should { + + "do POST /renku/project.edit with a relevant payload" in { + + val accessToken = userAccessTokens.generateOne + val updates = projectUpdatesGen.generateOne + + otherWireMockResource.evalMap { server => + val versionedUri = coreUrisVersioned(server.baseUri).generateOne + + server.stubFor { + post(s"/${versionedUri.apiVersion}/renku/project.edit") + .withRequestBody(equalToJson(updates.asJson.spaces2)) + .withAccessToken(accessToken.some) + .withHeader("renku-user-email", equalTo(updates.userInfo.email.value)) + .withHeader("renku-user-fullname", equalTo(updates.userInfo.name.value)) + .willReturn(ok(Result.success(json"""{"edited": {}}""").asJson.spaces2)) + } + + client + .postProjectUpdate(versionedUri, updates, accessToken) + .asserting(_ shouldBe Result.success(())) + }.use_ + } + } + private implicit val logger: Logger[IO] = TestLogger() private lazy val client = new LowLevelApisImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), ClientTools[IO]) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala index 5c2536397d..2b15e9a28e 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/ProjectMigrationCheckSpec.scala @@ -19,7 +19,7 @@ package io.renku.core.client import Generators._ -import ModelEncoders.projectMigrationCheckEnc +import TestModelCodecs.projectMigrationCheckEnc import io.circe.syntax._ import io.renku.generators.Generators.Implicits._ import org.scalatest.EitherValues diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ProjectUpdatesSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/ProjectUpdatesSpec.scala new file mode 100644 index 0000000000..ea1b525f86 --- /dev/null +++ b/renku-core-client/src/test/scala/io/renku/core/client/ProjectUpdatesSpec.scala @@ -0,0 +1,90 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.core.client + +import Generators.projectUpdatesGen +import cats.syntax.all._ +import io.circe.literal._ +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.{projectDescriptions, projectKeywords} +import org.scalatest.EitherValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should + +class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with EitherValues { + + it should "encode to JSON - case with no update on description and keywords" in { + + val updates = projectUpdatesGen.generateOne.copy(maybeDescription = None, maybeKeywords = None) + updates.asJson shouldBe json"""{ + "git_url": ${updates.projectUrl}, + "is_delayed": false, + "migrate_project": false + }""" + } + + it should "encode to JSON - case with an update on description to a non-blank value" in { + + val newDesc = projectDescriptions.generateOne + val updates = projectUpdatesGen.generateOne.copy(maybeDescription = newDesc.some.some, maybeKeywords = None) + updates.asJson shouldBe json"""{ + "git_url": ${updates.projectUrl}, + "is_delayed": false, + "migrate_project": false, + "description": $newDesc + }""" + } + + it should "encode to JSON - case with an update on description to remove it" in { + + val updates = projectUpdatesGen.generateOne.copy(maybeDescription = Some(None), maybeKeywords = None) + updates.asJson shouldBe json"""{ + "git_url": ${updates.projectUrl}, + "is_delayed": false, + "migrate_project": false, + "description": "" + }""" + } + + it should "encode to JSON - case with an update on keywords to a non-blank values" in { + + val newKeywords = projectKeywords.generateSet(min = 1) + val updates = projectUpdatesGen.generateOne.copy(maybeDescription = None, maybeKeywords = newKeywords.some) + updates.asJson shouldBe + json"""{ + "git_url": ${updates.projectUrl}, + "is_delayed": false, + "migrate_project": false, + "keywords": ${newKeywords.toList} + }""" + } + + it should "encode to JSON - case with an update on keywords to remove them" in { + + val updates = projectUpdatesGen.generateOne.copy(maybeDescription = None, maybeKeywords = Some(Set.empty)) + updates.asJson shouldBe + json"""{ + "git_url": ${updates.projectUrl}, + "is_delayed": false, + "migrate_project": false, + "keywords": [] + }""" + } +} diff --git a/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala index 8fee3df7c1..2c2398dc07 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/SchemaApiVersionsSpec.scala @@ -19,7 +19,7 @@ package io.renku.core.client import Generators._ -import ModelEncoders.schemaApiVersionsEnc +import TestModelCodecs.schemaApiVersionsEnc import io.circe.syntax._ import io.renku.generators.Generators.Implicits._ import org.scalatest.EitherValues diff --git a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala b/renku-core-client/src/test/scala/io/renku/core/client/TestModelCodecs.scala similarity index 98% rename from renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala rename to renku-core-client/src/test/scala/io/renku/core/client/TestModelCodecs.scala index 36545ddbeb..7b6fe0a29d 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/ModelEncoders.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/TestModelCodecs.scala @@ -23,7 +23,7 @@ import io.circe.literal._ import io.circe.syntax._ import io.renku.graph.model.versions.{CliVersion, SchemaVersion} -private object ModelEncoders { +private object TestModelCodecs { implicit val versionsEnc: Encoder[List[(SchemaVersion, CliVersion)]] = Encoder.instance { versions => val versionJsons = versions.map { case (schema, cli) => From 314e7ee185a5af47b88429038fd58b3f7f808055 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 16 Aug 2023 19:02:15 +0200 Subject: [PATCH 16/49] feat: BranchProtectionCheck --- .../update/BranchProtectionCheckImpl.scala | 59 ++++++ .../update/BranchProtectionCheckSpec.scala | 169 ++++++++++++++++++ 2 files changed, 228 insertions(+) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala new file mode 100644 index 0000000000..6f32006918 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala @@ -0,0 +1,59 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import BranchProtectionCheck._ +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Decoder +import io.renku.graph.model.projects +import io.renku.http.client.{AccessToken, GitLabClient} +import io.renku.http.tinytypes.TinyTypeURIEncoder._ +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe.CirceEntityDecoder._ +import org.http4s.implicits._ +import org.http4s.{Request, Response, Status} + +private trait BranchProtectionCheck[F[_]] { + def canPushToDefaultBranch(slug: projects.Slug, at: AccessToken): F[Boolean] +} + +private object BranchProtectionCheck { + + def apply[F[_]: Async: GitLabClient]: BranchProtectionCheck[F] = new BranchProtectionCheckImpl[F] + + case class BranchInfo(default: Boolean, canPush: Boolean) +} + +private class BranchProtectionCheckImpl[F[_]: Async: GitLabClient] extends BranchProtectionCheck[F] { + + override def canPushToDefaultBranch(slug: projects.Slug, at: AccessToken): F[Boolean] = + GitLabClient[F] + .get(uri"projects" / slug, "project-branches")(mapResponse)(at.some) + .map(_.exists(branch => branch.default && branch.canPush)) + + private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[List[BranchInfo]]] = { + case (Ok, _, resp) => resp.as[List[BranchInfo]] + case (NotFound, _, _) => List.empty[BranchInfo].pure[F] + } + + private implicit lazy val itemDecoder: Decoder[BranchInfo] = + Decoder.forProduct2("default", "can_push")(BranchInfo.apply) +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala new file mode 100644 index 0000000000..d94450e8da --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala @@ -0,0 +1,169 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import BranchProtectionCheck.BranchInfo +import cats.effect.IO +import cats.effect.testing.scalatest.AsyncIOSpec +import cats.syntax.all._ +import eu.timepit.refined.api.Refined +import eu.timepit.refined.auto._ +import eu.timepit.refined.collection.NonEmpty +import io.circe.Encoder +import io.circe.literal._ +import io.circe.syntax._ +import io.renku.generators.CommonGraphGenerators.accessTokens +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.booleans +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.projects +import io.renku.http.client.RestClient.ResponseMappingF +import io.renku.http.client.{AccessToken, GitLabClient} +import io.renku.http.tinytypes.TinyTypeURIEncoder._ +import io.renku.testtools.GitLabClientTools +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe._ +import org.http4s.implicits._ +import org.http4s.{Request, Response, Uri} +import org.scalacheck.Gen +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.EitherValues +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class BranchProtectionCheckSpec + extends AsyncFlatSpec + with AsyncIOSpec + with AsyncMockFactory + with should.Matchers + with EitherValues + with GitLabClientTools[IO] { + + it should "call GL's GET gl/projects/:slug/repository/branches and return true " + + "if there's a default branch with can_push=true" in { + + val slug = projectSlugs.generateOne + val accessToken = accessTokens.generateOne + + givenProjectBranches( + slug, + accessToken, + returning = List(BranchInfo(default = false, canPush = true), + BranchInfo(default = false, canPush = false), + BranchInfo(default = true, canPush = true) + ).pure[IO] + ) + + finder.canPushToDefaultBranch(slug, accessToken).asserting(_ shouldBe true) + } + + it should "call GL's GET gl/projects/:slug/repository/branches and return false " + + "if there's a default branch with can_push=false" in { + + val slug = projectSlugs.generateOne + val accessToken = accessTokens.generateOne + + givenProjectBranches( + slug, + accessToken, + returning = List(BranchInfo(default = false, canPush = true), + BranchInfo(default = false, canPush = false), + BranchInfo(default = true, canPush = false) + ).pure[IO] + ) + + finder.canPushToDefaultBranch(slug, accessToken).asserting(_ shouldBe false) + } + + it should "call GL's GET gl/projects/:slug/repository/branches and return false " + + "if there's no default branch" in { + + val slug = projectSlugs.generateOne + val accessToken = accessTokens.generateOne + + givenProjectBranches( + slug, + accessToken, + returning = + List(BranchInfo(default = false, canPush = true), BranchInfo(default = false, canPush = false)).pure[IO] + ) + + finder.canPushToDefaultBranch(slug, accessToken).asserting(_ shouldBe false) + } + + it should "call GL's GET gl/projects/:slug/repository/branches and return false " + + "if there are no branches found" in { + + val slug = projectSlugs.generateOne + val accessToken = accessTokens.generateOne + + givenProjectBranches( + slug, + accessToken, + returning = List.empty.pure[IO] + ) + + finder.canPushToDefaultBranch(slug, accessToken).asserting(_ shouldBe false) + } + + it should "return a list branches if GL returns 200 with a non empty list" in { + val branches = branchInfos.generateList(min = 1) + mapResponse(Ok, Request[IO](), Response[IO](Ok).withEntity(branches.asJson)) + .asserting(_ shouldBe branches) + } + + it should "return an empty list if GL returns 200 with an empty list of branches" in { + mapResponse(Ok, Request[IO](), Response[IO](Ok).withEntity(List.empty[BranchInfo].asJson)) + .asserting(_ shouldBe Nil) + } + + it should "return an empty list if GL returns 404 NOT_FOUND" in { + mapResponse(NotFound, Request[IO](), Response[IO](NotFound)) + .asserting(_ shouldBe Nil) + } + + private implicit val glClient: GitLabClient[IO] = mock[GitLabClient[IO]] + private lazy val finder = new BranchProtectionCheckImpl[IO] + + private def givenProjectBranches(slug: projects.Slug, accessToken: AccessToken, returning: IO[List[BranchInfo]]) = { + val endpointName: String Refined NonEmpty = "project-branches" + (glClient + .get(_: Uri, _: String Refined NonEmpty)(_: ResponseMappingF[IO, List[BranchInfo]])(_: Option[AccessToken])) + .expects(uri"projects" / slug, endpointName, *, accessToken.some) + .returning(returning) + } + + private lazy val mapResponse: ResponseMappingF[IO, List[BranchInfo]] = + captureMapping(glClient)( + finder + .canPushToDefaultBranch(projectSlugs.generateOne, accessTokens.generateOne) + .unsafeRunSync(), + branchInfos.toGeneratorOfList() + ) + + private implicit lazy val itemEncoder: Encoder[BranchInfo] = Encoder.instance { case BranchInfo(default, canPush) => + json"""{ + "can_push": $canPush, + "default": $default + }""" + } + + private lazy val branchInfos: Gen[BranchInfo] = + (booleans, booleans).mapN(BranchInfo.apply) +} From eb5ae6f3e56dfa9a22076225a08b879679e7d304 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 16 Aug 2023 19:10:26 +0200 Subject: [PATCH 17/49] refactor: TG's Project Update API changed from PUT to PATCH --- .../io/renku/knowledgegraph/MicroserviceRoutes.scala | 4 ++-- .../knowledgegraph/projects/update/Endpoint.scala | 4 ++-- .../renku/knowledgegraph/MicroserviceRoutesSpec.scala | 8 ++++---- .../knowledgegraph/projects/update/EndpointSpec.scala | 10 +++++----- .../triplesgenerator/api/TriplesGeneratorClient.scala | 2 +- .../api/TriplesGeneratorClientSpec.scala | 4 ++-- triples-generator/README.md | 4 ++-- 7 files changed, 18 insertions(+), 18 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala index 86af26a603..eaeffed367 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala @@ -175,13 +175,13 @@ private class MicroserviceRoutes[F[_]: Async]( case authReq @ GET -> "knowledge-graph" /: "projects" /: path as maybeUser => routeToProjectsEndpoints(path, maybeUser.option)(authReq.req) - case authReq @ PUT -> "knowledge-graph" /: "projects" /: path as maybeUser => + case authReq @ PATCH -> "knowledge-graph" /: "projects" /: path as maybeUser => maybeUser.withUserOrNotFound { user => path.segments.toList .map(_.toString) .toProjectSlug .flatTap(authorizeSlug(_, user.some).leftMap(_.toHttpResponse)) - .semiflatMap(`PUT /projects/:slug`(_, authReq.req, user)) + .semiflatMap(`PATCH /projects/:slug`(_, authReq.req, user)) .merge } } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 262f6df077..9b293250c1 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -35,7 +35,7 @@ import org.http4s.{Request, Response} import org.typelevel.log4cats.Logger trait Endpoint[F[_]] { - def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] + def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] } object Endpoint { @@ -48,7 +48,7 @@ private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdat ) extends Http4sDsl[F] with Endpoint[F] { - override def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = + override def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = decodePayload(request) .flatTap(updateGL(slug, authUser)) .flatMap(updateTG(slug)) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala index dcfe60c131..d1c495ce32 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala @@ -47,7 +47,7 @@ import io.renku.interpreters.TestRoutesMetrics import io.renku.knowledgegraph.datasets.details.RequestedDataset import io.renku.testtools.IOSpec import org.http4s.MediaType.application -import org.http4s.Method.{DELETE, GET, PUT} +import org.http4s.Method.{DELETE, GET, PATCH} import org.http4s.Status._ import org.http4s._ import org.http4s.headers.`Content-Type` @@ -563,10 +563,10 @@ class MicroserviceRoutesSpec } } - "PUT /knowledge-graph/projects/:namespace/../:name" should { + "PATCH /knowledge-graph/projects/:namespace/../:name" should { val projectSlug = projectSlugs.generateOne - val request = Request[IO](PUT, Uri.unsafeFromString(s"knowledge-graph/projects/$projectSlug")) + val request = Request[IO](PATCH, Uri.unsafeFromString(s"knowledge-graph/projects/$projectSlug")) s"return $Accepted for valid path parameters, user and payload" in new TestCase { @@ -577,7 +577,7 @@ class MicroserviceRoutesSpec .returning(rightT[IO, EndpointSecurityException](AuthContext(authUser.option, projectSlug, Set(projectSlug)))) (projectUpdateEndpoint - .`PUT /projects/:slug`(_: model.projects.Slug, _: Request[IO], _: AuthUser)) + .`PATCH /projects/:slug`(_: model.projects.Slug, _: Request[IO], _: AuthUser)) .expects(projectSlug, request, authUser.option.get) .returning(Response[IO](Accepted).pure[IO]) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 7b7fd92d7a..768731ffb5 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -55,7 +55,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) givenSyncRepoMetadataSending(slug, newValues, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> response.as[Json].asserting(_ shouldBe Message.Info("Project update accepted").asJson) } @@ -66,7 +66,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(Json.obj()), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(Json.obj()), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.BadRequest) >> response.as[Message].asserting(_ shouldBe Message.Error("Invalid payload")) } @@ -81,7 +81,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val error = jsons.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.left(error.pure[IO])) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.BadRequest) >> response.as[Message].asserting(_ shouldBe Message.Error.fromJsonUnsafe(error)) } @@ -100,7 +100,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc returning = EitherT(exception.raiseError[IO, Either[Json, Unit]]) ) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.InternalServerError) >> response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) } @@ -119,7 +119,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc returning = TriplesGeneratorClient.Result.failure(exception.getMessage).pure[IO] ) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.InternalServerError) >> response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) } diff --git a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala index 9f5fd7ea6f..c8060599ad 100644 --- a/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala +++ b/triples-generator-api/src/main/scala/io/renku/triplesgenerator/api/TriplesGeneratorClient.scala @@ -71,7 +71,7 @@ private class TriplesGeneratorClientImpl[F[_]: Async: Logger](tgUri: Uri) import org.http4s.circe._ override def updateProject(slug: projects.Slug, updates: ProjectUpdates): F[Result[Unit]] = - send(PUT(tgUri / "projects" / slug) withEntity updates.asJson) { + send(PATCH(tgUri / "projects" / slug) withEntity updates.asJson) { case (Ok, _, _) => Result.success(()).pure[F] case (NotFound, _, _) => Result.failure[Unit]("Project for update does not exist").pure[F] case (status, req, _) => diff --git a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala index 78f3f1ed1b..bab6a364f6 100644 --- a/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala +++ b/triples-generator-api/src/test/scala/io/renku/triplesgenerator/api/TriplesGeneratorClientSpec.scala @@ -51,7 +51,7 @@ class TriplesGeneratorClientSpec val updates = projectUpdatesGen.generateOne stubFor { - put(s"/projects/${urlEncode(slug.value)}") + patch(urlEqualTo(s"/projects/${urlEncode(slug.value)}")) .withRequestBody(equalToJson(updates.asJson.spaces2)) .willReturn(ok()) } @@ -65,7 +65,7 @@ class TriplesGeneratorClientSpec val updates = projectUpdatesGen.generateOne stubFor { - put(s"/projects/${urlEncode(slug.value)}") + patch(urlEqualTo(s"/projects/${urlEncode(slug.value)}")) .willReturn(aResponse.withStatus(NotFound.code)) } diff --git a/triples-generator/README.md b/triples-generator/README.md index cf949f1096..a1da32804a 100644 --- a/triples-generator/README.md +++ b/triples-generator/README.md @@ -9,7 +9,7 @@ This microservice deals with all Triples Store administrative and provisioning e | POST | ```/events``` | To send an event for processing | | GET | ```/metrics``` | Serves Prometheus metrics | | GET | ```/ping``` | To check if service is healthy | -| PUT | ```/projects/:slug``` | API to update project data in the TS | +| PATCH | ```/projects/:slug``` | API to update project data in the TS | | GET | ```/version``` | Returns info about service version | #### POST /events @@ -279,7 +279,7 @@ Verifies service health. | OK (200) | If service is healthy | | INTERNAL SERVER ERROR (500)| Otherwise | -#### PUT /knowledge-graph/projects/:slug +#### PATCH /knowledge-graph/projects/:slug API to update project data in the Triples Store. From f4c742d33af02c3ead6ec2700876d842d407b6dc Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 16 Aug 2023 19:23:36 +0200 Subject: [PATCH 18/49] refactor: KG's Project Update API to work for PUT and PATCH --- .../knowledgegraph/MicroserviceRoutes.scala | 2 +- .../MicroserviceRoutesSpec.scala | 70 ++++++++++--------- .../triplesgenerator/MicroserviceRoutes.scala | 10 +-- .../projects/update/Endpoint.scala | 4 +- .../MicroserviceRoutesSpec.scala | 8 +-- .../projects/update/EndpointSpec.scala | 6 +- 6 files changed, 52 insertions(+), 48 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala index eaeffed367..0538fabc04 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala @@ -175,7 +175,7 @@ private class MicroserviceRoutes[F[_]: Async]( case authReq @ GET -> "knowledge-graph" /: "projects" /: path as maybeUser => routeToProjectsEndpoints(path, maybeUser.option)(authReq.req) - case authReq @ PATCH -> "knowledge-graph" /: "projects" /: path as maybeUser => + case authReq @ (PATCH | PUT) -> "knowledge-graph" /: "projects" /: path as maybeUser => maybeUser.withUserOrNotFound { user => path.segments.toList .map(_.toString) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala index d1c495ce32..5716c15f8a 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/MicroserviceRoutesSpec.scala @@ -47,7 +47,7 @@ import io.renku.interpreters.TestRoutesMetrics import io.renku.knowledgegraph.datasets.details.RequestedDataset import io.renku.testtools.IOSpec import org.http4s.MediaType.application -import org.http4s.Method.{DELETE, GET, PATCH} +import org.http4s.Method.{DELETE, GET, PATCH, PUT} import org.http4s.Status._ import org.http4s._ import org.http4s.headers.`Content-Type` @@ -563,53 +563,57 @@ class MicroserviceRoutesSpec } } - "PATCH /knowledge-graph/projects/:namespace/../:name" should { + "PATCH and PUT /knowledge-graph/projects/:namespace/../:name" should { val projectSlug = projectSlugs.generateOne - val request = Request[IO](PATCH, Uri.unsafeFromString(s"knowledge-graph/projects/$projectSlug")) - s"return $Accepted for valid path parameters, user and payload" in new TestCase { + Request[IO](PATCH, Uri.unsafeFromString(s"knowledge-graph/projects/$projectSlug")) :: + Request[IO](PUT, Uri.unsafeFromString(s"knowledge-graph/projects/$projectSlug")) :: Nil foreach { request => + s"return $Accepted for valid path parameters, user and payload - case ${request.method}" in new TestCase { - val authUser = MaybeAuthUser(authUsers.generateOne) + val authUser = MaybeAuthUser(authUsers.generateOne) - (projectSlugAuthorizer.authorize _) - .expects(projectSlug, authUser.option) - .returning(rightT[IO, EndpointSecurityException](AuthContext(authUser.option, projectSlug, Set(projectSlug)))) + (projectSlugAuthorizer.authorize _) + .expects(projectSlug, authUser.option) + .returning( + rightT[IO, EndpointSecurityException](AuthContext(authUser.option, projectSlug, Set(projectSlug))) + ) - (projectUpdateEndpoint - .`PATCH /projects/:slug`(_: model.projects.Slug, _: Request[IO], _: AuthUser)) - .expects(projectSlug, request, authUser.option.get) - .returning(Response[IO](Accepted).pure[IO]) + (projectUpdateEndpoint + .`PATCH /projects/:slug`(_: model.projects.Slug, _: Request[IO], _: AuthUser)) + .expects(projectSlug, request, authUser.option.get) + .returning(Response[IO](Accepted).pure[IO]) - routes(authUser).call(request).status shouldBe Accepted + routes(authUser).call(request).status shouldBe Accepted - routesMetrics.clearRegistry() - } + routesMetrics.clearRegistry() + } - s"return $Unauthorized when authentication fails" in new TestCase { - routes(givenAuthAsUnauthorized) - .call(request) - .status shouldBe Unauthorized - } + s"return $Unauthorized when authentication fails - case ${request.method}" in new TestCase { + routes(givenAuthAsUnauthorized) + .call(request) + .status shouldBe Unauthorized + } - s"return $NotFound when no auth header" in new TestCase { - routes(maybeAuthUser = MaybeAuthUser.noUser).call(request).status shouldBe NotFound - } + s"return $NotFound when no auth header - case ${request.method}" in new TestCase { + routes(maybeAuthUser = MaybeAuthUser.noUser).call(request).status shouldBe NotFound + } - s"return $NotFound when the user has no rights to the project" in new TestCase { + s"return $NotFound when the user has no rights to the project - case ${request.method}" in new TestCase { - val authUser = MaybeAuthUser(authUsers.generateOne) + val authUser = MaybeAuthUser(authUsers.generateOne) - (projectSlugAuthorizer.authorize _) - .expects(projectSlug, authUser.option) - .returning(leftT[IO, AuthContext[model.projects.Slug]](AuthorizationFailure)) + (projectSlugAuthorizer.authorize _) + .expects(projectSlug, authUser.option) + .returning(leftT[IO, AuthContext[model.projects.Slug]](AuthorizationFailure)) - val response = routes(authUser).call(request) + val response = routes(authUser).call(request) - response.status shouldBe NotFound - response.contentType shouldBe Some(`Content-Type`(application.json)) - response.body[Message] shouldBe Message.Error.unsafeApply(AuthorizationFailure.getMessage) - } + response.status shouldBe NotFound + response.contentType shouldBe Some(`Content-Type`(application.json)) + response.body[Message] shouldBe Message.Error.unsafeApply(AuthorizationFailure.getMessage) + } + } } "GET /knowledge-graph/projects/:namespace/../:name/datasets" should { diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala index 2fd8fe3c71..c91d3ed837 100644 --- a/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/MicroserviceRoutes.scala @@ -43,15 +43,15 @@ private class MicroserviceRoutes[F[_]: MonadThrow]( import eventEndpoint._ import org.http4s.HttpRoutes - import projectUpdateEndpoint.`PUT /projects/:slug` + import projectUpdateEndpoint.`PATCH /projects/:slug` import routesMetrics._ // format: off lazy val routes: Resource[F, HttpRoutes[F]] = HttpRoutes.of[F] { - case req @ POST -> Root / "events" => processEvent(req) - case GET -> Root / "ping" => Ok("pong") - case req @ PUT -> Root / "projects" / ProjectSlug(slug) => `PUT /projects/:slug`(slug, req) - case GET -> Root / "config-info" => Ok(configInfo) + case req @ POST -> Root / "events" => processEvent(req) + case GET -> Root / "ping" => Ok("pong") + case req @ PATCH -> Root / "projects" / ProjectSlug(slug) => `PATCH /projects/:slug`(slug, req) + case GET -> Root / "config-info" => Ok(configInfo) }.withMetrics.map(_ <+> versionRoutes()) // format: on diff --git a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala index 28caec1f0d..f6f81c63b2 100644 --- a/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala +++ b/triples-generator/src/main/scala/io/renku/triplesgenerator/projects/update/Endpoint.scala @@ -32,7 +32,7 @@ import org.http4s.{EntityDecoder, Request, Response} import org.typelevel.log4cats.Logger trait Endpoint[F[_]] { - def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] + def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] } object Endpoint { @@ -43,7 +43,7 @@ object Endpoint { private class EndpointImpl[F[_]: Async](projectUpdater: ProjectUpdater[F]) extends Http4sDsl[F] with Endpoint[F] { - override def `PUT /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] = + override def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F]): F[Response[F]] = EitherT(decodePayload(request)) .semiflatMap(projectUpdater.updateProject(slug, _).map(toHttpResult)) .merge diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala index d192c83148..7ec46a1ae4 100644 --- a/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/MicroserviceRoutesSpec.scala @@ -70,12 +70,12 @@ class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory wi } } - "PUT /projects/:slug" should { + "PATCH /projects/:slug" should { "return Ok with message in response" in new TestCase { val slug = projectSlugs.generateOne - val request = Request[IO](Method.PUT, uri"/projects" / slug) + val request = Request[IO](Method.PATCH, uri"/projects" / slug) val responseInfo = Message.Info("Project updated") givenProjectUpdateEndpoint(slug, request, returning = Response[IO](Ok).withEntity(responseInfo)) @@ -88,7 +88,7 @@ class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory wi } "return the default response for illegal slug" in new TestCase { - routes.call(Request[IO](Method.PUT, uri"/projects" / "illegal-slug")).status shouldBe ServiceUnavailable + routes.call(Request[IO](Method.PATCH, uri"/projects" / "illegal-slug")).status shouldBe ServiceUnavailable } } @@ -131,7 +131,7 @@ class MicroserviceRoutesSpec extends AnyWordSpec with IOSpec with MockFactory wi } def givenProjectUpdateEndpoint(slug: model.projects.Slug, request: Request[IO], returning: Response[IO]) = - (projectUpdateEndpoint.`PUT /projects/:slug` _) + (projectUpdateEndpoint.`PATCH /projects/:slug` _) .expects(slug, request) .returning(returning.pure[IO]) } diff --git a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala index e62be5a2ff..c88fac5f1f 100644 --- a/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala +++ b/triples-generator/src/test/scala/io/renku/triplesgenerator/projects/update/EndpointSpec.scala @@ -47,7 +47,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val updates = projectUpdatesGen.generateOne givenProjectUpdating(slug, updates, returning = ProjectUpdater.Result.Updated.pure[IO]) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => response.status.pure[IO].asserting(_ shouldBe Ok) >> response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> response.as[Message].asserting(_ shouldBe Message.Info("Project updated")) @@ -59,7 +59,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val slug = projectSlugs.generateOne val request = Request[IO]().withEntity(Json.obj("visibility" -> Json.obj("newValue" -> "invalid".asJson))) - endpoint.`PUT /projects/:slug`(slug, request) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, request) >>= { response => response.status.pure[IO].asserting(_ shouldBe BadRequest) >> response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> response.as[Message].asserting(_ shouldBe Message.Error("Invalid payload")) @@ -72,7 +72,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val updates = projectUpdatesGen.generateOne givenProjectUpdating(slug, updates, returning = ProjectUpdater.Result.NotExists.pure[IO]) - endpoint.`PUT /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson)) >>= { response => response.status.pure[IO].asserting(_ shouldBe NotFound) >> response.contentType.pure[IO].asserting(_ shouldBe `Content-Type`(application.json).some) >> response.as[Message].asserting(_ shouldBe Message.Info("Project not found")) From 787bbc1d4429d5b7320214b057b6897ca76541ac Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 16 Aug 2023 19:29:57 +0200 Subject: [PATCH 19/49] refactor: KG's NewValues dto renamed to ProjectUpdates --- .../projects/update/Endpoint.scala | 18 ++++++++------- .../projects/update/GLProjectUpdater.scala | 6 ++--- .../projects/update/ProjectUpdater.scala | 23 +++++++++++++++++++ .../{NewValues.scala => ProjectUpdates.scala} | 8 +++---- .../projects/update/EndpointSpec.scala | 21 +++++++++-------- .../update/GLProjectUpdaterSpec.scala | 8 +++---- .../projects/update/Generators.scala | 2 +- 7 files changed, 56 insertions(+), 30 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala rename knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/{NewValues.scala => ProjectUpdates.scala} (82%) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 9b293250c1..fe94326ca7 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -28,7 +28,7 @@ import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry -import io.renku.triplesgenerator.api.{ProjectUpdates, TriplesGeneratorClient} +import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl import org.http4s.{Request, Response} @@ -55,12 +55,12 @@ private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdat .merge .handleErrorWith(serverError(slug)(_)) - private lazy val decodePayload: Request[F] => EitherT[F, Response[F], NewValues] = req => + private lazy val decodePayload: Request[F] => EitherT[F, Response[F], ProjectUpdates] = req => EitherT { req - .as[NewValues] + .as[ProjectUpdates] .map(_.asRight[Response[F]]) - .handleError(badRequest(_).asLeft[NewValues]) + .handleError(badRequest(_).asLeft[ProjectUpdates]) } private def badRequest: Throwable => Response[F] = { _ => @@ -70,13 +70,15 @@ private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdat private def badRequest(message: Json): Response[F] = Response[F](BadRequest).withEntity(Message.Error.fromJsonUnsafe(message)) - private def updateGL(slug: projects.Slug, authUser: AuthUser)(newValues: NewValues): EitherT[F, Response[F], Unit] = - glProjectUpdater.updateProject(slug, newValues, authUser.accessToken).leftMap(badRequest) + private def updateGL(slug: projects.Slug, authUser: AuthUser)( + updates: ProjectUpdates + ): EitherT[F, Response[F], Unit] = + glProjectUpdater.updateProject(slug, updates, authUser.accessToken).leftMap(badRequest) - private def updateTG(slug: projects.Slug)(newValues: NewValues): EitherT[F, Response[F], Response[F]] = + private def updateTG(slug: projects.Slug)(updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = EitherT { tgClient - .updateProject(slug, ProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) + .updateProject(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) .map(_.toEither) }.biSemiflatMap( serverError(slug), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 4bf8218898..f076801bd7 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -33,7 +33,7 @@ import org.http4s.implicits._ import org.http4s.{Request, Response, Status, UrlForm} private trait GLProjectUpdater[F[_]] { - def updateProject(slug: projects.Slug, newValues: NewValues, at: AccessToken): EitherT[F, Json, Unit] + def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] } private object GLProjectUpdater { @@ -42,9 +42,9 @@ private object GLProjectUpdater { private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectUpdater[F] { - override def updateProject(slug: projects.Slug, newValues: NewValues, at: AccessToken): EitherT[F, Json, Unit] = + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] = EitherT { - GitLabClient[F].put(uri"projects" / slug, "edit-project", UrlForm("visibility" -> newValues.visibility.value))( + GitLabClient[F].put(uri"projects" / slug, "edit-project", UrlForm("visibility" -> updates.visibility.value))( mapResponse )(at.some) } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala new file mode 100644 index 0000000000..b6fd1c71cf --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -0,0 +1,23 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +private trait ProjectUpdater[F[_]] { + +} diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/NewValues.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala similarity index 82% rename from knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/NewValues.scala rename to knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index 988d5a06da..8576f6c335 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/NewValues.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -21,9 +21,9 @@ package io.renku.knowledgegraph.projects.update import io.circe.Decoder import io.renku.graph.model.projects -private final case class NewValues(visibility: projects.Visibility) +private final case class ProjectUpdates(visibility: projects.Visibility) -private object NewValues { - implicit val decoder: Decoder[NewValues] = - Decoder.instance(cur => cur.downField("visibility").as[projects.Visibility].map(NewValues(_))) +private object ProjectUpdates { + implicit val decoder: Decoder[ProjectUpdates] = + Decoder.instance(cur => cur.downField("visibility").as[projects.Visibility].map(ProjectUpdates(_))) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 768731ffb5..08b2f47ae4 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -36,7 +36,7 @@ import io.renku.http.client.AccessToken import io.renku.http.server.EndpointTester._ import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec -import io.renku.triplesgenerator.api.{ProjectUpdates, TriplesGeneratorClient} +import io.renku.triplesgenerator.api.{ProjectUpdates => TGProjectUpdates, TriplesGeneratorClient} import org.http4s.{Request, Status} import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -50,7 +50,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) givenSyncRepoMetadataSending(slug, newValues, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) @@ -76,7 +76,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne val error = jsons.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.left(error.pure[IO])) @@ -91,7 +91,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne val exception = exceptions.generateOne givenUpdatingProjectInGL(slug, @@ -110,7 +110,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) val exception = exceptions.generateOne @@ -131,7 +131,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc private lazy val endpoint = new EndpointImpl[IO](glProjectUpdater, tgClient) private def givenUpdatingProjectInGL(slug: projects.Slug, - newValues: NewValues, + newValues: ProjectUpdates, at: AccessToken, returning: EitherT[IO, Json, Unit] ) = (glProjectUpdater.updateProject _) @@ -139,13 +139,14 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc .returning(returning) private def givenSyncRepoMetadataSending(slug: projects.Slug, - newValues: NewValues, + newValues: ProjectUpdates, returning: IO[TriplesGeneratorClient.Result[Unit]] ) = (tgClient.updateProject _) - .expects(slug, ProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) + .expects(slug, TGProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) .returning(returning) - private implicit lazy val payloadEncoder: Encoder[NewValues] = Encoder.instance { case NewValues(visibility) => - json"""{"visibility": $visibility}""" + private implicit lazy val payloadEncoder: Encoder[ProjectUpdates] = Encoder.instance { + case ProjectUpdates(visibility) => + json"""{"visibility": $visibility}""" } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index 76297f4a9a..e1cd7d495d 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -57,7 +57,7 @@ class GLProjectUpdaterSpec it should s"call GL's PUT gl/projects/:slug and return unit on success" in { val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne val accessToken = accessTokens.generateOne givenEditProjectAPICall(slug, newValues, accessToken, returning = ().asRight.pure[IO]) @@ -68,7 +68,7 @@ class GLProjectUpdaterSpec it should s"call GL's PUT gl/projects/:slug and return GL message if returned" in { val slug = projectSlugs.generateOne - val newValues = newValuesGen.generateOne + val newValues = projectUpdatesGen.generateOne val accessToken = accessTokens.generateOne val error = jsons.generateOne @@ -101,7 +101,7 @@ class GLProjectUpdaterSpec private lazy val finder = new GLProjectUpdaterImpl[IO] private def givenEditProjectAPICall(slug: projects.Slug, - newValues: NewValues, + newValues: ProjectUpdates, accessToken: AccessToken, returning: IO[Either[Json, Unit]] ) = { @@ -122,7 +122,7 @@ class GLProjectUpdaterSpec private lazy val mapResponse: ResponseMappingF[IO, Either[Json, Unit]] = captureMapping(glClient)( finder - .updateProject(projectSlugs.generateOne, newValuesGen.generateOne, accessTokens.generateOne) + .updateProject(projectSlugs.generateOne, projectUpdatesGen.generateOne, accessTokens.generateOne) .value .unsafeRunSync(), ().asRight[Json], diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala index 0d560f4218..cd094c0912 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala @@ -22,5 +22,5 @@ import io.renku.graph.model.RenkuTinyTypeGenerators.projectVisibilities import org.scalacheck.Gen private object Generators { - val newValuesGen: Gen[NewValues] = projectVisibilities.map(NewValues(_)) + val projectUpdatesGen: Gen[ProjectUpdates] = projectVisibilities.map(ProjectUpdates(_)) } From 35e4c0d932cb1d5887d20d226d0729e36b5ca34f Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 16 Aug 2023 20:12:32 +0200 Subject: [PATCH 20/49] refactor: ProjectUpdater with business logic extracted from the endpoint --- .../projects/update/Endpoint.scala | 30 +--- .../projects/update/ProjectUpdater.scala | 55 ++++++++ .../projects/update/EndpointSpec.scala | 109 ++++----------- .../projects/update/ProjectUpdaterSpec.scala | 131 ++++++++++++++++++ 4 files changed, 219 insertions(+), 106 deletions(-) create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index fe94326ca7..4853951d90 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -22,13 +22,11 @@ import cats.data.EitherT import cats.effect.Async import cats.syntax.all._ import eu.timepit.refined.auto._ -import io.circe.Json import io.renku.data.Message import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry -import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl import org.http4s.{Request, Response} @@ -40,18 +38,16 @@ trait Endpoint[F[_]] { object Endpoint { def apply[F[_]: Async: Logger: MetricsRegistry: GitLabClient]: F[Endpoint[F]] = - TriplesGeneratorClient[F].map(new EndpointImpl(GLProjectUpdater[F], _)) + ProjectUpdater[F].map(new EndpointImpl(_)) } -private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdater[F], - tgClient: TriplesGeneratorClient[F] -) extends Http4sDsl[F] +private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F]) + extends Http4sDsl[F] with Endpoint[F] { override def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = decodePayload(request) - .flatTap(updateGL(slug, authUser)) - .flatMap(updateTG(slug)) + .semiflatMap(projectUpdater.updateProject(slug, _, authUser)) .merge .handleErrorWith(serverError(slug)(_)) @@ -67,24 +63,6 @@ private class EndpointImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdat Response[F](BadRequest).withEntity(Message.Error("Invalid payload")) } - private def badRequest(message: Json): Response[F] = - Response[F](BadRequest).withEntity(Message.Error.fromJsonUnsafe(message)) - - private def updateGL(slug: projects.Slug, authUser: AuthUser)( - updates: ProjectUpdates - ): EitherT[F, Response[F], Unit] = - glProjectUpdater.updateProject(slug, updates, authUser.accessToken).leftMap(badRequest) - - private def updateTG(slug: projects.Slug)(updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = - EitherT { - tgClient - .updateProject(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) - .map(_.toEither) - }.biSemiflatMap( - serverError(slug), - _ => Response[F](Accepted).withEntity(Message.Info("Project update accepted")).pure[F] - ) - private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = Logger[F] .error(_)(show"Updating project $slug failed") diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index b6fd1c71cf..95c38799c9 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -18,6 +18,61 @@ package io.renku.knowledgegraph.projects.update +import cats.data.EitherT +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Json +import io.renku.data.Message +import io.renku.graph.model.projects +import io.renku.http.client.GitLabClient +import io.renku.http.server.security.model.AuthUser +import io.renku.metrics.MetricsRegistry +import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} +import org.http4s.Response +import org.http4s.Status.{Accepted, BadRequest, InternalServerError} +import org.typelevel.log4cats.Logger + private trait ProjectUpdater[F[_]] { + def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] +} + +private object ProjectUpdater { + def apply[F[_]: Async: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = + TriplesGeneratorClient[F].map(new ProjectUpdaterImpl[F](GLProjectUpdater[F], _)) +} + +private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdater[F], + tgClient: TriplesGeneratorClient[F] +) extends ProjectUpdater[F] { + + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = + updateGL(slug, updates, authUser) + .flatMap(_ => updateTG(slug, updates)) + .merge + + + private def updateGL(slug: projects.Slug, + updates: ProjectUpdates, + authUser: AuthUser + ): EitherT[F, Response[F], Unit] = + glProjectUpdater.updateProject(slug, updates, authUser.accessToken).leftMap(badRequest) + + private def badRequest(message: Json): Response[F] = + Response[F](BadRequest).withEntity(Message.Error.fromJsonUnsafe(message)) + + private def updateTG(slug: projects.Slug, updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = + EitherT { + tgClient + .updateProject(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) + .map(_.toEither) + }.biSemiflatMap( + serverError(slug), + _ => Response[F](Accepted).withEntity(Message.Info("Project update accepted")).pure[F] + ) + private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = + Logger[F] + .error(_)(show"Updating project $slug failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 08b2f47ae4..59268a9765 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -19,7 +19,6 @@ package io.renku.knowledgegraph.projects.update import Generators._ -import cats.data.EitherT import cats.effect.IO import cats.syntax.all._ import eu.timepit.refined.auto._ @@ -27,39 +26,35 @@ import io.circe.literal._ import io.circe.syntax._ import io.circe.{Encoder, Json} import io.renku.data.Message -import io.renku.generators.CommonGraphGenerators.authUsers +import io.renku.generators.CommonGraphGenerators.{authUsers, httpStatuses} import io.renku.generators.Generators.Implicits._ -import io.renku.generators.Generators.{exceptions, jsons} +import io.renku.generators.Generators.{exceptions, sentences} import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs import io.renku.graph.model.projects -import io.renku.http.client.AccessToken import io.renku.http.server.EndpointTester._ +import io.renku.http.server.security.model.AuthUser import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec -import io.renku.triplesgenerator.api.{ProjectUpdates => TGProjectUpdates, TriplesGeneratorClient} -import org.http4s.{Request, Status} +import org.http4s.{Request, Response, Status} import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.matchers.should class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { - it should "call the GL's Edit Project API with the new values extracted from the request, " + - "send project update to TG " + - "and return 202 Accepted" in { + it should "update the project and return the status got from the updater" in { - val authUser = authUsers.generateOne - val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne - givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) - givenSyncRepoMetadataSending(slug, newValues, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + val response = Response[IO](httpStatuses.generateOne).withEntity(Message.Info(sentences().generateOne).asJson) + givenUpdatingProject(slug, updates, authUser, returning = response.pure[IO]) - endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> - response.as[Json].asserting(_ shouldBe Message.Info("Project update accepted").asJson) - } - } + endpoint + .`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) + .asserting(_ shouldBe response) + } it should "return 400 BadRequest if payload is malformed" in { @@ -72,77 +67,31 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc } } - it should "return 400 BadRequest if GL returns 400" in { - - val authUser = authUsers.generateOne - val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne - - val error = jsons.generateOne - givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.left(error.pure[IO])) - - endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe Status.BadRequest) >> - response.as[Message].asserting(_ shouldBe Message.Error.fromJsonUnsafe(error)) - } - } - - it should "return 500 InternalServerError if updating GL failed" in { - - val authUser = authUsers.generateOne - val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne - - val exception = exceptions.generateOne - givenUpdatingProjectInGL(slug, - newValues, - authUser.accessToken, - returning = EitherT(exception.raiseError[IO, Either[Json, Unit]]) - ) - - endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe Status.InternalServerError) >> - response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) - } - } - - it should "return 500 InternalServerError if updating project in TG failed" in { + it should "return 500 InternalServerError if updating project fails" in { - val authUser = authUsers.generateOne - val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne - givenUpdatingProjectInGL(slug, newValues, authUser.accessToken, returning = EitherT.pure[IO, Json](())) val exception = exceptions.generateOne - givenSyncRepoMetadataSending(slug, - newValues, - returning = TriplesGeneratorClient.Result.failure(exception.getMessage).pure[IO] - ) + givenUpdatingProject(slug, updates, authUser, returning = exception.raiseError[IO, Nothing]) - endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(newValues.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.InternalServerError) >> response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) } } private implicit val logger: TestLogger[IO] = TestLogger[IO]() - private val glProjectUpdater = mock[GLProjectUpdater[IO]] - private val tgClient = mock[TriplesGeneratorClient[IO]] - private lazy val endpoint = new EndpointImpl[IO](glProjectUpdater, tgClient) - - private def givenUpdatingProjectInGL(slug: projects.Slug, - newValues: ProjectUpdates, - at: AccessToken, - returning: EitherT[IO, Json, Unit] - ) = (glProjectUpdater.updateProject _) - .expects(slug, newValues, at) - .returning(returning) - - private def givenSyncRepoMetadataSending(slug: projects.Slug, - newValues: ProjectUpdates, - returning: IO[TriplesGeneratorClient.Result[Unit]] - ) = (tgClient.updateProject _) - .expects(slug, TGProjectUpdates.empty.copy(newVisibility = newValues.visibility.some)) + private val projectUpdater = mock[ProjectUpdater[IO]] + private lazy val endpoint = new EndpointImpl[IO](projectUpdater) + + private def givenUpdatingProject(slug: projects.Slug, + udpates: ProjectUpdates, + authUser: AuthUser, + returning: IO[Response[IO]] + ) = (projectUpdater.updateProject _) + .expects(slug, udpates, authUser) .returning(returning) private implicit lazy val payloadEncoder: Encoder[ProjectUpdates] = Encoder.instance { diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala new file mode 100644 index 0000000000..8e08d1fc1c --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -0,0 +1,131 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import Generators._ +import cats.data.EitherT +import cats.effect.IO +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Json +import io.circe.syntax._ +import io.renku.data.Message +import io.renku.generators.CommonGraphGenerators.authUsers +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.{exceptions, jsons} +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.projects +import io.renku.http.client.AccessToken +import io.renku.interpreters.TestLogger +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} +import org.http4s.Status.{Accepted, BadRequest, InternalServerError} +import org.http4s.circe._ +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { + + it should "send project update to GL and TG " + + "and return 202 Accepted when no failures" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) + givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe Accepted) >> + response.as[Json].asserting(_ shouldBe Message.Info("Project update accepted").asJson) + } + } + + it should "return 400 BadRequest if GL returns 400" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + val error = jsons.generateOne + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.left(error.pure[IO])) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe BadRequest) >> + response.as[Message].asserting(_ shouldBe Message.Error.fromJsonUnsafe(error)) + } + } + + it should "fail if updating GL fails" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + val exception = exceptions.generateOne + givenUpdatingProjectInGL(slug, + updates, + authUser.accessToken, + returning = EitherT(exception.raiseError[IO, Either[Json, Unit]]) + ) + + updater.updateProject(slug, updates, authUser).assertThrowsError[Exception](_ shouldBe exception) + } + + it should "return 500 InternalServerError if updating project in TG failed" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) + val exception = exceptions.generateOne + givenSendingUpdateToTG(slug, + updates, + returning = TriplesGeneratorClient.Result.failure(exception.getMessage).pure[IO] + ) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) + } + } + + private implicit val logger: TestLogger[IO] = TestLogger[IO]() + private val glProjectUpdater = mock[GLProjectUpdater[IO]] + private val tgClient = mock[TriplesGeneratorClient[IO]] + private lazy val updater = new ProjectUpdaterImpl[IO](glProjectUpdater, tgClient) + + private def givenUpdatingProjectInGL(slug: projects.Slug, + updates: ProjectUpdates, + at: AccessToken, + returning: EitherT[IO, Json, Unit] + ) = (glProjectUpdater.updateProject _) + .expects(slug, updates, at) + .returning(returning) + + private def givenSendingUpdateToTG(slug: projects.Slug, + updates: ProjectUpdates, + returning: IO[TriplesGeneratorClient.Result[Unit]] + ) = (tgClient.updateProject _) + .expects(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) + .returning(returning) +} From 24de95c3f100b233055f17ca4c0417f178ec3a5a Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 12:43:58 +0200 Subject: [PATCH 21/49] feat: KG's Project Update API to update project image --- knowledge-graph/README.md | 16 ++++- .../renku/knowledgegraph/docs/Encoders.scala | 1 + .../io/renku/knowledgegraph/docs/model.scala | 24 +++++++ .../projects/update/EndpointDocs.scala | 22 +++++-- .../projects/update/GLProjectUpdater.scala | 10 ++- .../projects/update/ProjectUpdater.scala | 6 +- .../projects/update/ProjectUpdates.scala | 32 +++++++++- .../projects/update/EndpointSpec.scala | 11 ++-- .../update/GLProjectUpdaterSpec.scala | 16 +++-- .../projects/update/Generators.scala | 10 ++- .../projects/update/ProjectUpdaterSpec.scala | 6 +- .../projects/update/ProjectUpdatesSpec.scala | 63 +++++++++++++++++++ .../tinytypes/json/TinyTypeDecoders.scala | 4 +- 13 files changed, 189 insertions(+), 32 deletions(-) create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index f716e730d2..1592daf361 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -15,7 +15,7 @@ The following routes may be slightly different when accessed via the main Renku | GET | ```/knowledge-graph/ontology``` | Returns ontology used in the Knowledge Graph | | DELETE | ```/knowledge-graph/projects/:namespace/:name``` | Deletes the project with the given `namespace/name` from knowledge-graph and GitLab | | GET | ```/knowledge-graph/projects/:namespace/:name``` | Returns details of the project with the given `namespace/name` | -| PUT | ```/knowledge-graph/projects/:namespace/:name``` | Updates selected properties of the project with the given `namespace/name` | +| PATCH | ```/knowledge-graph/projects/:namespace/:name``` | Updates selected properties of the project with the given `namespace/name` | | GET | ```/knowledge-graph/projects/:namespace/:name/datasets``` | Returns datasets of the project with the given `slug` | | GET | ```/knowledge-graph/projects/:namespace/:name/datasets/:dsName/tags``` | Returns tags of the dataset with the given `dsName` on project with the given `slug` | | GET | ```/knowledge-graph/projects/:namespace/:name/files/:location/lineage``` | Returns the lineage for a the path (location) of a file on a project | @@ -811,10 +811,19 @@ Response body example for `Accept: application/ld+json`: } ``` -#### PUT /knowledge-graph/projects/:namespace/:name +#### PATCH /knowledge-graph/projects/:namespace/:name -API to update selected properies of the project with the given `namespace/name` in both the Triples Store and GitLab +API to update project data. +Each of the properties can be either set to a new value or omitted in case there's no new value. + +The properties that can be updated are: +* image - possible values are: + * `null` for removing the current image + * any relative or absolute link to the image +* visibility - possible values are: `public`, `internal`, `private` + +In case no properties are set, no data will be changed. The endpoint requires an authorization token to be passed. Supported headers are: - `Authorization: Bearer ` with OAuth Token obtained from GitLab @@ -824,6 +833,7 @@ The endpoint requires an authorization token to be passed. Supported headers are ```json { + "image": "image.png", "visibility": "public|internal|private" } ``` diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/Encoders.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/Encoders.scala index f4fef6a15a..8c47f7039f 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/Encoders.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/Encoders.scala @@ -56,6 +56,7 @@ private object Encoders { operation match { case _: Operation.Delete => json"""{"delete": $operation}""" case _: Operation.Get => json"""{"get": $operation}""" + case _: Operation.Patch => json"""{"patch": $operation}""" case _: Operation.Put => json"""{"put": $operation}""" } } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala index 75c899c231..383d4876c1 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala @@ -193,6 +193,22 @@ object model { ) } + def PATCH(summary: String, + description: String, + uri: Uri, + requestBody: RequestBody, + statusAndResponse: (Status, Response)* + ): OpMapping = { + val parameters = uri.parts.flatMap { + case ParameterPart(parameter) => Some(parameter) + case _ => None + } + + OpMapping(Uri.getTemplate(uri.parts), + Patch(summary.some, description.some, parameters, requestBody.some, statusAndResponse.toMap, Nil) + ) + } + def PUT(summary: String, description: String, uri: Uri, @@ -225,6 +241,14 @@ object model { security: List[SecurityRequirement] ) extends Operation + case class Patch(summary: Option[String], + description: Option[String], + parameters: List[Parameter], + requestBody: Option[RequestBody], + responses: Map[Status, Response], + security: List[SecurityRequirement] + ) extends Operation + case class Put(summary: Option[String], description: Option[String], parameters: List[Parameter], diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index ddf6d4bacd..a6424d0561 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -24,15 +24,26 @@ import eu.timepit.refined.auto._ import io.circe.literal._ import io.renku.data.Message import io.renku.graph.model.projects -import io.renku.knowledgegraph.docs.model.Operation.PUT +import io.renku.knowledgegraph.docs.model.Operation.PATCH import io.renku.knowledgegraph.docs.model._ object EndpointDocs extends docs.EndpointDocs { override lazy val path: Path = Path( - PUT( + PATCH( "Project Update", - "Update selected properties of the Project with the given slug", + """|API to update project data. + | + |Each of the properties can be either set to a new value or omitted in case there's no new value. + | + |The properties that can be updated are: + |* image - possible values are: + | * `null` for removing the current image + | * any relative or absolute link to the image + |* visibility - possible values are: `public`, `internal`, `private` + | + |In case no properties are set, no data will be changed. + |""".stripMargin, Uri / "projects" / namespace / projectName, RequestBody( "Properties with new values", @@ -40,7 +51,10 @@ object EndpointDocs extends docs.EndpointDocs { Contents( MediaType.`application/json`( Schema.`Object`(properties = Map("visibility" -> Schema.EnumString(projects.Visibility.all.map(_.value)))), - json"""{"visibility": "public|internal|private"}""" + json"""{ + "image": "image.png", + "visibility": "public|internal|private" + }""" ) ) ), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index f076801bd7..05fa31622c 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -44,11 +44,15 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] = EitherT { - GitLabClient[F].put(uri"projects" / slug, "edit-project", UrlForm("visibility" -> updates.visibility.value))( - mapResponse - )(at.some) + GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) } + private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(newImage, newVisibility) => + UrlForm.empty + .updateFormField("avatar", newImage.map(_.fold[String](ifEmpty = null)(_.value))) + .updateFormField("visibility", newVisibility.map(_.value)) + } + private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Either[Json, Unit]]] = { case (Ok, _, _) => ().asRight[Json].pure[F] case (BadRequest, _, response) => response.as[Json](MonadThrow[F], jsonOf(Async[F], errorDecoder)).map(_.asLeft) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 95c38799c9..04904e3c5c 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -51,7 +51,6 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjec .flatMap(_ => updateTG(slug, updates)) .merge - private def updateGL(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser @@ -64,7 +63,10 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjec private def updateTG(slug: projects.Slug, updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = EitherT { tgClient - .updateProject(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) + .updateProject( + slug, + TGProjectUpdates.empty.copy(newImages = updates.newImage.map(_.toList), newVisibility = updates.newVisibility) + ) .map(_.toEither) }.biSemiflatMap( serverError(slug), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index 8576f6c335..6d8ee87255 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -18,12 +18,38 @@ package io.renku.knowledgegraph.projects.update -import io.circe.Decoder +import cats.syntax.all._ +import io.circe.syntax._ +import io.circe.{Decoder, DecodingFailure, Encoder, Json} +import io.renku.graph.model.images.ImageUri import io.renku.graph.model.projects +import io.renku.tinytypes.json.TinyTypeDecoders._ -private final case class ProjectUpdates(visibility: projects.Visibility) +private final case class ProjectUpdates(newImage: Option[Option[ImageUri]], newVisibility: Option[projects.Visibility]) private object ProjectUpdates { + + lazy val empty: ProjectUpdates = ProjectUpdates(None, None) + + implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { case ProjectUpdates(newImage, newVisibility) => + Json.obj( + List( + newImage.map(v => "image" -> v.fold(Json.Null)(_.asJson)), + newVisibility.map(v => "visibility" -> v.asJson) + ).flatten: _* + ) + } + implicit val decoder: Decoder[ProjectUpdates] = - Decoder.instance(cur => cur.downField("visibility").as[projects.Visibility].map(ProjectUpdates(_))) + Decoder.instance { cur => + for { + newImage <- cur + .downField("image") + .success + .fold(Option.empty[Option[ImageUri]].asRight[DecodingFailure]) { + _.as[Option[ImageUri]](blankStringToNoneDecoder(ImageUri)).map(_.some) + } + newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] + } yield ProjectUpdates(newImage, newVisibility) + } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 59268a9765..e390183ff2 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -24,7 +24,6 @@ import cats.syntax.all._ import eu.timepit.refined.auto._ import io.circe.literal._ import io.circe.syntax._ -import io.circe.{Encoder, Json} import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.{authUsers, httpStatuses} import io.renku.generators.Generators.Implicits._ @@ -61,7 +60,10 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(Json.obj()), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, + Request[IO]().withEntity(json"""{"visibility": "unknown"}"""), + authUser + ) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.BadRequest) >> response.as[Message].asserting(_ shouldBe Message.Error("Invalid payload")) } @@ -93,9 +95,4 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc ) = (projectUpdater.updateProject _) .expects(slug, udpates, authUser) .returning(returning) - - private implicit lazy val payloadEncoder: Encoder[ProjectUpdates] = Encoder.instance { - case ProjectUpdates(visibility) => - json"""{"visibility": $visibility}""" - } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index e1cd7d495d..8d94eb5459 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -110,15 +110,19 @@ class GLProjectUpdaterSpec .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Json, Unit]])( _: Option[AccessToken] )) - .expects(uri"projects" / slug, - endpointName, - UrlForm("visibility" -> newValues.visibility.value), - *, - accessToken.some - ) + .expects(uri"projects" / slug, endpointName, toUrlForm(newValues), *, accessToken.some) .returning(returning) } + private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(newImage, newVisibility) => + UrlForm( + List( + newImage.map("avatar" -> _.fold[String](null)(_.value)), + newVisibility.map("visibility" -> _.value) + ).flatten: _* + ) + } + private lazy val mapResponse: ResponseMappingF[IO, Either[Json, Unit]] = captureMapping(glClient)( finder diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala index cd094c0912..faf2c6e849 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala @@ -18,9 +18,15 @@ package io.renku.knowledgegraph.projects.update -import io.renku.graph.model.RenkuTinyTypeGenerators.projectVisibilities +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.{imageUris, projectVisibilities} import org.scalacheck.Gen private object Generators { - val projectUpdatesGen: Gen[ProjectUpdates] = projectVisibilities.map(ProjectUpdates(_)) + + val projectUpdatesGen: Gen[ProjectUpdates] = + for { + maybeNewImages <- imageUris.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewVisibility <- projectVisibilities.toGeneratorOfOptions + } yield ProjectUpdates(maybeNewImages, maybeNewVisibility) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 8e08d1fc1c..182f4a1ee9 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -126,6 +126,10 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul updates: ProjectUpdates, returning: IO[TriplesGeneratorClient.Result[Unit]] ) = (tgClient.updateProject _) - .expects(slug, TGProjectUpdates.empty.copy(newVisibility = updates.visibility.some)) + .expects(slug, + TGProjectUpdates.empty.copy(newImages = updates.newImage.map(_.toList), + newVisibility = updates.newVisibility + ) + ) .returning(returning) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala new file mode 100644 index 0000000000..696eee8267 --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala @@ -0,0 +1,63 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import Generators._ +import io.circe.literal._ +import io.circe.syntax._ +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.blankStrings +import org.scalatest.EitherValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should +import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks + +class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaCheckPropertyChecks with EitherValues { + + it should "encode/decode " in { + forAll(projectUpdatesGen) { updates => + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + } + + it should "lack of the image property to be considered as no-op for the property" in { + + val updates = ProjectUpdates.empty.copy(newImage = None) + + updates.asJson shouldBe json"""{}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + it should "image = null to be considered as descriptions removals" in { + + val updates = ProjectUpdates.empty.copy(newImage = Some(None)) + + updates.asJson shouldBe json"""{"image": null}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + it should "image with a blank value to be considered as description removals" in { + + val json = json"""{"image": ${blankStrings().generateOne}}""" + + json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newImage = Some(None)) + } +} diff --git a/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala b/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala index 734eaf2eda..ca2b592f09 100644 --- a/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala +++ b/tiny-types/src/main/scala/io/renku/tinytypes/json/TinyTypeDecoders.scala @@ -93,7 +93,9 @@ object TinyTypeDecoders { .leftMap(_.getMessage) } - implicit def blankStringToNoneDecoder[TT <: StringTinyType](implicit tinyTypeFactory: From[TT]): Decoder[Option[TT]] = + implicit def blankStringToNoneDecoder[TT <: TinyType { type V = String }](implicit + tinyTypeFactory: From[TT] + ): Decoder[Option[TT]] = decodeOption(decodeString).emap((blankToNone andThen toOption[TT])(_).leftMap(_.getMessage())) private lazy val blankToNone: Option[String] => Option[NonBlank] = _.map(_.trim) >>= { From 0590f48aa1e038a08a537310afa9f54770b6e48b Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 13:55:29 +0200 Subject: [PATCH 22/49] feat: KG's Project Update API to update project desc and keywords --- knowledge-graph/README.md | 10 ++++- .../projects/update/EndpointDocs.scala | 10 ++++- .../projects/update/GLProjectUpdater.scala | 2 +- .../projects/update/ProjectUpdater.scala | 6 ++- .../projects/update/ProjectUpdates.scala | 45 ++++++++++++------- .../update/GLProjectUpdaterSpec.scala | 2 +- .../projects/update/Generators.scala | 8 ++-- .../projects/update/ProjectUpdaterSpec.scala | 11 +++-- .../projects/update/ProjectUpdatesSpec.scala | 29 +++++++++++- 9 files changed, 91 insertions(+), 32 deletions(-) diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index 1592daf361..5e17de05bc 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -818,9 +818,13 @@ API to update project data. Each of the properties can be either set to a new value or omitted in case there's no new value. The properties that can be updated are: +* description - possible values are: + * `null` for removing the current description + * any non-blank String value * image - possible values are: * `null` for removing the current image * any relative or absolute link to the image +* keywords - an array of String values; an empty array removes all the keywords * visibility - possible values are: `public`, `internal`, `private` In case no properties are set, no data will be changed. @@ -833,8 +837,10 @@ The endpoint requires an authorization token to be passed. Supported headers are ```json { - "image": "image.png", - "visibility": "public|internal|private" + "description": "a new project description", + "image": "image.png", + "keywords": ["keyword1", "keyword2"], + "visibility": "public|internal|private" } ``` diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index a6424d0561..25cb8ed489 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -37,9 +37,13 @@ object EndpointDocs extends docs.EndpointDocs { |Each of the properties can be either set to a new value or omitted in case there's no new value. | |The properties that can be updated are: + |* description - possible values are: + | * `null` for removing the current description + | * any non-blank String value |* image - possible values are: | * `null` for removing the current image | * any relative or absolute link to the image + |* keywords - an array of String values; an empty array removes all the keywords |* visibility - possible values are: `public`, `internal`, `private` | |In case no properties are set, no data will be changed. @@ -52,8 +56,10 @@ object EndpointDocs extends docs.EndpointDocs { MediaType.`application/json`( Schema.`Object`(properties = Map("visibility" -> Schema.EnumString(projects.Visibility.all.map(_.value)))), json"""{ - "image": "image.png", - "visibility": "public|internal|private" + "description": "a new project description", + "image": "image.png", + "keywords": ["keyword1", "keyword2"], + "visibility": "public|internal|private" }""" ) ) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 05fa31622c..527b213aee 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -47,7 +47,7 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) } - private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(newImage, newVisibility) => + private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => UrlForm.empty .updateFormField("avatar", newImage.map(_.fold[String](ifEmpty = null)(_.value))) .updateFormField("visibility", newVisibility.map(_.value)) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 04904e3c5c..2e12a92363 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -65,7 +65,11 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjec tgClient .updateProject( slug, - TGProjectUpdates.empty.copy(newImages = updates.newImage.map(_.toList), newVisibility = updates.newVisibility) + TGProjectUpdates(newDescription = updates.newDescription, + newImages = updates.newImage.map(_.toList), + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility + ) ) .map(_.toEither) }.biSemiflatMap( diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index 6d8ee87255..c058967f20 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -24,32 +24,45 @@ import io.circe.{Decoder, DecodingFailure, Encoder, Json} import io.renku.graph.model.images.ImageUri import io.renku.graph.model.projects import io.renku.tinytypes.json.TinyTypeDecoders._ +import io.renku.tinytypes.{From, TinyType} -private final case class ProjectUpdates(newImage: Option[Option[ImageUri]], newVisibility: Option[projects.Visibility]) +private final case class ProjectUpdates(newDescription: Option[Option[projects.Description]], + newImage: Option[Option[ImageUri]], + newKeywords: Option[Set[projects.Keyword]], + newVisibility: Option[projects.Visibility] +) private object ProjectUpdates { - lazy val empty: ProjectUpdates = ProjectUpdates(None, None) + lazy val empty: ProjectUpdates = ProjectUpdates(None, None, None, None) - implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { case ProjectUpdates(newImage, newVisibility) => - Json.obj( - List( - newImage.map(v => "image" -> v.fold(Json.Null)(_.asJson)), - newVisibility.map(v => "visibility" -> v.asJson) - ).flatten: _* - ) + implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { + case ProjectUpdates(newDescription, newImage, newKeywords, newVisibility) => + Json.obj( + List( + newDescription.map(v => "description" -> v.fold(Json.Null)(_.asJson)), + newImage.map(v => "image" -> v.fold(Json.Null)(_.asJson)), + newKeywords.map(v => "keywords" -> v.asJson), + newVisibility.map(v => "visibility" -> v.asJson) + ).flatten: _* + ) } implicit val decoder: Decoder[ProjectUpdates] = Decoder.instance { cur => + def toOptionOfOption[T <: TinyType { type V = String }](prop: String, ttFactory: From[T]) = + cur + .downField(prop) + .success + .fold(Option.empty[Option[T]].asRight[DecodingFailure]) { + _.as[Option[T]](blankStringToNoneDecoder(ttFactory)).map(_.some) + } + for { - newImage <- cur - .downField("image") - .success - .fold(Option.empty[Option[ImageUri]].asRight[DecodingFailure]) { - _.as[Option[ImageUri]](blankStringToNoneDecoder(ImageUri)).map(_.some) - } + newDesc <- toOptionOfOption("description", projects.Description) + newImage <- toOptionOfOption("image", ImageUri) + newKeywords <- cur.downField("keywords").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] - } yield ProjectUpdates(newImage, newVisibility) + } yield ProjectUpdates(newDesc, newImage, newKeywords, newVisibility) } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index 8d94eb5459..b174405929 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -114,7 +114,7 @@ class GLProjectUpdaterSpec .returning(returning) } - private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(newImage, newVisibility) => + private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => UrlForm( List( newImage.map("avatar" -> _.fold[String](null)(_.value)), diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala index faf2c6e849..e512018539 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala @@ -19,14 +19,16 @@ package io.renku.knowledgegraph.projects.update import io.renku.generators.Generators.Implicits._ -import io.renku.graph.model.RenkuTinyTypeGenerators.{imageUris, projectVisibilities} +import io.renku.graph.model.RenkuTinyTypeGenerators.{imageUris, projectDescriptions, projectKeywords, projectVisibilities} import org.scalacheck.Gen private object Generators { val projectUpdatesGen: Gen[ProjectUpdates] = for { - maybeNewImages <- imageUris.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewDesc <- projectDescriptions.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewImage <- imageUris.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewKeywords <- projectKeywords.toGeneratorOfSet().toGeneratorOfOptions maybeNewVisibility <- projectVisibilities.toGeneratorOfOptions - } yield ProjectUpdates(maybeNewImages, maybeNewVisibility) + } yield ProjectUpdates(maybeNewDesc, maybeNewImage, maybeNewKeywords, maybeNewVisibility) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 182f4a1ee9..033725788f 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -126,10 +126,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul updates: ProjectUpdates, returning: IO[TriplesGeneratorClient.Result[Unit]] ) = (tgClient.updateProject _) - .expects(slug, - TGProjectUpdates.empty.copy(newImages = updates.newImage.map(_.toList), - newVisibility = updates.newVisibility - ) + .expects( + slug, + TGProjectUpdates(newDescription = updates.newDescription, + newImages = updates.newImage.map(_.toList), + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility + ) ) .returning(returning) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala index 696eee8267..eb3bf0d9cf 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala @@ -36,6 +36,31 @@ class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaChec } } + it should "lack of the description property to be considered as no-op for the property" in { + + val updates = ProjectUpdates.empty.copy(newDescription = None) + + updates.asJson shouldBe json"""{}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + it should "description = null to be considered as description removal" in { + + val updates = ProjectUpdates.empty.copy(newDescription = Some(None)) + + updates.asJson shouldBe json"""{"description": null}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + it should "description with a blank value to be considered as description removal" in { + + val json = json"""{"description": ${blankStrings().generateOne}}""" + + json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newDescription = Some(None)) + } + it should "lack of the image property to be considered as no-op for the property" in { val updates = ProjectUpdates.empty.copy(newImage = None) @@ -45,7 +70,7 @@ class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaChec updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } - it should "image = null to be considered as descriptions removals" in { + it should "image = null to be considered as image removal" in { val updates = ProjectUpdates.empty.copy(newImage = Some(None)) @@ -54,7 +79,7 @@ class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaChec updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } - it should "image with a blank value to be considered as description removals" in { + it should "image with a blank value to be considered as image removal" in { val json = json"""{"image": ${blankStrings().generateOne}}""" From 70b643d453d78f96a7f9fe8241ac128ebc8ff149 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 16:33:15 +0200 Subject: [PATCH 23/49] feat: KG's Project Update API to return 409 if user cannot push --- knowledge-graph/README.md | 1 + .../io/renku/knowledgegraph/docs/model.scala | 1 + .../projects/update/EndpointDocs.scala | 9 ++ .../projects/update/GLProjectUpdater.scala | 5 +- .../projects/update/ProjectUpdater.scala | 32 ++++-- .../projects/update/ProjectUpdates.scala | 10 +- .../update/GLProjectUpdaterSpec.scala | 18 +++- .../projects/update/ProjectUpdaterSpec.scala | 59 ++++++++-- .../projects/update/ProjectUpdatesSpec.scala | 101 ++++++++++++------ 9 files changed, 182 insertions(+), 54 deletions(-) diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index 5e17de05bc..5a653b2fdf 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -852,6 +852,7 @@ The endpoint requires an authorization token to be passed. Supported headers are | BAD_REQUEST (400) | If the given payload is empty or malformed | | UNAUTHORIZED (401) | If given auth header cannot be authenticated | | NOT_FOUND (404) | If there is no project with the given `namespace/name` or the user is not authorised to access the project | +| CONFLICT (409) | If updating the data is not possible, e.g. the user cannot push to the default branch | | INTERNAL SERVER ERROR (500) | Otherwise | #### GET /knowledge-graph/projects/:namespace/:name/datasets diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala index 383d4876c1..33d685fa23 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala @@ -366,6 +366,7 @@ object model { case object BadRequest extends Status(400, "Bad Request") case object Unauthorized extends Status(401, "Unauthorized") case object NotFound extends Status(404, "Not Found") + case object Conflict extends Status(409, "Conflict") case object InternalServerError extends Status(500, "Internal Server Error") } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index 25cb8ed489..bb9a8218e6 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -80,6 +80,15 @@ object EndpointDocs extends docs.EndpointDocs { "Project not found", Contents(MediaType.`application/json`("Reason", Message.Info("Project does not exist"))) ), + Status.Conflict -> Response( + "When the update is not possible due to current project configuration", + Contents( + MediaType.`application/json`( + "Reason", + Message.Info("Updating project not possible; quite likely the user cannot push to the default branch") + ) + ) + ), Status.InternalServerError -> Response("Error", Contents(MediaType.`application/json`("Reason", Message.Info("Message"))) ) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 527b213aee..5b0d33ea76 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -44,7 +44,10 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] = EitherT { - GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) + if ((updates.newImage orElse updates.newVisibility).isDefined) + GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) + else + ().asRight[Json].pure[F] } private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 2e12a92363..64ece17552 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -30,7 +30,7 @@ import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} import org.http4s.Response -import org.http4s.Status.{Accepted, BadRequest, InternalServerError} +import org.http4s.Status.{Accepted, BadRequest, Conflict, InternalServerError} import org.typelevel.log4cats.Logger private trait ProjectUpdater[F[_]] { @@ -39,17 +39,27 @@ private trait ProjectUpdater[F[_]] { private object ProjectUpdater { def apply[F[_]: Async: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = - TriplesGeneratorClient[F].map(new ProjectUpdaterImpl[F](GLProjectUpdater[F], _)) + TriplesGeneratorClient[F].map(new ProjectUpdaterImpl[F](BranchProtectionCheck[F], GLProjectUpdater[F], _)) } -private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjectUpdater[F], - tgClient: TriplesGeneratorClient[F] +private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: BranchProtectionCheck[F], + glProjectUpdater: GLProjectUpdater[F], + tgClient: TriplesGeneratorClient[F] ) extends ProjectUpdater[F] { + import branchProtectionCheck.canPushToDefaultBranch + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = - updateGL(slug, updates, authUser) - .flatMap(_ => updateTG(slug, updates)) - .merge + if ((updates.newDescription orElse updates.newKeywords).isEmpty) + updateGL(slug, updates, authUser) + .flatMap(_ => updateTG(slug, updates)) + .merge + else + canPushToDefaultBranch(slug, authUser.accessToken) + .flatMap { + case false => conflictResponse.pure[F] + case true => acceptedResponse.pure[F] + } private def updateGL(slug: projects.Slug, updates: ProjectUpdates, @@ -74,11 +84,17 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](glProjectUpdater: GLProjec .map(_.toEither) }.biSemiflatMap( serverError(slug), - _ => Response[F](Accepted).withEntity(Message.Info("Project update accepted")).pure[F] + _ => acceptedResponse.pure[F] ) private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = Logger[F] .error(_)(show"Updating project $slug failed") .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) + + private lazy val acceptedResponse = Response[F](Accepted).withEntity(Message.Info("Project update accepted")) + private lazy val conflictResponse = Response[F](Conflict) + .withEntity( + Message.Error("Updating project not possible; quite likely the user cannot push to the default branch") + ) } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index c058967f20..2578ce2aec 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -30,7 +30,15 @@ private final case class ProjectUpdates(newDescription: Option[Option[projects.D newImage: Option[Option[ImageUri]], newKeywords: Option[Set[projects.Keyword]], newVisibility: Option[projects.Visibility] -) +) { + + lazy val onlyGLUpdateNeeded: Boolean = + (newImage orElse newVisibility).isDefined && + (newDescription orElse newKeywords).isEmpty + + lazy val coreUpdateNeeded: Boolean = + (newDescription orElse newKeywords).isDefined +} private object ProjectUpdates { diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index b174405929..4eb06e516b 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -57,7 +57,7 @@ class GLProjectUpdaterSpec it should s"call GL's PUT gl/projects/:slug and return unit on success" in { val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne + val newValues = projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne val accessToken = accessTokens.generateOne givenEditProjectAPICall(slug, newValues, accessToken, returning = ().asRight.pure[IO]) @@ -65,10 +65,19 @@ class GLProjectUpdaterSpec finder.updateProject(slug, newValues, accessToken).value.asserting(_.value shouldBe ()) } + it should s"do nothing if neither new image nor visibility is set in the update" in { + + val slug = projectSlugs.generateOne + val newValues = projectUpdatesGen.generateOne.copy(newImage = None, newVisibility = None) + val accessToken = accessTokens.generateOne + + finder.updateProject(slug, newValues, accessToken).value.asserting(_.value shouldBe ()) + } + it should s"call GL's PUT gl/projects/:slug and return GL message if returned" in { val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.generateOne + val newValues = projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne val accessToken = accessTokens.generateOne val error = jsons.generateOne @@ -126,7 +135,10 @@ class GLProjectUpdaterSpec private lazy val mapResponse: ResponseMappingF[IO, Either[Json, Unit]] = captureMapping(glClient)( finder - .updateProject(projectSlugs.generateOne, projectUpdatesGen.generateOne, accessTokens.generateOne) + .updateProject(projectSlugs.generateOne, + projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne, + accessTokens.generateOne + ) .value .unsafeRunSync(), ().asRight[Json], diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 033725788f..0a1d292de5 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -35,7 +35,7 @@ import io.renku.http.client.AccessToken import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} -import org.http4s.Status.{Accepted, BadRequest, InternalServerError} +import org.http4s.Status.{Accepted, BadRequest, InternalServerError, Conflict} import org.http4s.circe._ import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -43,12 +43,16 @@ import org.scalatest.matchers.should class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { - it should "send project update to GL and TG " + + it should "if only GL update needed " + + "send update only to GL and TG " + "and return 202 Accepted when no failures" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen + .map(_.copy(newDescription = None, newKeywords = None)) + .suchThat(_.onlyGLUpdateNeeded) + .generateOne givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) @@ -59,11 +63,36 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } } + it should "if core update needed " + + "check if pushing to the default branch is allowed " + + "and return 409 Conflict if it's not" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = false.pure[IO]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe Conflict) >> + response + .as[Json] + .asserting( + _ shouldBe Message + .Error("Updating project not possible; quite likely the user cannot push to the default branch") + .asJson + ) + } + } + it should "return 400 BadRequest if GL returns 400" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen + .map(_.copy(newDescription = None, newKeywords = None)) + .suchThat(_.onlyGLUpdateNeeded) + .generateOne val error = jsons.generateOne givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.left(error.pure[IO])) @@ -78,7 +107,10 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen + .map(_.copy(newDescription = None, newKeywords = None)) + .suchThat(_.onlyGLUpdateNeeded) + .generateOne val exception = exceptions.generateOne givenUpdatingProjectInGL(slug, @@ -94,7 +126,10 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen + .map(_.copy(newDescription = None, newKeywords = None)) + .suchThat(_.onlyGLUpdateNeeded) + .generateOne givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) val exception = exceptions.generateOne @@ -110,9 +145,15 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } private implicit val logger: TestLogger[IO] = TestLogger[IO]() - private val glProjectUpdater = mock[GLProjectUpdater[IO]] - private val tgClient = mock[TriplesGeneratorClient[IO]] - private lazy val updater = new ProjectUpdaterImpl[IO](glProjectUpdater, tgClient) + private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] + private val glProjectUpdater = mock[GLProjectUpdater[IO]] + private val tgClient = mock[TriplesGeneratorClient[IO]] + private lazy val updater = new ProjectUpdaterImpl[IO](branchProtectionCheck, glProjectUpdater, tgClient) + + private def givenBranchProtectionChecking(slug: projects.Slug, at: AccessToken, returning: IO[Boolean]) = + (branchProtectionCheck.canPushToDefaultBranch _) + .expects(slug, at) + .returning(returning) private def givenUpdatingProjectInGL(slug: projects.Slug, updates: ProjectUpdates, diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala index eb3bf0d9cf..76bde0657e 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala @@ -24,65 +24,102 @@ import io.circe.syntax._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.blankStrings import org.scalatest.EitherValues -import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should +import org.scalatest.wordspec.AnyWordSpec import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks -class ProjectUpdatesSpec extends AnyFlatSpec with should.Matchers with ScalaCheckPropertyChecks with EitherValues { +class ProjectUpdatesSpec extends AnyWordSpec with should.Matchers with ScalaCheckPropertyChecks with EitherValues { - it should "encode/decode " in { - forAll(projectUpdatesGen) { updates => - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + "onlyGLUpdateNeeded" should { + + "return true if at least image and/or visibility is updated but not desc and keywords" in { + forAll( + projectUpdatesGen + .suchThat(u => (u.newImage orElse u.newVisibility).isDefined) + .map(_.copy(newDescription = None, newKeywords = None)) + )(_.onlyGLUpdateNeeded shouldBe true) } - } - it should "lack of the description property to be considered as no-op for the property" in { + "return false otherwise" in { + forAll( + projectUpdatesGen + .suchThat(u => (u.newDescription orElse u.newKeywords).isDefined) + )(_.onlyGLUpdateNeeded shouldBe false) + } + } - val updates = ProjectUpdates.empty.copy(newDescription = None) + "coreUpdateNeeded" should { - updates.asJson shouldBe json"""{}""" + "return true if at least description and/or keywords is updated" in { + forAll( + projectUpdatesGen + .suchThat(u => (u.newDescription orElse u.newKeywords).isDefined) + )(_.coreUpdateNeeded shouldBe true) + } - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + "return false otherwise" in { + forAll(projectUpdatesGen.map(_.copy(newDescription = None, newKeywords = None)))( + _.coreUpdateNeeded shouldBe false + ) + } } - it should "description = null to be considered as description removal" in { + "encode/decode" should { - val updates = ProjectUpdates.empty.copy(newDescription = Some(None)) + "encode/decode all standard cases" in { + forAll(projectUpdatesGen) { updates => + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + } - updates.asJson shouldBe json"""{"description": null}""" + "lack of the description property to be considered as no-op for the property" in { - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates - } + val updates = ProjectUpdates.empty.copy(newDescription = None) - it should "description with a blank value to be considered as description removal" in { + updates.asJson shouldBe json"""{}""" - val json = json"""{"description": ${blankStrings().generateOne}}""" + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } - json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newDescription = Some(None)) - } + "description = null to be considered as description removal" in { + + val updates = ProjectUpdates.empty.copy(newDescription = Some(None)) + + updates.asJson shouldBe json"""{"description": null}""" - it should "lack of the image property to be considered as no-op for the property" in { + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } - val updates = ProjectUpdates.empty.copy(newImage = None) + "description with a blank value to be considered as description removal" in { - updates.asJson shouldBe json"""{}""" + val json = json"""{"description": ${blankStrings().generateOne}}""" - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates - } + json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newDescription = Some(None)) + } - it should "image = null to be considered as image removal" in { + "lack of the image property to be considered as no-op for the property" in { - val updates = ProjectUpdates.empty.copy(newImage = Some(None)) + val updates = ProjectUpdates.empty.copy(newImage = None) - updates.asJson shouldBe json"""{"image": null}""" + updates.asJson shouldBe json"""{}""" - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates - } + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } - it should "image with a blank value to be considered as image removal" in { + "image = null to be considered as image removal" in { - val json = json"""{"image": ${blankStrings().generateOne}}""" + val updates = ProjectUpdates.empty.copy(newImage = Some(None)) - json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newImage = Some(None)) + updates.asJson shouldBe json"""{"image": null}""" + + updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates + } + + "image with a blank value to be considered as image removal" in { + + val json = json"""{"image": ${blankStrings().generateOne}}""" + + json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newImage = Some(None)) + } } } From 52bb2872d36543b1271be9d80cd5779c66d6f63f Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 16:45:27 +0200 Subject: [PATCH 24/49] feat: RenkuCoreClient to expose updateProject API --- .../renku/core/client/RenkuCoreClient.scala | 11 +++++++- .../io/renku/core/client/Generators.scala | 3 +++ .../core/client/RenkuCoreClientSpec.scala | 27 +++++++++++++++++-- 3 files changed, 38 insertions(+), 3 deletions(-) diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index 99158b6309..49f6ecde4d 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -25,7 +25,7 @@ import com.typesafe.config.{Config, ConfigFactory} import io.renku.control.Throttler import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion -import io.renku.http.client.{AccessToken, RestClient} +import io.renku.http.client.{AccessToken, RestClient, UserAccessToken} import org.http4s.client.dsl.Http4sClientDsl import org.http4s.dsl.Http4sDsl import org.typelevel.log4cats.Logger @@ -33,6 +33,10 @@ import org.typelevel.log4cats.Logger trait RenkuCoreClient[F[_]] { def findCoreUri(projectUrl: projects.GitHttpUrl, accessToken: AccessToken): F[Result[RenkuCoreUri.Versioned]] def findCoreUri(schemaVersion: SchemaVersion): F[Result[RenkuCoreUri.Versioned]] + def updateProject(coreUri: RenkuCoreUri.Versioned, + updates: ProjectUpdates, + accessToken: UserAccessToken + ): F[Result[Unit]] } object RenkuCoreClient { @@ -82,4 +86,9 @@ private class RenkuCoreClientImpl[F[_]: Async: Logger](coreUriForSchemaLoader: R uriForSchema <- coreUriForSchemaLoader.loadFromConfig[F](schemaVersion, config) apiVersionsRes <- lowLevelApis.getApiVersion(uriForSchema) } yield apiVersionsRes.map(_.max).map(RenkuCoreUri.Versioned(uriForSchema, _)) + + override def updateProject(coreUri: RenkuCoreUri.Versioned, + updates: ProjectUpdates, + accessToken: UserAccessToken + ): F[Result[Unit]] = lowLevelApis.postProjectUpdate(coreUri, updates, accessToken) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index 74af1f91e7..5dc9566fad 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -27,6 +27,9 @@ import org.scalacheck.Gen object Generators { + def resultsGen[T](payloadGen: Gen[T]): Gen[Result[T]] = + Gen.oneOf(resultSuccesses(payloadGen), resultDetailedFailures) + def resultSuccesses[T](payloadGen: Gen[T]): Gen[Result[T]] = payloadGen.map(Result.success) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index a810d21f0e..fa8342b8c6 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -23,13 +23,13 @@ import cats.MonadThrow import cats.effect.IO import cats.syntax.all._ import com.typesafe.config.Config -import io.renku.generators.CommonGraphGenerators.accessTokens +import io.renku.generators.CommonGraphGenerators.{accessTokens, userAccessTokens} import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.exceptions import io.renku.graph.model.GraphModelGenerators.{projectGitHttpUrls, projectSchemaVersions} import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion -import io.renku.http.client.AccessToken +import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import org.scalamock.scalatest.AsyncMockFactory @@ -208,6 +208,21 @@ class RenkuCoreClientSpec } } + "updateProject" should { + + "call the Core's postProjectUpdate API" in { + + val coreUri = coreUrisVersioned.generateOne + val updates = projectUpdatesGen.generateOne + val accessToken = userAccessTokens.generateOne + + val result = resultsGen(()).generateOne + givenPostingProjectUpdate(coreUri, updates, accessToken, returning = result) + + client.updateProject(coreUri, updates, accessToken).asserting(_ shouldBe result) + } + } + private implicit val logger: Logger[IO] = TestLogger() private val coreUriForSchemaLoader = mock[RenkuCoreUri.ForSchemaLoader] private val lowLevelApis = mock[LowLevelApis[IO]] @@ -254,4 +269,12 @@ class RenkuCoreClientSpec .loadFromConfig[IO](_: SchemaVersion, _: Config)(_: MonadThrow[IO])) .expects(*, config, *) .returning(failsWith.raiseError[IO, Nothing]) + + private def givenPostingProjectUpdate(coreUri: RenkuCoreUri.Versioned, + updates: ProjectUpdates, + accessToken: UserAccessToken, + returning: Result[Unit] + ) = (lowLevelApis.postProjectUpdate _) + .expects(coreUri, updates, accessToken) + .returning(returning.pure[IO]) } From 69423055d506f14b653d85f790fcf313e58ee5eb Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 17:10:12 +0200 Subject: [PATCH 25/49] feat: ProjectGitUrlFinder --- .../update/BranchProtectionCheckImpl.scala | 2 +- .../projects/update/ProjectGitUrlFinder.scala | 56 +++++++++ .../update/BranchProtectionCheckSpec.scala | 2 +- .../update/ProjectGitUrlFinderSpec.scala | 108 ++++++++++++++++++ 4 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinder.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala index 6f32006918..a88cf00385 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckImpl.scala @@ -46,7 +46,7 @@ private class BranchProtectionCheckImpl[F[_]: Async: GitLabClient] extends Branc override def canPushToDefaultBranch(slug: projects.Slug, at: AccessToken): F[Boolean] = GitLabClient[F] - .get(uri"projects" / slug, "project-branches")(mapResponse)(at.some) + .get(uri"projects" / slug / "repository" / "branches", "project-branches")(mapResponse)(at.some) .map(_.exists(branch => branch.default && branch.canPush)) private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[List[BranchInfo]]] = { diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinder.scala new file mode 100644 index 0000000000..13842386be --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinder.scala @@ -0,0 +1,56 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.Async +import cats.effect.kernel.Concurrent +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Decoder +import io.renku.graph.model.projects +import io.renku.http.client.{GitLabClient, UserAccessToken} +import io.renku.http.tinytypes.TinyTypeURIEncoder._ +import io.renku.tinytypes.json.TinyTypeDecoders._ +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe.jsonOf +import org.http4s.implicits._ +import org.http4s.{EntityDecoder, Request, Response, Status} + +private trait ProjectGitUrlFinder[F[_]] { + def findGitUrl(slug: projects.Slug, accessToken: UserAccessToken): F[Option[projects.GitHttpUrl]] +} + +private object ProjectGitUrlFinder { + def apply[F[_]: Async: GitLabClient]: ProjectGitUrlFinder[F] = new ProjectGitUrlFinderImpl[F] +} + +private class ProjectGitUrlFinderImpl[F[_]: Async: GitLabClient] extends ProjectGitUrlFinder[F] { + + override def findGitUrl(slug: projects.Slug, at: UserAccessToken): F[Option[projects.GitHttpUrl]] = + GitLabClient[F] + .get(uri"projects" / slug, "single-project")(mapResponse)(at.some) + + private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Option[projects.GitHttpUrl]]] = { + case (Ok, _, resp) => resp.as[Option[projects.GitHttpUrl]] + case (NotFound, _, _) => Option.empty[projects.GitHttpUrl].pure[F] + } + + private implicit lazy val entityDecoder: EntityDecoder[F, Option[projects.GitHttpUrl]] = + jsonOf(implicitly[Concurrent[F]], Decoder.instance(_.downField("http_url_to_repo").as[Option[projects.GitHttpUrl]])) +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala index d94450e8da..8e71a1fedf 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/BranchProtectionCheckSpec.scala @@ -145,7 +145,7 @@ class BranchProtectionCheckSpec val endpointName: String Refined NonEmpty = "project-branches" (glClient .get(_: Uri, _: String Refined NonEmpty)(_: ResponseMappingF[IO, List[BranchInfo]])(_: Option[AccessToken])) - .expects(uri"projects" / slug, endpointName, *, accessToken.some) + .expects(uri"projects" / slug / "repository" / "branches", endpointName, *, accessToken.some) .returning(returning) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala new file mode 100644 index 0000000000..490deb1418 --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala @@ -0,0 +1,108 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.IO +import cats.effect.testing.scalatest.AsyncIOSpec +import cats.syntax.all._ +import eu.timepit.refined.api.Refined +import eu.timepit.refined.auto._ +import eu.timepit.refined.collection.NonEmpty +import io.circe.literal._ +import io.circe.syntax._ +import io.circe.{Encoder, Json} +import io.renku.generators.CommonGraphGenerators.userAccessTokens +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.{projectGitHttpUrls, projectSlugs} +import io.renku.graph.model.projects +import io.renku.http.client.RestClient.ResponseMappingF +import io.renku.http.client.{AccessToken, GitLabClient} +import io.renku.http.tinytypes.TinyTypeURIEncoder._ +import io.renku.testtools.GitLabClientTools +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe._ +import org.http4s.implicits._ +import org.http4s.{Request, Response, Uri} +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.EitherValues +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class ProjectGitUrlFinderSpec + extends AsyncFlatSpec + with AsyncIOSpec + with AsyncMockFactory + with should.Matchers + with EitherValues + with GitLabClientTools[IO] { + + it should "call GL's GET gl/projects/:slug and return the http git url" in { + + val slug = projectSlugs.generateOne + val accessToken = userAccessTokens.generateOne + val maybeGitUrl = projectGitHttpUrls.generateOption + + givenProjectGitUrl( + slug, + accessToken, + returning = maybeGitUrl.pure[IO] + ) + + finder.findGitUrl(slug, accessToken).asserting(_ shouldBe maybeGitUrl) + } + + it should "return some gitUrl if GL returns 200 with payload containing the 'http_url_to_repo'" in { + val gitUrl = projectGitHttpUrls.generateOne + mapResponse(Ok, Request[IO](), Response[IO](Ok).withEntity(gitUrl.asJson(payloadEncoder))) + .asserting(_ shouldBe Some(gitUrl)) + } + + it should "return no gitUrl if GL returns 404 NOT_FOUND" in { + mapResponse(NotFound, Request[IO](), Response[IO](NotFound)) + .asserting(_ shouldBe None) + } + + private implicit val glClient: GitLabClient[IO] = mock[GitLabClient[IO]] + private lazy val finder = new ProjectGitUrlFinderImpl[IO] + + private def givenProjectGitUrl(slug: projects.Slug, + accessToken: AccessToken, + returning: IO[Option[projects.GitHttpUrl]] + ) = { + val endpointName: String Refined NonEmpty = "single-project" + (glClient + .get(_: Uri, _: String Refined NonEmpty)(_: ResponseMappingF[IO, Option[projects.GitHttpUrl]])( + _: Option[AccessToken] + )) + .expects(uri"projects" / slug, endpointName, *, accessToken.some) + .returning(returning) + } + + private lazy val mapResponse: ResponseMappingF[IO, Option[projects.GitHttpUrl]] = + captureMapping(glClient)( + finder + .findGitUrl(projectSlugs.generateOne, userAccessTokens.generateOne) + .unsafeRunSync(), + projectGitHttpUrls.toGeneratorOfOptions + ) + + private lazy val payloadEncoder: Encoder[projects.GitHttpUrl] = Encoder.instance { url => + Json.obj("http_url_to_repo" -> url.asJson) + } +} From defed1a48708f842802272d9234251eeeef15a52 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 17:24:46 +0200 Subject: [PATCH 26/49] feat: UserInfoFinder --- .../projects/update/UserInfoFinder.scala | 54 +++++++++ .../update/ProjectGitUrlFinderSpec.scala | 8 +- .../projects/update/UserInfoFinderSpec.scala | 103 ++++++++++++++++++ .../io/renku/core/client/ProjectUpdates.scala | 4 +- .../scala/io/renku/core/client/UserInfo.scala | 23 ---- 5 files changed, 164 insertions(+), 28 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/UserInfoFinder.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/UserInfoFinderSpec.scala delete mode 100644 renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/UserInfoFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/UserInfoFinder.scala new file mode 100644 index 0000000000..a162c94b28 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/UserInfoFinder.scala @@ -0,0 +1,54 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.Async +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.circe.Decoder +import io.renku.core.client.UserInfo +import io.renku.http.client.{GitLabClient, UserAccessToken} +import io.renku.tinytypes.json.TinyTypeDecoders._ +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe.CirceEntityDecoder._ +import org.http4s.implicits._ +import org.http4s.{Request, Response, Status} + +private trait UserInfoFinder[F[_]] { + def findUserInfo(generateOne: UserAccessToken): F[Option[UserInfo]] +} + +private object UserInfoFinder { + def apply[F[_]: Async: GitLabClient]: UserInfoFinder[F] = new UserInfoFinderImpl[F] +} + +private class UserInfoFinderImpl[F[_]: Async: GitLabClient] extends UserInfoFinder[F] { + + override def findUserInfo(at: UserAccessToken): F[Option[UserInfo]] = + GitLabClient[F] + .get(uri"user", "user")(mapResponse)(at.some) + + private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Option[UserInfo]]] = { + case (Ok, _, resp) => resp.as[Option[UserInfo]] + case (NotFound, _, _) => Option.empty[UserInfo].pure[F] + } + + private implicit lazy val infoDecoder: Decoder[UserInfo] = + Decoder.forProduct2("name", "email")(UserInfo.apply) +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala index 490deb1418..b71b5b4e4d 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectGitUrlFinderSpec.scala @@ -58,7 +58,7 @@ class ProjectGitUrlFinderSpec val accessToken = userAccessTokens.generateOne val maybeGitUrl = projectGitHttpUrls.generateOption - givenProjectGitUrl( + givenFetchProjectGitUrl( slug, accessToken, returning = maybeGitUrl.pure[IO] @@ -81,9 +81,9 @@ class ProjectGitUrlFinderSpec private implicit val glClient: GitLabClient[IO] = mock[GitLabClient[IO]] private lazy val finder = new ProjectGitUrlFinderImpl[IO] - private def givenProjectGitUrl(slug: projects.Slug, - accessToken: AccessToken, - returning: IO[Option[projects.GitHttpUrl]] + private def givenFetchProjectGitUrl(slug: projects.Slug, + accessToken: AccessToken, + returning: IO[Option[projects.GitHttpUrl]] ) = { val endpointName: String Refined NonEmpty = "single-project" (glClient diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/UserInfoFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/UserInfoFinderSpec.scala new file mode 100644 index 0000000000..97fd055349 --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/UserInfoFinderSpec.scala @@ -0,0 +1,103 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.IO +import cats.effect.testing.scalatest.AsyncIOSpec +import cats.syntax.all._ +import eu.timepit.refined.api.Refined +import eu.timepit.refined.auto._ +import eu.timepit.refined.collection.NonEmpty +import io.circe.Encoder +import io.circe.literal._ +import io.circe.syntax._ +import io.renku.core.client.Generators.userInfos +import io.renku.core.client.UserInfo +import io.renku.generators.CommonGraphGenerators.userAccessTokens +import io.renku.generators.Generators.Implicits._ +import io.renku.http.client.RestClient.ResponseMappingF +import io.renku.http.client.{AccessToken, GitLabClient} +import io.renku.testtools.GitLabClientTools +import org.http4s.Status.{NotFound, Ok} +import org.http4s.circe._ +import org.http4s.implicits._ +import org.http4s.{Request, Response, Uri} +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.EitherValues +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class UserInfoFinderSpec + extends AsyncFlatSpec + with AsyncIOSpec + with AsyncMockFactory + with should.Matchers + with EitherValues + with GitLabClientTools[IO] { + + it should "call GL's GET gl/user and return the user info" in { + + val accessToken = userAccessTokens.generateOne + val someUserInfo = userInfos.generateSome + + givenFetchUserInfo( + accessToken, + returning = someUserInfo.pure[IO] + ) + + finder.findUserInfo(accessToken).asserting(_ shouldBe someUserInfo) + } + + it should "return some user info if GL returns 200 with relevant values in the payload" in { + val userInfo = userInfos.generateOne + mapResponse(Ok, Request[IO](), Response[IO](Ok).withEntity(userInfo.asJson)) + .asserting(_ shouldBe Some(userInfo)) + } + + it should "return no user info if GL returns 404 NOT_FOUND" in { + mapResponse(NotFound, Request[IO](), Response[IO](NotFound)) + .asserting(_ shouldBe None) + } + + private implicit val glClient: GitLabClient[IO] = mock[GitLabClient[IO]] + private lazy val finder = new UserInfoFinderImpl[IO] + + private def givenFetchUserInfo(accessToken: AccessToken, returning: IO[Option[UserInfo]]) = { + val endpointName: String Refined NonEmpty = "user" + (glClient + .get(_: Uri, _: String Refined NonEmpty)(_: ResponseMappingF[IO, Option[UserInfo]])(_: Option[AccessToken])) + .expects(uri"user", endpointName, *, accessToken.some) + .returning(returning) + } + + private lazy val mapResponse: ResponseMappingF[IO, Option[UserInfo]] = + captureMapping(glClient)( + finder + .findUserInfo(userAccessTokens.generateOne) + .unsafeRunSync(), + userInfos.toGeneratorOfOptions + ) + + private implicit lazy val payloadEncoder: Encoder[UserInfo] = Encoder.instance { case UserInfo(name, email) => + json"""{ + "name": $name, + "email": $email + }""" + } +} diff --git a/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala b/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala index b67f6e21cb..9b50af81de 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/ProjectUpdates.scala @@ -21,7 +21,9 @@ package io.renku.core.client import cats.syntax.all._ import io.circe.syntax._ import io.circe.{Encoder, Json} -import io.renku.graph.model.projects +import io.renku.graph.model.{persons, projects} + +final case class UserInfo(name: persons.Name, email: persons.Email) final case class ProjectUpdates(projectUrl: projects.GitHttpUrl, userInfo: UserInfo, diff --git a/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala b/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala deleted file mode 100644 index 748ff1f267..0000000000 --- a/renku-core-client/src/main/scala/io/renku/core/client/UserInfo.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Copyright 2023 Swiss Data Science Center (SDSC) - * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and - * Eidgenössische Technische Hochschule Zürich (ETHZ). - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package io.renku.core.client - -import io.renku.graph.model.persons - -final case class UserInfo(name: persons.Name, email: persons.Email) From 6bca77ec2b0df9193b070f43683ceacf34da8bc9 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 18:33:40 +0200 Subject: [PATCH 27/49] feat: ProjectUpdater to check pushing and find git url --- .../projects/update/GLProjectUpdater.scala | 15 ++- .../projects/update/ProjectUpdater.scala | 86 ++++++++++++---- .../update/GLProjectUpdaterSpec.scala | 7 +- .../projects/update/ProjectUpdaterSpec.scala | 98 +++++++++++++++---- 4 files changed, 155 insertions(+), 51 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 5b0d33ea76..bf9c5496e0 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -19,7 +19,6 @@ package io.renku.knowledgegraph.projects.update import cats.MonadThrow -import cats.data.EitherT import cats.effect.Async import cats.syntax.all._ import eu.timepit.refined.auto._ @@ -33,7 +32,7 @@ import org.http4s.implicits._ import org.http4s.{Request, Response, Status, UrlForm} private trait GLProjectUpdater[F[_]] { - def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] + def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Json, Unit]] } private object GLProjectUpdater { @@ -42,13 +41,11 @@ private object GLProjectUpdater { private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectUpdater[F] { - override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): EitherT[F, Json, Unit] = - EitherT { - if ((updates.newImage orElse updates.newVisibility).isDefined) - GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) - else - ().asRight[Json].pure[F] - } + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Json, Unit]] = + if ((updates.newImage orElse updates.newVisibility).isDefined) + GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) + else + ().asRight[Json].pure[F] private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => UrlForm.empty diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 64ece17552..3bca8c271a 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -39,37 +39,46 @@ private trait ProjectUpdater[F[_]] { private object ProjectUpdater { def apply[F[_]: Async: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = - TriplesGeneratorClient[F].map(new ProjectUpdaterImpl[F](BranchProtectionCheck[F], GLProjectUpdater[F], _)) + TriplesGeneratorClient[F].map( + new ProjectUpdaterImpl[F](BranchProtectionCheck[F], ProjectGitUrlFinder[F], GLProjectUpdater[F], _) + ) } private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: BranchProtectionCheck[F], - glProjectUpdater: GLProjectUpdater[F], - tgClient: TriplesGeneratorClient[F] + projectGitUrlFinder: ProjectGitUrlFinder[F], + glProjectUpdater: GLProjectUpdater[F], + tgClient: TriplesGeneratorClient[F] ) extends ProjectUpdater[F] { - import branchProtectionCheck.canPushToDefaultBranch - - override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = { if ((updates.newDescription orElse updates.newKeywords).isEmpty) updateGL(slug, updates, authUser) .flatMap(_ => updateTG(slug, updates)) - .merge else - canPushToDefaultBranch(slug, authUser.accessToken) - .flatMap { - case false => conflictResponse.pure[F] - case true => acceptedResponse.pure[F] + canPushToDefaultBranch(slug, authUser) + .flatMap { _ => + findProjectGitUrl(slug, authUser).as(acceptedResponse) } + }.merge private def updateGL(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser - ): EitherT[F, Response[F], Unit] = - glProjectUpdater.updateProject(slug, updates, authUser.accessToken).leftMap(badRequest) + ): EitherT[F, Response[F], Unit] = EitherT { + glProjectUpdater + .updateProject(slug, updates, authUser.accessToken) + .map(_.leftMap(badRequest)) + .handleErrorWith(glUpdateError(slug)) + } private def badRequest(message: Json): Response[F] = Response[F](BadRequest).withEntity(Message.Error.fromJsonUnsafe(message)) + private def glUpdateError(slug: projects.Slug): Throwable => F[Either[Response[F], Unit]] = + Logger[F] + .error(_)(show"Updating project $slug in GL failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Updating GL failed")).asLeft) + private def updateTG(slug: projects.Slug, updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = EitherT { tgClient @@ -82,19 +91,54 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: Bra ) ) .map(_.toEither) + .handleError(_.asLeft) }.biSemiflatMap( - serverError(slug), + tsUpdateError(slug), _ => acceptedResponse.pure[F] ) - private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = + private def tsUpdateError(slug: projects.Slug): Throwable => F[Response[F]] = Logger[F] - .error(_)(show"Updating project $slug failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) + .error(_)(show"Updating project $slug in TS failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Updating TS failed"))) + + private def canPushToDefaultBranch(slug: projects.Slug, authUser: AuthUser) = EitherT { + branchProtectionCheck + .canPushToDefaultBranch(slug, authUser.accessToken) + .flatMap { + case false => + Response[F](Conflict) + .withEntity(Message.Error("Updating project not possible; the user cannot push to the default branch")) + .asLeft[Unit] + .pure[F] + case true => ().asRight[Response[F]].pure[F] + } + .handleErrorWith(pushCheckError(slug)) + } + + private def pushCheckError(slug: projects.Slug): Throwable => F[Either[Response[F], Unit]] = + Logger[F] + .error(_)(show"Check if pushing to git for $slug possible failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project repository access failed")).asLeft) + + private def findProjectGitUrl(slug: projects.Slug, authUser: AuthUser) = EitherT { + projectGitUrlFinder + .findGitUrl(slug, authUser.accessToken) + .flatMap { + case Some(url) => url.asRight[Response[F]].pure[F] + case None => + Response[F](InternalServerError) + .withEntity(Message.Error("Cannot find project info")) + .asLeft[projects.GitHttpUrl] + .pure[F] + } + .handleErrorWith(findingGLUrlError(slug)) + } + + private def findingGLUrlError(slug: projects.Slug): Throwable => F[Either[Response[F], projects.GitHttpUrl]] = + Logger[F] + .error(_)(show"Finding git url for $slug failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project git url failed")).asLeft) private lazy val acceptedResponse = Response[F](Accepted).withEntity(Message.Info("Project update accepted")) - private lazy val conflictResponse = Response[F](Conflict) - .withEntity( - Message.Error("Updating project not possible; quite likely the user cannot push to the default branch") - ) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index 4eb06e516b..d67dd606a8 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -62,7 +62,7 @@ class GLProjectUpdaterSpec givenEditProjectAPICall(slug, newValues, accessToken, returning = ().asRight.pure[IO]) - finder.updateProject(slug, newValues, accessToken).value.asserting(_.value shouldBe ()) + finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe ()) } it should s"do nothing if neither new image nor visibility is set in the update" in { @@ -71,7 +71,7 @@ class GLProjectUpdaterSpec val newValues = projectUpdatesGen.generateOne.copy(newImage = None, newVisibility = None) val accessToken = accessTokens.generateOne - finder.updateProject(slug, newValues, accessToken).value.asserting(_.value shouldBe ()) + finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe ()) } it should s"call GL's PUT gl/projects/:slug and return GL message if returned" in { @@ -83,7 +83,7 @@ class GLProjectUpdaterSpec val error = jsons.generateOne givenEditProjectAPICall(slug, newValues, accessToken, returning = error.asLeft.pure[IO]) - finder.updateProject(slug, newValues, accessToken).value.asserting(_.left.value shouldBe error) + finder.updateProject(slug, newValues, accessToken).asserting(_.left.value shouldBe error) } it should "succeed if PUT gl/projects/:slug returns 200 OK" in { @@ -139,7 +139,6 @@ class GLProjectUpdaterSpec projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne, accessTokens.generateOne ) - .value .unsafeRunSync(), ().asRight[Json], method = PUT diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 0a1d292de5..225a503362 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -19,7 +19,6 @@ package io.renku.knowledgegraph.projects.update import Generators._ -import cats.data.EitherT import cats.effect.IO import cats.syntax.all._ import eu.timepit.refined.auto._ @@ -31,11 +30,11 @@ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.{exceptions, jsons} import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs import io.renku.graph.model.projects -import io.renku.http.client.AccessToken +import io.renku.http.client.UserAccessToken import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} -import org.http4s.Status.{Accepted, BadRequest, InternalServerError, Conflict} +import org.http4s.Status.{Accepted, BadRequest, Conflict, InternalServerError} import org.http4s.circe._ import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -44,8 +43,8 @@ import org.scalatest.matchers.should class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { it should "if only GL update needed " + - "send update only to GL and TG " + - "and return 202 Accepted when no failures" in { + "send update only to GL and TG and " + + "return 202 Accepted when no failures" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -54,7 +53,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .suchThat(_.onlyGLUpdateNeeded) .generateOne - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) updater.updateProject(slug, updates, authUser) >>= { response => @@ -79,12 +78,65 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .as[Json] .asserting( _ shouldBe Message - .Error("Updating project not possible; quite likely the user cannot push to the default branch") + .Error("Updating project not possible; the user cannot push to the default branch") .asJson ) } } + it should "if core update needed " + + "check if pushing to the default branch is allowed " + + "and return 500 InternalServerError if check failed" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + val exception = exceptions.generateOne + givenBranchProtectionChecking(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error("Finding project repository access failed").asJson) + } + } + + it should "if core update needed but " + + "finding project git url fails " + + "return 500 InternalServerError" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val exception = exceptions.generateOne + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error("Finding project git url failed").asJson) + } + } + + it should "fail if core update needed and " + + "finding project git url returns None" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = None.pure[IO]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error("Cannot find project info").asJson) + } + } + it should "return 400 BadRequest if GL returns 400" in { val authUser = authUsers.generateOne @@ -95,7 +147,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .generateOne val error = jsons.generateOne - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.left(error.pure[IO])) + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = error.asLeft.pure[IO]) updater.updateProject(slug, updates, authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe BadRequest) >> @@ -103,7 +155,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } } - it should "fail if updating GL fails" in { + it should "return 500 InternalServerError if updating GL fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -116,10 +168,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenUpdatingProjectInGL(slug, updates, authUser.accessToken, - returning = EitherT(exception.raiseError[IO, Either[Json, Unit]]) + returning = exception.raiseError[IO, Either[Json, Unit]] ) - updater.updateProject(slug, updates, authUser).assertThrowsError[Exception](_ shouldBe exception) + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Message].asserting(_ shouldBe Message.Error("Updating GL failed")) + } } it should "return 500 InternalServerError if updating project in TG failed" in { @@ -131,7 +186,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .suchThat(_.onlyGLUpdateNeeded) .generateOne - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = EitherT.pure[IO, Json](())) + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) val exception = exceptions.generateOne givenSendingUpdateToTG(slug, updates, @@ -140,25 +195,34 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul updater.updateProject(slug, updates, authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Message].asserting(_ shouldBe Message.Error("Update failed")) + response.as[Message].asserting(_ shouldBe Message.Error("Updating TS failed")) } } private implicit val logger: TestLogger[IO] = TestLogger[IO]() private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] + private val projectGitUrlFinder = mock[ProjectGitUrlFinder[IO]] private val glProjectUpdater = mock[GLProjectUpdater[IO]] private val tgClient = mock[TriplesGeneratorClient[IO]] - private lazy val updater = new ProjectUpdaterImpl[IO](branchProtectionCheck, glProjectUpdater, tgClient) + private lazy val updater = + new ProjectUpdaterImpl[IO](branchProtectionCheck, projectGitUrlFinder, glProjectUpdater, tgClient) - private def givenBranchProtectionChecking(slug: projects.Slug, at: AccessToken, returning: IO[Boolean]) = + private def givenBranchProtectionChecking(slug: projects.Slug, at: UserAccessToken, returning: IO[Boolean]) = (branchProtectionCheck.canPushToDefaultBranch _) .expects(slug, at) .returning(returning) + private def givenProjectGitUrlFinding(slug: projects.Slug, + at: UserAccessToken, + returning: IO[Option[projects.GitHttpUrl]] + ) = (projectGitUrlFinder.findGitUrl _) + .expects(slug, at) + .returning(returning) + private def givenUpdatingProjectInGL(slug: projects.Slug, updates: ProjectUpdates, - at: AccessToken, - returning: EitherT[IO, Json, Unit] + at: UserAccessToken, + returning: IO[Either[Json, Unit]] ) = (glProjectUpdater.updateProject _) .expects(slug, updates, at) .returning(returning) From f25bc698748fad5ca3105e62a61915a422ebafbe Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Thu, 17 Aug 2023 19:58:09 +0200 Subject: [PATCH 28/49] feat: ProjectUpdater to collect relevant data and update in Core --- .../projects/update/Endpoint.scala | 3 +- .../projects/update/ProjectUpdater.scala | 73 +++++++++-- .../projects/update/ProjectUpdaterSpec.scala | 117 +++++++++++++++++- 3 files changed, 173 insertions(+), 20 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 4853951d90..85b8d4d32e 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -18,6 +18,7 @@ package io.renku.knowledgegraph.projects.update +import cats.NonEmptyParallel import cats.data.EitherT import cats.effect.Async import cats.syntax.all._ @@ -37,7 +38,7 @@ trait Endpoint[F[_]] { } object Endpoint { - def apply[F[_]: Async: Logger: MetricsRegistry: GitLabClient]: F[Endpoint[F]] = + def apply[F[_]: Async: NonEmptyParallel: Logger: MetricsRegistry: GitLabClient]: F[Endpoint[F]] = ProjectUpdater[F].map(new EndpointImpl(_)) } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 3bca8c271a..90ae41ca36 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -18,11 +18,13 @@ package io.renku.knowledgegraph.projects.update +import cats.NonEmptyParallel import cats.data.EitherT import cats.effect.Async import cats.syntax.all._ import eu.timepit.refined.auto._ import io.circe.Json +import io.renku.core.client.{RenkuCoreClient, UserInfo, ProjectUpdates => CoreProjectUpdates} import io.renku.data.Message import io.renku.graph.model.projects import io.renku.http.client.GitLabClient @@ -38,16 +40,25 @@ private trait ProjectUpdater[F[_]] { } private object ProjectUpdater { - def apply[F[_]: Async: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = - TriplesGeneratorClient[F].map( - new ProjectUpdaterImpl[F](BranchProtectionCheck[F], ProjectGitUrlFinder[F], GLProjectUpdater[F], _) - ) + def apply[F[_]: Async: NonEmptyParallel: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = + (TriplesGeneratorClient[F], RenkuCoreClient[F]()) + .mapN( + new ProjectUpdaterImpl[F](BranchProtectionCheck[F], + ProjectGitUrlFinder[F], + UserInfoFinder[F], + GLProjectUpdater[F], + _, + _ + ) + ) } -private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: BranchProtectionCheck[F], - projectGitUrlFinder: ProjectGitUrlFinder[F], - glProjectUpdater: GLProjectUpdater[F], - tgClient: TriplesGeneratorClient[F] +private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchProtectionCheck: BranchProtectionCheck[F], + projectGitUrlFinder: ProjectGitUrlFinder[F], + userInfoFinder: UserInfoFinder[F], + glProjectUpdater: GLProjectUpdater[F], + tgClient: TriplesGeneratorClient[F], + renkuCoreClient: RenkuCoreClient[F] ) extends ProjectUpdater[F] { override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = { @@ -56,9 +67,8 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: Bra .flatMap(_ => updateTG(slug, updates)) else canPushToDefaultBranch(slug, authUser) - .flatMap { _ => - findProjectGitUrl(slug, authUser).as(acceptedResponse) - } + .flatMap(_ => findCoreProjectUpdates(slug, updates, authUser)) + .flatMap(updates => findCoreUri(updates, authUser).map(updates -> _).map(_ => acceptedResponse)) }.merge private def updateGL(slug: projects.Slug, @@ -121,7 +131,15 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: Bra .error(_)(show"Check if pushing to git for $slug possible failed") .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project repository access failed")).asLeft) - private def findProjectGitUrl(slug: projects.Slug, authUser: AuthUser) = EitherT { + private def findCoreProjectUpdates(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser) = EitherT { + (findProjectGitUrl(slug, authUser) -> findUserInfo(authUser)) + .parMapN { (maybeProjectGitUrl, maybeUserInfo) => + (maybeProjectGitUrl -> maybeUserInfo) + .mapN(CoreProjectUpdates(_, _, updates.newDescription, updates.newKeywords)) + } + } + + private def findProjectGitUrl(slug: projects.Slug, authUser: AuthUser) = projectGitUrlFinder .findGitUrl(slug, authUser.accessToken) .flatMap { @@ -133,12 +151,41 @@ private class ProjectUpdaterImpl[F[_]: Async: Logger](branchProtectionCheck: Bra .pure[F] } .handleErrorWith(findingGLUrlError(slug)) - } private def findingGLUrlError(slug: projects.Slug): Throwable => F[Either[Response[F], projects.GitHttpUrl]] = Logger[F] .error(_)(show"Finding git url for $slug failed") .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project git url failed")).asLeft) + private def findUserInfo(authUser: AuthUser) = + userInfoFinder + .findUserInfo(authUser.accessToken) + .flatMap { + case Some(info) => info.asRight[Response[F]].pure[F] + case None => + Response[F](InternalServerError) + .withEntity(Message.Error("Cannot find user info")) + .asLeft[UserInfo] + .pure[F] + } + .handleErrorWith(findingUserInfoError(authUser)) + + private def findingUserInfoError(user: AuthUser): Throwable => F[Either[Response[F], UserInfo]] = + Logger[F] + .error(_)(show"Finding userInfo for ${user.id} failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Finding user info failed")).asLeft) + + private def findCoreUri(updates: CoreProjectUpdates, authUser: AuthUser) = EitherT { + renkuCoreClient + .findCoreUri(updates.projectUrl, authUser.accessToken) + .map(_.toEither) + .handleError(_.asLeft) + }.leftSemiflatMap(findingCoreUriError(updates)) + + private def findingCoreUriError(updates: CoreProjectUpdates): Throwable => F[Response[F]] = ex => + Logger[F] + .error(ex)(show"Finding core uri for ${updates.projectUrl} failed") + .as(Response[F](InternalServerError).withEntity(Message.Error.fromExceptionMessage(ex))) + private lazy val acceptedResponse = Response[F](Accepted).withEntity(Message.Info("Project update accepted")) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 225a503362..170e86e1f1 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -24,13 +24,15 @@ import cats.syntax.all._ import eu.timepit.refined.auto._ import io.circe.Json import io.circe.syntax._ +import io.renku.core.client.Generators.{coreUrisVersioned, resultDetailedFailures, userInfos} +import io.renku.core.client.{RenkuCoreClient, RenkuCoreUri, Result, UserInfo, ProjectUpdates => CoreProjectUpdates} import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.authUsers import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.{exceptions, jsons} -import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.RenkuTinyTypeGenerators.{projectGitHttpUrls, projectSlugs} import io.renku.graph.model.projects -import io.renku.http.client.UserAccessToken +import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} @@ -113,6 +115,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val exception = exceptions.generateOne givenProjectGitUrlFinding(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) + givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) updater.updateProject(slug, updates, authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe InternalServerError) >> @@ -120,8 +123,9 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } } - it should "fail if core update needed and " + - "finding project git url returns None" in { + it should "if core update needed and " + + "finding project git url returns None " + + "return 500 InternalServerError" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -130,6 +134,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) givenProjectGitUrlFinding(slug, authUser.accessToken, returning = None.pure[IO]) + givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) updater.updateProject(slug, updates, authUser) >>= { response => response.pure[IO].asserting(_.status shouldBe InternalServerError) >> @@ -137,6 +142,78 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } } + it should "if core update needed and " + + "finding user info returns None " + + "return 500 InternalServerError" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitHttpUrls.generateSome.pure[IO]) + givenUserInfoFinding(authUser.accessToken, returning = None.pure[IO]) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error("Cannot find user info").asJson) + } + } + + it should "if core update needed and " + + "finding renku core URI fails " + + "return 500 InternalServerError" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val projectGitUrl = projectGitHttpUrls.generateOne + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) + givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) + + val failedResult = resultDetailedFailures.generateOne + givenFindingCoreUri(projectGitUrl, authUser.accessToken, returning = failedResult) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error.fromExceptionMessage(failedResult).asJson) + } + } + + it should "if core update needed and " + + "updating renku core fails " + + "return 500 InternalServerError" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val projectGitUrl = projectGitHttpUrls.generateOne + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) + val userInfo = userInfos.generateOne + givenUserInfoFinding(authUser.accessToken, returning = userInfo.some.pure[IO]) + val coreUri = coreUrisVersioned.generateOne + givenFindingCoreUri(projectGitUrl, authUser.accessToken, returning = Result.success(coreUri)) + + val failedResult = resultDetailedFailures.generateOne + givenUpdatingProjectInCore( + coreUri, + CoreProjectUpdates(projectGitUrl, userInfo, updates.newDescription, updates.newKeywords), + authUser.accessToken, + returning = failedResult + ) + + updater.updateProject(slug, updates, authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe InternalServerError) >> + response.as[Json].asserting(_ shouldBe Message.Error.fromExceptionMessage(failedResult).asJson) + } + } + it should "return 400 BadRequest if GL returns 400" in { val authUser = authUsers.generateOne @@ -202,10 +279,17 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul private implicit val logger: TestLogger[IO] = TestLogger[IO]() private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] private val projectGitUrlFinder = mock[ProjectGitUrlFinder[IO]] + private val userInfoFinder = mock[UserInfoFinder[IO]] private val glProjectUpdater = mock[GLProjectUpdater[IO]] private val tgClient = mock[TriplesGeneratorClient[IO]] - private lazy val updater = - new ProjectUpdaterImpl[IO](branchProtectionCheck, projectGitUrlFinder, glProjectUpdater, tgClient) + private val renkuCoreClient = mock[RenkuCoreClient[IO]] + private lazy val updater = new ProjectUpdaterImpl[IO](branchProtectionCheck, + projectGitUrlFinder, + userInfoFinder, + glProjectUpdater, + tgClient, + renkuCoreClient + ) private def givenBranchProtectionChecking(slug: projects.Slug, at: UserAccessToken, returning: IO[Boolean]) = (branchProtectionCheck.canPushToDefaultBranch _) @@ -219,6 +303,11 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .expects(slug, at) .returning(returning) + private def givenUserInfoFinding(at: UserAccessToken, returning: IO[Option[UserInfo]]) = + (userInfoFinder.findUserInfo _) + .expects(at) + .returning(returning) + private def givenUpdatingProjectInGL(slug: projects.Slug, updates: ProjectUpdates, at: UserAccessToken, @@ -240,4 +329,20 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul ) ) .returning(returning) + + private def givenFindingCoreUri(gitUrl: projects.GitHttpUrl, + at: UserAccessToken, + returning: Result[RenkuCoreUri.Versioned] + ) = (renkuCoreClient + .findCoreUri(_: projects.GitHttpUrl, _: AccessToken)) + .expects(gitUrl, at) + .returning(returning.pure[IO]) + + private def givenUpdatingProjectInCore(coreUri: RenkuCoreUri.Versioned, + updates: CoreProjectUpdates, + at: UserAccessToken, + returning: Result[Unit] + ) = (renkuCoreClient.updateProject _) + .expects(coreUri, updates, at) + .returning(returning.pure[IO]) } From 12aa32e2b3d634711865fdede1c363f75f726c83 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 18 Aug 2023 16:46:21 +0200 Subject: [PATCH 29/49] feat: ProjectUpdater and Endpoint implementation --- .../projects/update/Endpoint.scala | 22 ++- .../projects/update/Failure.scala | 81 +++++++++ .../projects/update/GLProjectUpdater.scala | 15 +- .../projects/update/ProjectUpdater.scala | 159 ++++++------------ .../projects/update/EndpointSpec.scala | 47 ++++-- .../update/GLProjectUpdaterSpec.scala | 19 ++- .../projects/update/ProjectUpdaterSpec.scala | 158 ++++++++--------- 7 files changed, 273 insertions(+), 228 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 85b8d4d32e..34518439e6 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -48,9 +48,13 @@ private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F override def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = decodePayload(request) - .semiflatMap(projectUpdater.updateProject(slug, _, authUser)) + .semiflatMap( + projectUpdater + .updateProject(slug, _, authUser) + .as(Response[F](Accepted).withEntity(Message.Info("Project update accepted"))) + ) .merge - .handleErrorWith(serverError(slug)(_)) + .handleErrorWith(relevantError(slug)(_)) private lazy val decodePayload: Request[F] => EitherT[F, Response[F], ProjectUpdates] = req => EitherT { @@ -64,8 +68,14 @@ private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F Response[F](BadRequest).withEntity(Message.Error("Invalid payload")) } - private def serverError(slug: projects.Slug): Throwable => F[Response[F]] = - Logger[F] - .error(_)(show"Updating project $slug failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) + private def relevantError(slug: projects.Slug): Throwable => F[Response[F]] = { + case f: Failure => + Logger[F] + .error(f)(show"Updating project $slug failed") + .as(Response[F](f.status).withEntity(f.message)) + case ex: Exception => + Logger[F] + .error(ex)(show"Updating project $slug failed") + .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) + } } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala new file mode 100644 index 0000000000..39256be1f2 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala @@ -0,0 +1,81 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import io.renku.data.Message +import io.renku.graph.model.{persons, projects} +import org.http4s.Status +import org.http4s.Status.{BadRequest, Conflict, InternalServerError} + +private sealed trait Failure extends Exception { + val status: Status + val message: Message +} + +private object Failure { + + final case class Simple(status: Status, message: Message) extends Exception(message.show) with Failure + final case class WithCause(status: Status, message: Message, cause: Throwable) + extends Exception(message.show, cause) + with Failure + + def apply(status: Status, message: Message): Failure = + Failure.Simple(status, message) + + def apply(status: Status, message: Message, cause: Throwable): Failure = + Failure.WithCause(status, message, cause) + + def badRequestOnGLUpdate(message: Message): Failure = + Failure(BadRequest, message) + + def onGLUpdate(slug: projects.Slug, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Updating project $slug in GitLab failed"), cause) + + def onTSUpdate(slug: projects.Slug, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Updating project $slug in TS failed"), cause) + + val cannotPushToBranch: Failure = + Failure(Conflict, Message.Error("Updating project not possible; the user cannot push to the default branch")) + + def onBranchAccessCheck(slug: projects.Slug, userId: persons.GitLabId, cause: Throwable): Failure = + Failure(InternalServerError, + Message.Error.unsafeApply(show"Check if pushing to git for $slug and $userId failed"), + cause + ) + + val cannotFindProjectGitUrl: Failure = + Failure(InternalServerError, Message.Error("Cannot find project info")) + + def onFindingProjectGitUrl(slug: projects.Slug, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Finding project git url for $slug failed"), cause) + + def cannotFindUserInfo(userId: persons.GitLabId): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Cannot find info about user $userId")) + + def onFindingUserInfo(userId: persons.GitLabId, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Finding info about $userId failed"), cause) + + def onFindingCoreUri(cause: Throwable): Failure = + Failure(Conflict, Message.Error.fromExceptionMessage(cause), cause) + + def onCoreUpdate(cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.fromExceptionMessage(cause), cause) +} diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index bf9c5496e0..28a742b7ad 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -23,6 +23,7 @@ import cats.effect.Async import cats.syntax.all._ import eu.timepit.refined.auto._ import io.circe.{Decoder, Json} +import io.renku.data.Message import io.renku.graph.model.projects import io.renku.http.client.{AccessToken, GitLabClient} import io.renku.http.tinytypes.TinyTypeURIEncoder._ @@ -32,7 +33,7 @@ import org.http4s.implicits._ import org.http4s.{Request, Response, Status, UrlForm} private trait GLProjectUpdater[F[_]] { - def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Json, Unit]] + def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Message, Unit]] } private object GLProjectUpdater { @@ -41,11 +42,11 @@ private object GLProjectUpdater { private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectUpdater[F] { - override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Json, Unit]] = + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Message, Unit]] = if ((updates.newImage orElse updates.newVisibility).isDefined) GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) else - ().asRight[Json].pure[F] + ().asRight[Message].pure[F] private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => UrlForm.empty @@ -53,9 +54,11 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU .updateFormField("visibility", newVisibility.map(_.value)) } - private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Either[Json, Unit]]] = { - case (Ok, _, _) => ().asRight[Json].pure[F] - case (BadRequest, _, response) => response.as[Json](MonadThrow[F], jsonOf(Async[F], errorDecoder)).map(_.asLeft) + private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Either[Message, Unit]]] = { + case (Ok, _, _) => + ().asRight[Message].pure[F] + case (BadRequest, _, response) => + response.as[Json](MonadThrow[F], jsonOf(Async[F], errorDecoder)).map(Message.Error.fromJsonUnsafe).map(_.asLeft) } private lazy val errorDecoder: Decoder[Json] = { cur => diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 90ae41ca36..4e8b20cc5f 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -19,24 +19,18 @@ package io.renku.knowledgegraph.projects.update import cats.NonEmptyParallel -import cats.data.EitherT import cats.effect.Async import cats.syntax.all._ -import eu.timepit.refined.auto._ -import io.circe.Json -import io.renku.core.client.{RenkuCoreClient, UserInfo, ProjectUpdates => CoreProjectUpdates} -import io.renku.data.Message +import io.renku.core.client.{RenkuCoreClient, RenkuCoreUri, UserInfo, ProjectUpdates => CoreProjectUpdates} import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} -import org.http4s.Response -import org.http4s.Status.{Accepted, BadRequest, Conflict, InternalServerError} import org.typelevel.log4cats.Logger private trait ProjectUpdater[F[_]] { - def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] + def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] } private object ProjectUpdater { @@ -61,131 +55,82 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr renkuCoreClient: RenkuCoreClient[F] ) extends ProjectUpdater[F] { - override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Response[F]] = { - if ((updates.newDescription orElse updates.newKeywords).isEmpty) - updateGL(slug, updates, authUser) - .flatMap(_ => updateTG(slug, updates)) + override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] = + if (updates.onlyGLUpdateNeeded) + updateGL(slug, updates, authUser) >> updateTG(slug, updates) else - canPushToDefaultBranch(slug, authUser) - .flatMap(_ => findCoreProjectUpdates(slug, updates, authUser)) - .flatMap(updates => findCoreUri(updates, authUser).map(updates -> _).map(_ => acceptedResponse)) - }.merge + canPushToDefaultBranch(slug, authUser) >> { + for { + coreUpdates <- findCoreProjectUpdates(slug, updates, authUser) + coreUri <- findCoreUri(coreUpdates, authUser) + _ <- updateCore(coreUri, coreUpdates, authUser) + _ <- updateGL(slug, updates, authUser) + _ <- updateTG(slug, updates) + } yield () + } - private def updateGL(slug: projects.Slug, - updates: ProjectUpdates, - authUser: AuthUser - ): EitherT[F, Response[F], Unit] = EitherT { + private def updateGL(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] = glProjectUpdater .updateProject(slug, updates, authUser.accessToken) - .map(_.leftMap(badRequest)) - .handleErrorWith(glUpdateError(slug)) - } - - private def badRequest(message: Json): Response[F] = - Response[F](BadRequest).withEntity(Message.Error.fromJsonUnsafe(message)) - - private def glUpdateError(slug: projects.Slug): Throwable => F[Either[Response[F], Unit]] = - Logger[F] - .error(_)(show"Updating project $slug in GL failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Updating GL failed")).asLeft) - - private def updateTG(slug: projects.Slug, updates: ProjectUpdates): EitherT[F, Response[F], Response[F]] = - EitherT { - tgClient - .updateProject( - slug, - TGProjectUpdates(newDescription = updates.newDescription, - newImages = updates.newImage.map(_.toList), - newKeywords = updates.newKeywords, - newVisibility = updates.newVisibility - ) + .adaptError(Failure.onGLUpdate(slug, _)) + .flatMap(_.fold(errMsg => Failure.badRequestOnGLUpdate(errMsg).raiseError[F, Unit], _.pure[F])) + + private def updateTG(slug: projects.Slug, updates: ProjectUpdates): F[Unit] = + tgClient + .updateProject( + slug, + TGProjectUpdates(newDescription = updates.newDescription, + newImages = updates.newImage.map(_.toList), + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility ) - .map(_.toEither) - .handleError(_.asLeft) - }.biSemiflatMap( - tsUpdateError(slug), - _ => acceptedResponse.pure[F] - ) - - private def tsUpdateError(slug: projects.Slug): Throwable => F[Response[F]] = - Logger[F] - .error(_)(show"Updating project $slug in TS failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Updating TS failed"))) + ) + .map(_.toEither) + .handleError(_.asLeft) + .flatMap(_.fold(Failure.onTSUpdate(slug, _).raiseError[F, Unit], _.pure[F])) - private def canPushToDefaultBranch(slug: projects.Slug, authUser: AuthUser) = EitherT { + private def canPushToDefaultBranch(slug: projects.Slug, authUser: AuthUser): F[Unit] = branchProtectionCheck .canPushToDefaultBranch(slug, authUser.accessToken) + .adaptError(Failure.onBranchAccessCheck(slug, authUser.id, _)) .flatMap { - case false => - Response[F](Conflict) - .withEntity(Message.Error("Updating project not possible; the user cannot push to the default branch")) - .asLeft[Unit] - .pure[F] - case true => ().asRight[Response[F]].pure[F] + case false => Failure.cannotPushToBranch.raiseError[F, Unit] + case true => ().pure[F] } - .handleErrorWith(pushCheckError(slug)) - } - private def pushCheckError(slug: projects.Slug): Throwable => F[Either[Response[F], Unit]] = - Logger[F] - .error(_)(show"Check if pushing to git for $slug possible failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project repository access failed")).asLeft) - - private def findCoreProjectUpdates(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser) = EitherT { + private def findCoreProjectUpdates(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser) = (findProjectGitUrl(slug, authUser) -> findUserInfo(authUser)) - .parMapN { (maybeProjectGitUrl, maybeUserInfo) => - (maybeProjectGitUrl -> maybeUserInfo) - .mapN(CoreProjectUpdates(_, _, updates.newDescription, updates.newKeywords)) - } - } + .parMapN(CoreProjectUpdates(_, _, updates.newDescription, updates.newKeywords)) - private def findProjectGitUrl(slug: projects.Slug, authUser: AuthUser) = + private def findProjectGitUrl(slug: projects.Slug, authUser: AuthUser): F[projects.GitHttpUrl] = projectGitUrlFinder .findGitUrl(slug, authUser.accessToken) + .adaptError(Failure.onFindingProjectGitUrl(slug, _)) .flatMap { - case Some(url) => url.asRight[Response[F]].pure[F] - case None => - Response[F](InternalServerError) - .withEntity(Message.Error("Cannot find project info")) - .asLeft[projects.GitHttpUrl] - .pure[F] + case Some(url) => url.pure[F] + case None => Failure.cannotFindProjectGitUrl.raiseError[F, projects.GitHttpUrl] } - .handleErrorWith(findingGLUrlError(slug)) - private def findingGLUrlError(slug: projects.Slug): Throwable => F[Either[Response[F], projects.GitHttpUrl]] = - Logger[F] - .error(_)(show"Finding git url for $slug failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Finding project git url failed")).asLeft) - - private def findUserInfo(authUser: AuthUser) = + private def findUserInfo(authUser: AuthUser): F[UserInfo] = userInfoFinder .findUserInfo(authUser.accessToken) + .adaptError(Failure.onFindingUserInfo(authUser.id, _)) .flatMap { - case Some(info) => info.asRight[Response[F]].pure[F] - case None => - Response[F](InternalServerError) - .withEntity(Message.Error("Cannot find user info")) - .asLeft[UserInfo] - .pure[F] + case Some(info) => info.pure[F] + case None => Failure.cannotFindUserInfo(authUser.id).raiseError[F, UserInfo] } - .handleErrorWith(findingUserInfoError(authUser)) - - private def findingUserInfoError(user: AuthUser): Throwable => F[Either[Response[F], UserInfo]] = - Logger[F] - .error(_)(show"Finding userInfo for ${user.id} failed") - .as(Response[F](InternalServerError).withEntity(Message.Error("Finding user info failed")).asLeft) - private def findCoreUri(updates: CoreProjectUpdates, authUser: AuthUser) = EitherT { + private def findCoreUri(updates: CoreProjectUpdates, authUser: AuthUser): F[RenkuCoreUri.Versioned] = renkuCoreClient .findCoreUri(updates.projectUrl, authUser.accessToken) .map(_.toEither) .handleError(_.asLeft) - }.leftSemiflatMap(findingCoreUriError(updates)) + .flatMap(_.fold(Failure.onFindingCoreUri(_).raiseError[F, RenkuCoreUri.Versioned], _.pure[F])) - private def findingCoreUriError(updates: CoreProjectUpdates): Throwable => F[Response[F]] = ex => - Logger[F] - .error(ex)(show"Finding core uri for ${updates.projectUrl} failed") - .as(Response[F](InternalServerError).withEntity(Message.Error.fromExceptionMessage(ex))) - - private lazy val acceptedResponse = Response[F](Accepted).withEntity(Message.Info("Project update accepted")) + private def updateCore(coreUri: RenkuCoreUri.Versioned, updates: CoreProjectUpdates, authUser: AuthUser): F[Unit] = + renkuCoreClient + .updateProject(coreUri, updates, authUser.accessToken) + .map(_.toEither) + .handleError(_.asLeft) + .flatMap(_.fold(Failure.onCoreUpdate(_).raiseError[F, Unit], _.pure[F])) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index e390183ff2..4680cf3cf2 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -25,34 +25,36 @@ import eu.timepit.refined.auto._ import io.circe.literal._ import io.circe.syntax._ import io.renku.data.Message -import io.renku.generators.CommonGraphGenerators.{authUsers, httpStatuses} +import io.renku.generators.CommonGraphGenerators.authUsers import io.renku.generators.Generators.Implicits._ -import io.renku.generators.Generators.{exceptions, sentences} +import io.renku.generators.Generators.{exceptions, nonBlankStrings} import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs import io.renku.graph.model.projects import io.renku.http.server.EndpointTester._ import io.renku.http.server.security.model.AuthUser import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec -import org.http4s.{Request, Response, Status} +import org.http4s.{Request, Status} +import org.scalacheck.Gen import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.matchers.should class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { - it should "update the project and return the status got from the updater" in { + it should "update the project and return Accepted on success" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne val updates = projectUpdatesGen.generateOne - val response = Response[IO](httpStatuses.generateOne).withEntity(Message.Info(sentences().generateOne).asJson) - givenUpdatingProject(slug, updates, authUser, returning = response.pure[IO]) + givenUpdatingProject(slug, updates, authUser, returning = ().pure[IO]) endpoint - .`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) - .asserting(_ shouldBe response) + .`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> + response.as[Message].asserting(_ shouldBe Message.Info("Project update accepted")) + } } it should "return 400 BadRequest if payload is malformed" in { @@ -69,7 +71,28 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc } } - it should "return 500 InternalServerError if updating project fails" in { + it should "return response with the status from the known exception" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + val failure = Gen + .oneOf( + Failure.badRequestOnGLUpdate(Message.Error(nonBlankStrings().generateOne)), + Failure.onGLUpdate(slug, exceptions.generateOne), + Failure.cannotPushToBranch + ) + .generateOne + givenUpdatingProject(slug, updates, authUser, returning = failure.raiseError[IO, Nothing]) + + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe failure.status) >> + response.as[Message].asserting(_ shouldBe failure.message) + } + } + + it should "return 500 InternalServerError if updating project fails with an unknown exception" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -89,10 +112,10 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc private lazy val endpoint = new EndpointImpl[IO](projectUpdater) private def givenUpdatingProject(slug: projects.Slug, - udpates: ProjectUpdates, + updates: ProjectUpdates, authUser: AuthUser, - returning: IO[Response[IO]] + returning: IO[Unit] ) = (projectUpdater.updateProject _) - .expects(slug, udpates, authUser) + .expects(slug, updates, authUser) .returning(returning) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index d67dd606a8..f1064f129b 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -27,6 +27,7 @@ import eu.timepit.refined.auto._ import eu.timepit.refined.collection.NonEmpty import io.circe.Json import io.circe.literal._ +import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.accessTokens import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.{jsons, nonEmptyStrings} @@ -80,10 +81,12 @@ class GLProjectUpdaterSpec val newValues = projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne val accessToken = accessTokens.generateOne - val error = jsons.generateOne + val error = Message.Error.fromJsonUnsafe(jsons.generateOne) givenEditProjectAPICall(slug, newValues, accessToken, returning = error.asLeft.pure[IO]) - finder.updateProject(slug, newValues, accessToken).asserting(_.left.value shouldBe error) + finder + .updateProject(slug, newValues, accessToken) + .asserting(_.left.value shouldBe error) } it should "succeed if PUT gl/projects/:slug returns 200 OK" in { @@ -95,7 +98,7 @@ class GLProjectUpdaterSpec val error = nonEmptyStrings().generateOne mapResponse(BadRequest, Request[IO](), Response[IO](BadRequest).withEntity(json"""{"error": $error}""")) - .asserting(_.left.value shouldBe Json.fromString(error)) + .asserting(_.left.value shouldBe Message.Error.fromJsonUnsafe(Json.fromString(error))) } it should "return left if PUT gl/projects/:slug returns 400 BAD_REQUEST with a message" in { @@ -103,7 +106,7 @@ class GLProjectUpdaterSpec val message = jsons.generateOne mapResponse(BadRequest, Request[IO](), Response[IO](BadRequest).withEntity(json"""{"message": $message}""")) - .asserting(_.left.value shouldBe message) + .asserting(_.left.value shouldBe Message.Error.fromJsonUnsafe(message)) } private implicit val glClient: GitLabClient[IO] = mock[GitLabClient[IO]] @@ -112,11 +115,11 @@ class GLProjectUpdaterSpec private def givenEditProjectAPICall(slug: projects.Slug, newValues: ProjectUpdates, accessToken: AccessToken, - returning: IO[Either[Json, Unit]] + returning: IO[Either[Message, Unit]] ) = { val endpointName: String Refined NonEmpty = "edit-project" (glClient - .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Json, Unit]])( + .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Message, Unit]])( _: Option[AccessToken] )) .expects(uri"projects" / slug, endpointName, toUrlForm(newValues), *, accessToken.some) @@ -132,7 +135,7 @@ class GLProjectUpdaterSpec ) } - private lazy val mapResponse: ResponseMappingF[IO, Either[Json, Unit]] = + private lazy val mapResponse: ResponseMappingF[IO, Either[Message, Unit]] = captureMapping(glClient)( finder .updateProject(projectSlugs.generateOne, @@ -140,7 +143,7 @@ class GLProjectUpdaterSpec accessTokens.generateOne ) .unsafeRunSync(), - ().asRight[Json], + ().asRight[Message], method = PUT ) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 170e86e1f1..10031699ab 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -21,9 +21,6 @@ package io.renku.knowledgegraph.projects.update import Generators._ import cats.effect.IO import cats.syntax.all._ -import eu.timepit.refined.auto._ -import io.circe.Json -import io.circe.syntax._ import io.renku.core.client.Generators.{coreUrisVersioned, resultDetailedFailures, userInfos} import io.renku.core.client.{RenkuCoreClient, RenkuCoreUri, Result, UserInfo, ProjectUpdates => CoreProjectUpdates} import io.renku.data.Message @@ -36,8 +33,6 @@ import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} -import org.http4s.Status.{Accepted, BadRequest, Conflict, InternalServerError} -import org.http4s.circe._ import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.matchers.should @@ -45,8 +40,7 @@ import org.scalatest.matchers.should class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { it should "if only GL update needed " + - "send update only to GL and TG and " + - "return 202 Accepted when no failures" in { + "send update only to GL and TG" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -58,15 +52,11 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe Accepted) >> - response.as[Json].asserting(_ shouldBe Message.Info("Project update accepted").asJson) - } + updater.updateProject(slug, updates, authUser).assertNoException } - it should "if core update needed " + - "check if pushing to the default branch is allowed " + - "and return 409 Conflict if it's not" in { + it should "if core update needed, " + + "fail if pushing to the default branch check return false" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -74,21 +64,11 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenBranchProtectionChecking(slug, authUser.accessToken, returning = false.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe Conflict) >> - response - .as[Json] - .asserting( - _ shouldBe Message - .Error("Updating project not possible; the user cannot push to the default branch") - .asJson - ) - } + updater.updateProject(slug, updates, authUser).assertThrowsError[Exception](_ shouldBe Failure.cannotPushToBranch) } - it should "if core update needed " + - "check if pushing to the default branch is allowed " + - "and return 500 InternalServerError if check failed" in { + it should "if core update needed, " + + "fail if pushing to the default branch check fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -97,15 +77,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val exception = exceptions.generateOne givenBranchProtectionChecking(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error("Finding project repository access failed").asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onBranchAccessCheck(slug, authUser.id, exception)) } - it should "if core update needed but " + - "finding project git url fails " + - "return 500 InternalServerError" in { + it should "if core update needed, " + + "fail if finding project git url fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -117,15 +95,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenProjectGitUrlFinding(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error("Finding project git url failed").asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onFindingProjectGitUrl(slug, exception)) } - it should "if core update needed and " + - "finding project git url returns None " + - "return 500 InternalServerError" in { + it should "if core update needed, " + + "fail if finding project git url returns None" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -136,15 +112,30 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenProjectGitUrlFinding(slug, authUser.accessToken, returning = None.pure[IO]) givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error("Cannot find project info").asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.cannotFindProjectGitUrl) } - it should "if core update needed and " + - "finding user info returns None " + - "return 500 InternalServerError" in { + it should "if core update needed, " + + "fail finding user info fails" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitHttpUrls.generateSome.pure[IO]) + val exception = exceptions.generateOne + givenUserInfoFinding(authUser.accessToken, returning = exception.raiseError[IO, Nothing]) + + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onFindingUserInfo(authUser.id, exception)) + } + + it should "if core update needed, " + + "fail finding user info returns None" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -154,15 +145,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitHttpUrls.generateSome.pure[IO]) givenUserInfoFinding(authUser.accessToken, returning = None.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error("Cannot find user info").asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.cannotFindUserInfo(authUser.id)) } - it should "if core update needed and " + - "finding renku core URI fails " + - "return 500 InternalServerError" in { + it should "if core update needed, " + + "fail if finding renku core URI fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -177,15 +166,13 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val failedResult = resultDetailedFailures.generateOne givenFindingCoreUri(projectGitUrl, authUser.accessToken, returning = failedResult) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error.fromExceptionMessage(failedResult).asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onFindingCoreUri(failedResult)) } - it should "if core update needed and " + - "updating renku core fails " + - "return 500 InternalServerError" in { + it should "if core update needed, " + + "fail if updating renku core fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -208,13 +195,12 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul returning = failedResult ) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Json].asserting(_ shouldBe Message.Error.fromExceptionMessage(failedResult).asJson) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onCoreUpdate(failedResult)) } - it should "return 400 BadRequest if GL returns 400" in { + it should "fail if updating GL returns an error" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -223,16 +209,15 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .suchThat(_.onlyGLUpdateNeeded) .generateOne - val error = jsons.generateOne + val error = Message.Error.fromJsonUnsafe(jsons.generateOne) givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = error.asLeft.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe BadRequest) >> - response.as[Message].asserting(_ shouldBe Message.Error.fromJsonUnsafe(error)) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.badRequestOnGLUpdate(error)) } - it should "return 500 InternalServerError if updating GL fails" in { + it should "fail if updating GL fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -245,16 +230,15 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenUpdatingProjectInGL(slug, updates, authUser.accessToken, - returning = exception.raiseError[IO, Either[Json, Unit]] + returning = exception.raiseError[IO, Either[Message, Unit]] ) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Message].asserting(_ shouldBe Message.Error("Updating GL failed")) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onGLUpdate(slug, exception)) } - it should "return 500 InternalServerError if updating project in TG failed" in { + it should "fail if updating TG failed" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -264,16 +248,12 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .generateOne givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) - val exception = exceptions.generateOne - givenSendingUpdateToTG(slug, - updates, - returning = TriplesGeneratorClient.Result.failure(exception.getMessage).pure[IO] - ) + val exception = TriplesGeneratorClient.Result.Failure(exceptions.generateOne.getMessage) + givenSendingUpdateToTG(slug, updates, returning = exception.pure[IO]) - updater.updateProject(slug, updates, authUser) >>= { response => - response.pure[IO].asserting(_.status shouldBe InternalServerError) >> - response.as[Message].asserting(_ shouldBe Message.Error("Updating TS failed")) - } + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onTSUpdate(slug, exception)) } private implicit val logger: TestLogger[IO] = TestLogger[IO]() @@ -311,7 +291,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul private def givenUpdatingProjectInGL(slug: projects.Slug, updates: ProjectUpdates, at: UserAccessToken, - returning: IO[Either[Json, Unit]] + returning: IO[Either[Message, Unit]] ) = (glProjectUpdater.updateProject _) .expects(slug, updates, at) .returning(returning) From 7fc18352f9cff3a1926183afbcbd9f81f220ffd6 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 18 Aug 2023 18:31:39 +0200 Subject: [PATCH 30/49] feat: ProjectUpdater to run update on core in a fiber --- .../io/renku/interpreters/TestLogger.scala | 33 +++++++++++-- .../projects/update/Failure.scala | 3 -- .../projects/update/ProjectUpdater.scala | 11 +++-- .../projects/update/ProjectUpdaterSpec.scala | 47 +++++++++++++++---- 4 files changed, 75 insertions(+), 19 deletions(-) diff --git a/graph-commons/src/test/scala/io/renku/interpreters/TestLogger.scala b/graph-commons/src/test/scala/io/renku/interpreters/TestLogger.scala index 4ff8f24415..2fce94197a 100644 --- a/graph-commons/src/test/scala/io/renku/interpreters/TestLogger.scala +++ b/graph-commons/src/test/scala/io/renku/interpreters/TestLogger.scala @@ -19,23 +19,25 @@ package io.renku.interpreters import cats.Show -import cats.effect.kernel.Sync +import cats.effect.Async import cats.syntax.all._ +import fs2.Stream import io.renku.interpreters.TestLogger.LogMessage._ import org.scalatest.matchers.should -import org.scalatest.{Assertion, Succeeded} +import org.scalatest.{Assertion, Failed, Outcome, Succeeded} import org.typelevel.log4cats.Logger import java.util.concurrent.ConcurrentLinkedQueue +import scala.concurrent.duration._ import scala.jdk.CollectionConverters._ -class TestLogger[F[_]: Sync] extends Logger[F] with should.Matchers { +class TestLogger[F[_]: Async] extends Logger[F] with should.Matchers { import TestLogger.Level._ import TestLogger._ import LogMessage._ - private[this] val F = Sync[F] + private[this] val F = Async[F] private[this] val invocations = new ConcurrentLinkedQueue[LogEntry]() def getMessages(severity: Level): List[LogMessage] = @@ -70,6 +72,27 @@ class TestLogger[F[_]: Sync] extends Logger[F] with should.Matchers { def reset(): Unit = invocations.clear() + def waitFor(expected: LogEntry*): F[Assertion] = { + + val interval = 100 millis + val attempts = 20 + + val logCheck = + F.delay(loggedOnly(expected.toList)) + .as(Succeeded) + .widen[Outcome] + .handleError(Failed(_)) + + val stream = Stream.eval(logCheck) ++ + Stream + .awakeDelay[F](interval) + .void + .evalMap(_ => logCheck) + .take(attempts) + .takeThrough(_ != Succeeded) + stream.covary[F].compile.lastOrError.map(_ shouldBe Succeeded) + } + private def add(entry: LogEntry): F[Unit] = F.delay { invocations.add(entry) () @@ -119,7 +142,7 @@ class TestLogger[F[_]: Sync] extends Logger[F] with should.Matchers { object TestLogger { - def apply[F[_]: Sync](): TestLogger[F] = new TestLogger[F] + def apply[F[_]: Async](): TestLogger[F] = new TestLogger[F] private[TestLogger] case class LogEntry(level: Level, message: LogMessage) { override lazy val toString = show"\n\t$level: $message" diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala index 39256be1f2..9f3382cc39 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala @@ -75,7 +75,4 @@ private object Failure { def onFindingCoreUri(cause: Throwable): Failure = Failure(Conflict, Message.Error.fromExceptionMessage(cause), cause) - - def onCoreUpdate(cause: Throwable): Failure = - Failure(InternalServerError, Message.Error.fromExceptionMessage(cause), cause) } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 4e8b20cc5f..6f1950cea0 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -63,7 +63,7 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr for { coreUpdates <- findCoreProjectUpdates(slug, updates, authUser) coreUri <- findCoreUri(coreUpdates, authUser) - _ <- updateCore(coreUri, coreUpdates, authUser) + _ <- updateCore(slug, coreUri, coreUpdates, authUser) _ <- updateGL(slug, updates, authUser) _ <- updateTG(slug, updates) } yield () @@ -127,10 +127,15 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr .handleError(_.asLeft) .flatMap(_.fold(Failure.onFindingCoreUri(_).raiseError[F, RenkuCoreUri.Versioned], _.pure[F])) - private def updateCore(coreUri: RenkuCoreUri.Versioned, updates: CoreProjectUpdates, authUser: AuthUser): F[Unit] = + private def updateCore(slug: projects.Slug, + coreUri: RenkuCoreUri.Versioned, + updates: CoreProjectUpdates, + authUser: AuthUser + ): F[Unit] = Async[F].start { renkuCoreClient .updateProject(coreUri, updates, authUser.accessToken) .map(_.toEither) .handleError(_.asLeft) - .flatMap(_.fold(Failure.onCoreUpdate(_).raiseError[F, Unit], _.pure[F])) + .flatMap(_.fold(Logger[F].error(_)(show"Updating project $slug failed"), _.pure[F])) + }.void } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 10031699ab..348ef6db71 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -21,7 +21,7 @@ package io.renku.knowledgegraph.projects.update import Generators._ import cats.effect.IO import cats.syntax.all._ -import io.renku.core.client.Generators.{coreUrisVersioned, resultDetailedFailures, userInfos} +import io.renku.core.client.Generators.{coreUrisVersioned, resultDetailedFailures, resultSuccesses, userInfos} import io.renku.core.client.{RenkuCoreClient, RenkuCoreUri, Result, UserInfo, ProjectUpdates => CoreProjectUpdates} import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.authUsers @@ -31,6 +31,7 @@ import io.renku.graph.model.RenkuTinyTypeGenerators.{projectGitHttpUrls, project import io.renku.graph.model.projects import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger +import io.renku.interpreters.TestLogger.Level.Error import io.renku.testtools.CustomAsyncIOSpec import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} import org.scalamock.scalatest.AsyncMockFactory @@ -39,8 +40,8 @@ import org.scalatest.matchers.should class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { - it should "if only GL update needed " + - "send update only to GL and TG" in { + it should "if only GL update needed, " + + "send update only to GL and TG and succeed on success" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -55,6 +56,34 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul updater.updateProject(slug, updates, authUser).assertNoException } + it should "if core update needed, " + + "send update to Core, GL and TG and succeed on success" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val projectGitUrl = projectGitHttpUrls.generateOne + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) + val userInfo = userInfos.generateOne + givenUserInfoFinding(authUser.accessToken, returning = userInfo.some.pure[IO]) + val coreUri = coreUrisVersioned.generateOne + givenFindingCoreUri(projectGitUrl, authUser.accessToken, returning = Result.success(coreUri)) + + givenUpdatingProjectInCore( + coreUri, + CoreProjectUpdates(projectGitUrl, userInfo, updates.newDescription, updates.newKeywords), + authUser.accessToken, + returning = resultSuccesses(()).generateOne + ) + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) + givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + + updater.updateProject(slug, updates, authUser).assertNoException + } + it should "if core update needed, " + "fail if pushing to the default branch check return false" in { @@ -172,7 +201,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } it should "if core update needed, " + - "fail if updating renku core fails" in { + "log an error and succeed if updating renku core fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -194,10 +223,12 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul authUser.accessToken, returning = failedResult ) + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) + givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) - updater - .updateProject(slug, updates, authUser) - .assertThrowsError[Exception](_ shouldBe Failure.onCoreUpdate(failedResult)) + updater.updateProject(slug, updates, authUser).assertNoException >> { + logger.waitFor(Error(show"Updating project $slug failed", failedResult)) + } } it should "fail if updating GL returns an error" in { @@ -256,7 +287,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .assertThrowsError[Exception](_ shouldBe Failure.onTSUpdate(slug, exception)) } - private implicit val logger: TestLogger[IO] = TestLogger[IO]() + private implicit lazy val logger: TestLogger[IO] = TestLogger[IO]() private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] private val projectGitUrlFinder = mock[ProjectGitUrlFinder[IO]] private val userInfoFinder = mock[UserInfoFinder[IO]] From 057403a493946e1389de6cb2d4d205dfb36fdc47 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 18 Aug 2023 18:55:05 +0200 Subject: [PATCH 31/49] chore: fixing core urls in the config for testing --- renku-core-client/src/main/resources/application.conf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index 19812442f4..1f043adad9 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -2,19 +2,19 @@ services { renku-core-current { - url = "http://localhost:80" + url = "http://renku-kg-dev-core-v10" url = ${?RENKU_CORE_CURRENT_URL} } renku-core-v10 { schemaVersion = "10" - url = "http://localhost:80" + url = "http://renku-kg-dev-core-v10" url = ${?RENKU_CORE_V10_URL} } renku-core-v9 { schemaVersion = "9" - url = "http://localhost:80" + url = "http://renku-kg-dev-core-v9" url = ${?RENKU_CORE_V9_URL} } } From d4b0f426db0ab9f102971604a4ee4f4d52867c48 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 18 Aug 2023 18:57:01 +0200 Subject: [PATCH 32/49] chore: fixing core urls in the config for testing --- renku-core-client/src/main/resources/application.conf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index 1f043adad9..8db548114e 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -2,19 +2,19 @@ services { renku-core-current { - url = "http://renku-kg-dev-core-v10" + url = "http://renku-ci-gr-1652-core-v10" url = ${?RENKU_CORE_CURRENT_URL} } renku-core-v10 { schemaVersion = "10" - url = "http://renku-kg-dev-core-v10" + url = "http://renku-ci-gr-1652-core-v10" url = ${?RENKU_CORE_V10_URL} } renku-core-v9 { schemaVersion = "9" - url = "http://renku-kg-dev-core-v9" + url = "http://renku-ci-gr-1652-core-v9" url = ${?RENKU_CORE_V9_URL} } } From 737cf70980436778ede7d522f568d5ee6066ebf4 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 13:56:09 +0200 Subject: [PATCH 33/49] refactor: RenkuCoreUri.Current -> RenkuCoreUri.Latest --- .../src/main/resources/application.conf | 4 ++-- .../io/renku/core/client/LowLevelApis.scala | 8 ++++---- .../io/renku/core/client/RenkuCoreClient.scala | 2 +- .../io/renku/core/client/RenkuCoreUri.scala | 8 ++++---- .../scala/io/renku/core/client/Generators.scala | 4 ++-- .../io/renku/core/client/LowLevelApisSpec.scala | 2 +- .../io/renku/core/client/RenkuCoreUriSpec.scala | 16 ++++++++-------- 7 files changed, 22 insertions(+), 22 deletions(-) diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index 8db548114e..c513356f5e 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -1,9 +1,9 @@ services { - renku-core-current { + renku-core-latest { url = "http://renku-ci-gr-1652-core-v10" - url = ${?RENKU_CORE_CURRENT_URL} + url = ${?RENKU_CORE_LATEST_URL} } renku-core-v10 { diff --git a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala index ff2ef3edc7..d92fb0d01c 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala @@ -49,11 +49,11 @@ private trait LowLevelApis[F[_]] { } private object LowLevelApis { - def apply[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current): LowLevelApis[F] = - new LowLevelApisImpl[F](coreCurrentUri, ClientTools[F]) + def apply[F[_]: Async: Logger](coreLatestUri: RenkuCoreUri.Latest): LowLevelApis[F] = + new LowLevelApisImpl[F](coreLatestUri, ClientTools[F]) } -private class LowLevelApisImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri.Current, clientTools: ClientTools[F]) +private class LowLevelApisImpl[F[_]: Async: Logger](coreLatestUri: RenkuCoreUri.Latest, clientTools: ClientTools[F]) extends RestClient[F, Nothing](Throttler.noThrottling) with LowLevelApis[F] with Http4sDsl[F] @@ -95,7 +95,7 @@ private class LowLevelApisImpl[F[_]: Async: Logger](coreCurrentUri: RenkuCoreUri res.downField("versions").as(decodeList(singleVersionDecoder)) } - send(GET(coreCurrentUri.uri / "renku" / "versions")) { + send(GET(coreLatestUri.uri / "renku" / "versions")) { case (Ok, _, resp) => toResult[List[SchemaVersion]](resp)(decoder) case reqInfo => toFailure[List[SchemaVersion]]("Version info cannot be found")(reqInfo) } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index 49f6ecde4d..6d7b7edb69 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -41,7 +41,7 @@ trait RenkuCoreClient[F[_]] { object RenkuCoreClient { def apply[F[_]: Async: Logger](config: Config = ConfigFactory.load): F[RenkuCoreClient[F]] = - RenkuCoreUri.Current.loadFromConfig[F](config).map { coreCurrentUri => + RenkuCoreUri.Latest.loadFromConfig[F](config).map { coreCurrentUri => new RenkuCoreClientImpl[F](RenkuCoreUri.ForSchema, LowLevelApis[F](coreCurrentUri), config) } } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala index 7213ff1e2a..8a7e5c5d40 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala @@ -31,15 +31,15 @@ sealed trait RenkuCoreUri { object RenkuCoreUri { - final case class Current(uri: Uri) extends RenkuCoreUri + final case class Latest(uri: Uri) extends RenkuCoreUri final case class ForSchema(uri: Uri, schemaVersion: SchemaVersion) extends RenkuCoreUri final case class Versioned(baseUri: ForSchema, apiVersion: ApiVersion) extends RenkuCoreUri { val uri: Uri = baseUri.uri / apiVersion.value } - object Current { - def loadFromConfig[F[_]: MonadThrow](config: Config = ConfigFactory.load): F[Current] = - loadUrlFromConfig("services.renku-core-current.url", config).map(Current(_)) + object Latest { + def loadFromConfig[F[_]: MonadThrow](config: Config = ConfigFactory.load): F[Latest] = + loadUrlFromConfig("services.renku-core-latest.url", config).map(Latest(_)) } trait ForSchemaLoader { diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index 5dc9566fad..10492932f6 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -43,8 +43,8 @@ object Generators { implicit lazy val migrationRequiredGen: Gen[MigrationRequired] = Gen.oneOf(MigrationRequired.yes, MigrationRequired.no) - implicit lazy val coreCurrentUris: Gen[RenkuCoreUri.Current] = - httpUrls().map(uri => RenkuCoreUri.Current(Uri.unsafeFromString(uri))) + implicit lazy val coreLatestUris: Gen[RenkuCoreUri.Latest] = + httpUrls().map(uri => RenkuCoreUri.Latest(Uri.unsafeFromString(uri))) implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = for { diff --git a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala index 4841ad43bd..03f558d15c 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala @@ -133,5 +133,5 @@ class LowLevelApisSpec } private implicit val logger: Logger[IO] = TestLogger() - private lazy val client = new LowLevelApisImpl[IO](RenkuCoreUri.Current(externalServiceBaseUri), ClientTools[IO]) + private lazy val client = new LowLevelApisImpl[IO](RenkuCoreUri.Latest(externalServiceBaseUri), ClientTools[IO]) } diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala index e9c733d450..0bb0339154 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala @@ -33,19 +33,19 @@ import scala.util.Try class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { - "Current.loadFromConfig" should { + "Latest.loadFromConfig" should { - "read the 'services.renku-core-current.url' from the Config" in { + "read the 'services.renku-core-latest.url' from the Config" in { val url = httpUrls().generateOne val config = configForCurrent(url) - RenkuCoreUri.Current.loadFromConfig[Try](config).success.value shouldBe - RenkuCoreUri.Current(Uri.unsafeFromString(url)) + RenkuCoreUri.Latest.loadFromConfig[Try](config).success.value shouldBe + RenkuCoreUri.Latest(Uri.unsafeFromString(url)) } "fail if the url does not exist" in { - RenkuCoreUri.Current.loadFromConfig[Try](ConfigFactory.empty()).failure.exception.getMessage should include( + RenkuCoreUri.Latest.loadFromConfig[Try](ConfigFactory.empty()).failure.exception.getMessage should include( "Key not found: 'services'" ) } @@ -56,8 +56,8 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { val config = configForCurrent(illegalUrl) - RenkuCoreUri.Current.loadFromConfig[Try](config).failure.exception.getMessage should include( - s"'$illegalUrl' is not a valid 'services.renku-core-current.url' uri" + RenkuCoreUri.Latest.loadFromConfig[Try](config).failure.exception.getMessage should include( + s"'$illegalUrl' is not a valid 'services.renku-core-latest.url' uri" ) } } @@ -120,7 +120,7 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { private def configForCurrent(url: String) = ConfigFactory.parseString( s"""services { - renku-core-current { + renku-core-latest { url = "$url" } }""" From 5a6e96cf7dd471eefaee4c93848b21c4ffcfcd65 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 15:51:39 +0200 Subject: [PATCH 34/49] refactor: helm config & conf reading redone --- .../io/renku/generators/Generators.scala | 16 ++++---- helm-chart/renku-graph/templates/_helpers.tpl | 19 ++++++++++ .../templates/knowledge-graph-deployment.yaml | 4 ++ .../src/main/resources/application.conf | 14 +------ .../io/renku/core/client/RenkuCoreUri.scala | 28 +++++++++++--- .../io/renku/core/client/Generators.scala | 4 +- .../renku/core/client/RenkuCoreUriSpec.scala | 38 ++++++++++--------- 7 files changed, 78 insertions(+), 45 deletions(-) diff --git a/generators/src/test/scala/io/renku/generators/Generators.scala b/generators/src/test/scala/io/renku/generators/Generators.scala index df96e3f6f4..b2eb634d02 100644 --- a/generators/src/test/scala/io/renku/generators/Generators.scala +++ b/generators/src/test/scala/io/renku/generators/Generators.scala @@ -191,15 +191,17 @@ object Generators { val httpPorts: Gen[Port] = choose(2000, 10000).map(Port.fromInt).map(_.getOrElse(sys.error("Invalid generated port"))) - def httpUrls(hostGenerator: Gen[String] = nonEmptyStrings(), - pathGenerator: Gen[String] = relativePaths(minSegments = 0, maxSegments = 2) + def httpUrls(protocolGenerator: Gen[String] = Gen.oneOf("http", "https"), + hostGenerator: Gen[String] = nonEmptyStrings(), + portGenerator: Gen[Option[Port]] = Gen.option(httpPorts), + pathGenerator: Gen[String] = relativePaths(minSegments = 0, maxSegments = 2) ): Gen[String] = for { - protocol <- Gen.oneOf("http", "https") - port <- httpPorts - host <- hostGenerator - path <- pathGenerator + protocol <- protocolGenerator + maybePort <- portGenerator + host <- hostGenerator + path <- pathGenerator pathValidated = if (path.isEmpty) "" else s"/$path" - } yield s"$protocol://$host:$port$pathValidated" + } yield s"$protocol://$host${maybePort.map(p => s":$p").getOrElse("")}$pathValidated" val localHttpUrls: Gen[String] = for { protocol <- Gen.oneOf("http", "https") diff --git a/helm-chart/renku-graph/templates/_helpers.tpl b/helm-chart/renku-graph/templates/_helpers.tpl index 5c12d3db00..3834d60409 100644 --- a/helm-chart/renku-graph/templates/_helpers.tpl +++ b/helm-chart/renku-graph/templates/_helpers.tpl @@ -90,6 +90,25 @@ If release name contains chart name it will be used as a full name. {{- printf "%s-commit-event-service" .Release.Name | trunc 63 | trimSuffix "-" -}} {{- end -}} {{- end -}} + +{{- define "renkuCoreLatest.fullname" -}} +{{- $coreBaseName := printf "%s-core" .Release.Name -}} +{{- printf "%s-%s" $coreBaseName (get $.Values.global.core.versions "latest").name -}} +{{- end -}} + +{{/* +Comma separated list of renku-core service names +*/}} +{{- define "renkuCore.serviceNames" -}} +{{- $serviceNames := list -}} +{{- $coreBaseName := printf "%s-core" .Release.Name -}} +{{- range $i, $k := (keys .Values.global.core.versions | sortAlpha) -}} +{{- $serviceName := printf "%s-%s" $coreBaseName (get $.Values.global.core.versions $k).name -}} +{{- $serviceNames = mustAppend $serviceNames $serviceName -}} +{{- end -}} +{{- join "," $serviceNames | quote -}} +{{- end -}} + {{/* Create chart name and version as used by the chart label. */}} diff --git a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml index 3045986926..b67323d878 100644 --- a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml +++ b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml @@ -55,6 +55,10 @@ spec: value: "http://{{ template "tokenRepository.fullname" . }}:{{ .Values.tokenRepository.service.port }}" - name: EVENT_LOG_BASE_URL value: "http://{{ template "eventLog.fullname" . }}:{{ .Values.eventLog.service.port }}" + - name: RENKU_CORE_LATEST_URL + value: "http://{{ template "renkuCoreLatest.fullname" . }}" + - name: RENKU_CORE_SERVICE_NAMES + value: "{{ template "renkuCore.serviceNames" . }}" - name: GITLAB_BASE_URL value: {{ .Values.gitlab.url }} - name: GITLAB_RATE_LIMIT diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index c513356f5e..ce71ccf26c 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -2,19 +2,9 @@ services { renku-core-latest { - url = "http://renku-ci-gr-1652-core-v10" + url = "http://localhost:80" url = ${?RENKU_CORE_LATEST_URL} } - renku-core-v10 { - schemaVersion = "10" - url = "http://renku-ci-gr-1652-core-v10" - url = ${?RENKU_CORE_V10_URL} - } - - renku-core-v9 { - schemaVersion = "9" - url = "http://renku-ci-gr-1652-core-v9" - url = ${?RENKU_CORE_V9_URL} - } + renku-core-service-names = ${?RENKU_CORE_SERVICE_NAMES} } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala index 8a7e5c5d40..7d286d74bc 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala @@ -31,15 +31,21 @@ sealed trait RenkuCoreUri { object RenkuCoreUri { - final case class Latest(uri: Uri) extends RenkuCoreUri + final case class Latest(uri: Uri) extends RenkuCoreUri final case class ForSchema(uri: Uri, schemaVersion: SchemaVersion) extends RenkuCoreUri final case class Versioned(baseUri: ForSchema, apiVersion: ApiVersion) extends RenkuCoreUri { val uri: Uri = baseUri.uri / apiVersion.value } object Latest { + + private val key = "services.renku-core-latest.url" + def loadFromConfig[F[_]: MonadThrow](config: Config = ConfigFactory.load): F[Latest] = - loadUrlFromConfig("services.renku-core-latest.url", config).map(Latest(_)) + loadUrlFromConfig(config).map(Latest(_)) + + private def loadUrlFromConfig[F[_]: MonadThrow](config: Config): F[Uri] = + find[F, String](key, config).flatMap(toUri[F](key, _)) } trait ForSchemaLoader { @@ -49,17 +55,27 @@ object RenkuCoreUri { } object ForSchema extends ForSchemaLoader { + private val key = "services.renku-core-service-names" + override def loadFromConfig[F[_]: MonadThrow](schemaVersion: SchemaVersion, config: Config = ConfigFactory.load ): F[ForSchema] = - loadUrlFromConfig(s"services.renku-core-v$schemaVersion.url", config).map(ForSchema(_, schemaVersion)) + loadServiceNamesFromConfig(config) + .flatMap(serviceNameForSchema(schemaVersion)) + .map(v => s"http://$v") + .flatMap(toUri[F](key, _)) + .map(ForSchema(_, schemaVersion)) + + private def loadServiceNamesFromConfig[F[_]: MonadThrow](config: Config) = + find[F, String](key, config).map(_.split(",").toList.map(_.trim)) + + private def serviceNameForSchema[F[_]: MonadThrow](schemaVersion: SchemaVersion): List[String] => F[String] = + _.find(_ contains show"v$schemaVersion") + .fold(new Exception(show"No renku-core for $schemaVersion in the config").raiseError[F, String])(_.pure[F]) } implicit def show[U <: RenkuCoreUri]: Show[U] = Show.show(_.uri.renderString) - private def loadUrlFromConfig[F[_]: MonadThrow](key: String, config: Config): F[Uri] = - find[F, String](key, config).flatMap(toUri[F](key, _)) - private def toUri[F[_]: MonadThrow](key: String, uri: String): F[Uri] = Uri .fromString(uri) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index 10492932f6..c35bc11467 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -48,9 +48,9 @@ object Generators { implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = for { - baseUri <- httpUrls() + baseUri <- httpUrls(protocolGenerator = "http", portGenerator = httpPorts.toGeneratorOfNones, pathGenerator = "") schema <- projectSchemaVersions - } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(baseUri), schema) + } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(show"$baseUri-v$schema"), schema) def coreUrisForSchema(baseUri: Uri): Gen[RenkuCoreUri.ForSchema] = projectSchemaVersions.map(RenkuCoreUri.ForSchema(baseUri, _)) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala index 0bb0339154..0d1ac7c1b8 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala @@ -23,13 +23,12 @@ import com.typesafe.config.ConfigFactory import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.httpUrls import io.renku.graph.model.GraphModelGenerators.projectSchemaVersions -import io.renku.graph.model.versions.SchemaVersion import org.http4s.Uri import org.scalatest.TryValues import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpec -import scala.util.Try +import scala.util.{Random, Try} class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { @@ -64,13 +63,17 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { "ForSchema.loadFromConfig" should { - "read the 'services.renku-core-vXXX.url' from the Config" in { + "read the 'services.renku-core-service-names' from the Config, " + + "find the match by checking if the name contains the schemaVersion and " + + "convert it to RenkuCoreUri.ForSchema" in { - val uri = coreUrisForSchema.generateOne - val config = configForSchema(uri.schemaVersion, uri.uri.renderString) + val uris = coreUrisForSchema.generateNonEmptyList().toList + val config = configForServiceNames(toSchemaNames(uris)) - RenkuCoreUri.ForSchema.loadFromConfig[Try](uri.schemaVersion, config).success.value shouldBe uri - } + val selectedUri = Random.shuffle(uris).head + + RenkuCoreUri.ForSchema.loadFromConfig[Try](selectedUri.schemaVersion, config).success.value shouldBe selectedUri + } "fail if the url does not exist" in { RenkuCoreUri.ForSchema @@ -80,18 +83,17 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { .getMessage should include("Key not found: 'services'") } - "fail if the url is invalid" in { - - val illegalUrl = "?ddkf !&&" - val schemaVersion = projectSchemaVersions.generateOne + "fail if there's no schema version for the schema" in { - val config = configForSchema(schemaVersion, illegalUrl) + val uris = coreUrisForSchema.generateNonEmptyList().toList + val config = configForServiceNames(toSchemaNames(uris)) + val schemaVersion = projectSchemaVersions.generateOne RenkuCoreUri.ForSchema .loadFromConfig[Try](schemaVersion, config) .failure .exception - .getMessage should include(s"'$illegalUrl' is not a valid 'services.renku-core-v$schemaVersion.url' uri") + .getMessage should include(s"No renku-core for $schemaVersion in the config") } } @@ -126,13 +128,13 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { }""" ) - private def configForSchema(schema: SchemaVersion, url: String) = + private def configForServiceNames(schemaNames: List[String]) = ConfigFactory.parseString( s"""services { - renku-core-v$schema { - schemaVersion = "$schema" - url = "$url" - } + renku-core-service-names = "${schemaNames.mkString(",")}" }""" ) + + private lazy val toSchemaNames: List[RenkuCoreUri.ForSchema] => List[String] = + _.map(_.uri.host.getOrElse(fail("no host in the uri")).value) } From 2a22279e356b5d434f5fb71a4c7b70c6dc2a66d0 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 16:01:40 +0200 Subject: [PATCH 35/49] refactor: core protocol to be defined in helm template --- helm-chart/renku-graph/templates/_helpers.tpl | 10 +++++----- .../templates/knowledge-graph-deployment.yaml | 4 ++-- renku-core-client/src/main/resources/application.conf | 2 +- .../main/scala/io/renku/core/client/RenkuCoreUri.scala | 3 +-- .../test/scala/io/renku/core/client/Generators.scala | 4 ++-- .../scala/io/renku/core/client/RenkuCoreUriSpec.scala | 6 +++--- 6 files changed, 14 insertions(+), 15 deletions(-) diff --git a/helm-chart/renku-graph/templates/_helpers.tpl b/helm-chart/renku-graph/templates/_helpers.tpl index 3834d60409..673d89b85a 100644 --- a/helm-chart/renku-graph/templates/_helpers.tpl +++ b/helm-chart/renku-graph/templates/_helpers.tpl @@ -99,14 +99,14 @@ If release name contains chart name it will be used as a full name. {{/* Comma separated list of renku-core service names */}} -{{- define "renkuCore.serviceNames" -}} -{{- $serviceNames := list -}} +{{- define "renkuCore.serviceUrls" -}} +{{- $serviceUrls := list -}} {{- $coreBaseName := printf "%s-core" .Release.Name -}} {{- range $i, $k := (keys .Values.global.core.versions | sortAlpha) -}} -{{- $serviceName := printf "%s-%s" $coreBaseName (get $.Values.global.core.versions $k).name -}} -{{- $serviceNames = mustAppend $serviceNames $serviceName -}} +{{- $serviceUrl := printf "http://%s-%s" $coreBaseName (get $.Values.global.core.versions $k).name -}} +{{- $serviceUrls = mustAppend $serviceUrls $serviceUrl -}} {{- end -}} -{{- join "," $serviceNames | quote -}} +{{- join "," $serviceUrls | quote -}} {{- end -}} {{/* diff --git a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml index b67323d878..3fa0e1e102 100644 --- a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml +++ b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml @@ -57,8 +57,8 @@ spec: value: "http://{{ template "eventLog.fullname" . }}:{{ .Values.eventLog.service.port }}" - name: RENKU_CORE_LATEST_URL value: "http://{{ template "renkuCoreLatest.fullname" . }}" - - name: RENKU_CORE_SERVICE_NAMES - value: "{{ template "renkuCore.serviceNames" . }}" + - name: RENKU_CORE_SERVICE_URLS + value: "{{ template "renkuCore.serviceUrls" . }}" - name: GITLAB_BASE_URL value: {{ .Values.gitlab.url }} - name: GITLAB_RATE_LIMIT diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index ce71ccf26c..0718ffc2cc 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -6,5 +6,5 @@ services { url = ${?RENKU_CORE_LATEST_URL} } - renku-core-service-names = ${?RENKU_CORE_SERVICE_NAMES} + renku-core-service-urls = ${?RENKU_CORE_SERVICE_URLS} } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala index 7d286d74bc..7f54177c3e 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala @@ -55,14 +55,13 @@ object RenkuCoreUri { } object ForSchema extends ForSchemaLoader { - private val key = "services.renku-core-service-names" + private val key = "services.renku-core-service-urls" override def loadFromConfig[F[_]: MonadThrow](schemaVersion: SchemaVersion, config: Config = ConfigFactory.load ): F[ForSchema] = loadServiceNamesFromConfig(config) .flatMap(serviceNameForSchema(schemaVersion)) - .map(v => s"http://$v") .flatMap(toUri[F](key, _)) .map(ForSchema(_, schemaVersion)) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala index c35bc11467..11dc1b1194 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/Generators.scala @@ -48,9 +48,9 @@ object Generators { implicit lazy val coreUrisForSchema: Gen[RenkuCoreUri.ForSchema] = for { - baseUri <- httpUrls(protocolGenerator = "http", portGenerator = httpPorts.toGeneratorOfNones, pathGenerator = "") schema <- projectSchemaVersions - } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(show"$baseUri-v$schema"), schema) + baseUri <- httpUrls(hostGenerator = nonEmptyStrings().map(v => show"$v-v$schema")) + } yield RenkuCoreUri.ForSchema(Uri.unsafeFromString(baseUri), schema) def coreUrisForSchema(baseUri: Uri): Gen[RenkuCoreUri.ForSchema] = projectSchemaVersions.map(RenkuCoreUri.ForSchema(baseUri, _)) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala index 0d1ac7c1b8..69d39c730c 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala @@ -63,7 +63,7 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { "ForSchema.loadFromConfig" should { - "read the 'services.renku-core-service-names' from the Config, " + + "read the 'services.renku-core-service-urls' from the Config, " + "find the match by checking if the name contains the schemaVersion and " + "convert it to RenkuCoreUri.ForSchema" in { @@ -131,10 +131,10 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { private def configForServiceNames(schemaNames: List[String]) = ConfigFactory.parseString( s"""services { - renku-core-service-names = "${schemaNames.mkString(",")}" + renku-core-service-urls = "${schemaNames.mkString(",")}" }""" ) private lazy val toSchemaNames: List[RenkuCoreUri.ForSchema] => List[String] = - _.map(_.uri.host.getOrElse(fail("no host in the uri")).value) + _.map(_.uri.renderString) } From bdf3c0f6abe8dd0c0a2ea407bce7c4bd017b7937 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 18:06:11 +0200 Subject: [PATCH 36/49] chore: fix in the knowledge-graph-deployment.yaml --- .../renku-graph/templates/knowledge-graph-deployment.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml index 3fa0e1e102..8cde7d192d 100644 --- a/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml +++ b/helm-chart/renku-graph/templates/knowledge-graph-deployment.yaml @@ -58,7 +58,7 @@ spec: - name: RENKU_CORE_LATEST_URL value: "http://{{ template "renkuCoreLatest.fullname" . }}" - name: RENKU_CORE_SERVICE_URLS - value: "{{ template "renkuCore.serviceUrls" . }}" + value: {{ template "renkuCore.serviceUrls" . }} - name: GITLAB_BASE_URL value: {{ .Values.gitlab.url }} - name: GITLAB_RATE_LIMIT From 5caf25e430c9f256b9c576ea9de4672601d350c7 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 18:43:46 +0200 Subject: [PATCH 37/49] fix: unit tests failures after httpUrls gen changes --- .../test/scala/io/renku/generators/Generators.scala | 7 ++++--- .../consumers/subscriptions/SubscriberUrlSpec.scala | 3 +-- .../test/scala/io/renku/graph/model/plansSpec.scala | 11 +++++++---- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/generators/src/test/scala/io/renku/generators/Generators.scala b/generators/src/test/scala/io/renku/generators/Generators.scala index b2eb634d02..8dc2ea60ac 100644 --- a/generators/src/test/scala/io/renku/generators/Generators.scala +++ b/generators/src/test/scala/io/renku/generators/Generators.scala @@ -193,15 +193,16 @@ object Generators { def httpUrls(protocolGenerator: Gen[String] = Gen.oneOf("http", "https"), hostGenerator: Gen[String] = nonEmptyStrings(), - portGenerator: Gen[Option[Port]] = Gen.option(httpPorts), + portGenerator: Gen[Option[Port]] = Gen.some(httpPorts), pathGenerator: Gen[String] = relativePaths(minSegments = 0, maxSegments = 2) ): Gen[String] = for { protocol <- protocolGenerator - maybePort <- portGenerator host <- hostGenerator + maybePort <- portGenerator path <- pathGenerator + portValidated = maybePort.map(p => s":$p").getOrElse("") pathValidated = if (path.isEmpty) "" else s"/$path" - } yield s"$protocol://$host${maybePort.map(p => s":$p").getOrElse("")}$pathValidated" + } yield s"$protocol://$host$portValidated$pathValidated" val localHttpUrls: Gen[String] = for { protocol <- Gen.oneOf("http", "https") diff --git a/graph-commons/src/test/scala/io/renku/events/consumers/subscriptions/SubscriberUrlSpec.scala b/graph-commons/src/test/scala/io/renku/events/consumers/subscriptions/SubscriberUrlSpec.scala index 701f218415..f538237081 100644 --- a/graph-commons/src/test/scala/io/renku/events/consumers/subscriptions/SubscriberUrlSpec.scala +++ b/graph-commons/src/test/scala/io/renku/events/consumers/subscriptions/SubscriberUrlSpec.scala @@ -23,7 +23,6 @@ import io.renku.generators.CommonGraphGenerators.microserviceBaseUrls import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.{httpUrls, nonBlankStrings, relativePaths} import io.renku.microservices.MicroserviceBaseUrl -import org.scalacheck.Gen import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpec import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks @@ -44,7 +43,7 @@ class SubscriberUrlSpec extends AnyWordSpec with should.Matchers with ScalaCheck "to MicroserviceBaseUrl conversion" should { "successfully convert MicroserviceBaseUrl if well defined" in { - forAll(httpUrls(pathGenerator = Gen.const("")), relativePaths(minSegments = 0, maxSegments = 2)) { (url, path) => + forAll(httpUrls(pathGenerator = ""), relativePaths(minSegments = 0, maxSegments = 2)) { (url, path) => val pathValidated = if (path.isEmpty) "" else s"/$path" SubscriberUrl(s"$url$pathValidated").toUnsafe[MicroserviceBaseUrl] shouldBe MicroserviceBaseUrl(url) } diff --git a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/plansSpec.scala b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/plansSpec.scala index c45168973d..01423d9736 100644 --- a/renku-model-tiny-types/src/test/scala/io/renku/graph/model/plansSpec.scala +++ b/renku-model-tiny-types/src/test/scala/io/renku/graph/model/plansSpec.scala @@ -22,16 +22,18 @@ import cats.syntax.all._ import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators._ import org.scalacheck.Gen +import org.scalatest.TryValues import org.scalatest.matchers.should import org.scalatest.wordspec.AnyWordSpec import org.scalatestplus.scalacheck.ScalaCheckPropertyChecks -import scala.util.{Failure, Try} +import scala.util.Try class ResourceIdSpec extends AnyWordSpec with ScalaCheckPropertyChecks with should.Matchers + with TryValues with RenkuTinyTypeGenerators { implicit val renkuUrl: RenkuUrl = RenkuTinyTypeGenerators.renkuUrls.generateOne @@ -41,18 +43,19 @@ class ResourceIdSpec "be successful for valid ResourceIds" in { forAll(planIdentifiers) { id => val resourceId = plans.ResourceId(id) - resourceId.as[Try, plans.Identifier] shouldBe id.pure[Try] + resourceId.as[Try, plans.Identifier].success.value shouldBe id } } "fail for an unknown resourceIds" in { val resourceId = plans.ResourceId { httpUrls( - relativePaths(partsGenerator = Gen.frequency(9 -> planIdentifiers.generateOne.show, 1 -> nonEmptyStrings())) + pathGenerator = + relativePaths(partsGenerator = Gen.frequency(9 -> planIdentifiers.generateOne.show, 1 -> nonEmptyStrings())) ).generateOne } - val Failure(exception) = resourceId.as[Try, plans.Identifier] + val exception = resourceId.as[Try, plans.Identifier].failure.exception exception shouldBe an[IllegalArgumentException] exception.getMessage shouldBe s"'${resourceId.value}' cannot be converted to a plans.Identifier" From 3b299bd7adb4d54d3357bab6b71faa009705af3c Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Mon, 21 Aug 2023 19:32:02 +0200 Subject: [PATCH 38/49] fix: versioned core api uri --- .../src/main/scala/io/renku/core/client/LowLevelApis.scala | 3 ++- .../src/main/scala/io/renku/core/client/RenkuCoreUri.scala | 2 +- .../test/scala/io/renku/core/client/LowLevelApisSpec.scala | 2 +- .../test/scala/io/renku/core/client/RenkuCoreUriSpec.scala | 4 ++-- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala index d92fb0d01c..8f8bf0ad9b 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/LowLevelApis.scala @@ -28,6 +28,7 @@ import io.renku.control.Throttler import io.renku.graph.model.projects import io.renku.graph.model.versions.SchemaVersion import io.renku.http.client.{AccessToken, RestClient, UserAccessToken} +import io.renku.http.tinytypes.TinyTypeURIEncoder._ import org.http4s.Header import org.http4s.circe._ import org.http4s.client.dsl.Http4sClientDsl @@ -106,7 +107,7 @@ private class LowLevelApisImpl[F[_]: Async: Logger](coreLatestUri: RenkuCoreUri. accessToken: UserAccessToken ): F[Result[Unit]] = send( - request(POST, uri.uri / "renku" / "project.edit", accessToken) + request(POST, uri.uri / "renku" / uri.apiVersion / "project.edit", accessToken) .withEntity(updates.asJson) .putHeaders(Header.Raw(ci"renku-user-email", updates.userInfo.email.value)) .putHeaders(Header.Raw(ci"renku-user-fullname", updates.userInfo.name.value)) diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala index 7f54177c3e..8192d9b843 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreUri.scala @@ -34,7 +34,7 @@ object RenkuCoreUri { final case class Latest(uri: Uri) extends RenkuCoreUri final case class ForSchema(uri: Uri, schemaVersion: SchemaVersion) extends RenkuCoreUri final case class Versioned(baseUri: ForSchema, apiVersion: ApiVersion) extends RenkuCoreUri { - val uri: Uri = baseUri.uri / apiVersion.value + val uri: Uri = baseUri.uri } object Latest { diff --git a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala index 03f558d15c..28f9742ac3 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/LowLevelApisSpec.scala @@ -117,7 +117,7 @@ class LowLevelApisSpec val versionedUri = coreUrisVersioned(server.baseUri).generateOne server.stubFor { - post(s"/${versionedUri.apiVersion}/renku/project.edit") + post(s"/renku/${versionedUri.apiVersion}/project.edit") .withRequestBody(equalToJson(updates.asJson.spaces2)) .withAccessToken(accessToken.some) .withHeader("renku-user-email", equalTo(updates.userInfo.email.value)) diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala index 69d39c730c..95d60971fd 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreUriSpec.scala @@ -110,12 +110,12 @@ class RenkuCoreUriSpec extends AnyWordSpec with should.Matchers with TryValues { "Versioned.uri" should { - "return a value that is composed from the baseUri and the apiVersion" in { + "return a value that is composed from the baseUri" in { val baseUri = coreUrisForSchema.generateOne val apiVersion = apiVersions.generateOne - RenkuCoreUri.Versioned(baseUri, apiVersion).uri shouldBe baseUri.uri / apiVersion.value + RenkuCoreUri.Versioned(baseUri, apiVersion).uri shouldBe baseUri.uri } } From c0f141f507462beece7d8700e5463c7fc804abd9 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Tue, 22 Aug 2023 15:11:24 +0200 Subject: [PATCH 39/49] feat: TS to be updated with avatar_url from GL --- .../projects/update/Failure.scala | 3 + .../projects/update/GLProjectUpdater.scala | 30 ++++-- .../projects/update/GLUpdatedProject.scala | 31 ++++++ .../projects/update/ProjectUpdater.scala | 37 ++++---- .../projects/update/ProjectUpdates.scala | 3 + .../projects/update/TGUpdatesFinder.scala | 65 +++++++++++++ .../update/GLProjectUpdaterSpec.scala | 40 +++++--- .../projects/update/Generators.scala | 6 ++ .../projects/update/ProjectUpdaterSpec.scala | 79 +++++++++++----- .../projects/update/ProjectUpdatesSpec.scala | 16 ++++ .../projects/update/TGUpdatesFinderSpec.scala | 94 +++++++++++++++++++ 11 files changed, 343 insertions(+), 61 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLUpdatedProject.scala create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinder.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinderSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala index 9f3382cc39..a149c26833 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala @@ -49,6 +49,9 @@ private object Failure { def onGLUpdate(slug: projects.Slug, cause: Throwable): Failure = Failure(InternalServerError, Message.Error.unsafeApply(show"Updating project $slug in GitLab failed"), cause) + def onTGUpdatesFinding(slug: projects.Slug, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Finding TS updates for $slug failed"), cause) + def onTSUpdate(slug: projects.Slug, cause: Throwable): Failure = Failure(InternalServerError, Message.Error.unsafeApply(show"Updating project $slug in TS failed"), cause) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 28a742b7ad..46cbf3e7bc 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -28,12 +28,16 @@ import io.renku.graph.model.projects import io.renku.http.client.{AccessToken, GitLabClient} import io.renku.http.tinytypes.TinyTypeURIEncoder._ import org.http4s.Status._ +import org.http4s.circe.CirceEntityDecoder._ import org.http4s.circe.jsonOf import org.http4s.implicits._ import org.http4s.{Request, Response, Status, UrlForm} private trait GLProjectUpdater[F[_]] { - def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Message, Unit]] + def updateProject(slug: projects.Slug, + updates: ProjectUpdates, + at: AccessToken + ): F[Either[Message, Option[GLUpdatedProject]]] } private object GLProjectUpdater { @@ -42,11 +46,16 @@ private object GLProjectUpdater { private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectUpdater[F] { - override def updateProject(slug: projects.Slug, updates: ProjectUpdates, at: AccessToken): F[Either[Message, Unit]] = - if ((updates.newImage orElse updates.newVisibility).isDefined) - GitLabClient[F].put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) + override def updateProject(slug: projects.Slug, + updates: ProjectUpdates, + at: AccessToken + ): F[Either[Message, Option[GLUpdatedProject]]] = + if (updates.glUpdateNeeded) + GitLabClient[F] + .put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) + .map(_.map(_.some)) else - ().asRight[Message].pure[F] + Option.empty[GLUpdatedProject].asRight[Message].pure[F] private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => UrlForm.empty @@ -54,11 +63,12 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU .updateFormField("visibility", newVisibility.map(_.value)) } - private lazy val mapResponse: PartialFunction[(Status, Request[F], Response[F]), F[Either[Message, Unit]]] = { - case (Ok, _, _) => - ().asRight[Message].pure[F] - case (BadRequest, _, response) => - response.as[Json](MonadThrow[F], jsonOf(Async[F], errorDecoder)).map(Message.Error.fromJsonUnsafe).map(_.asLeft) + private lazy val mapResponse + : PartialFunction[(Status, Request[F], Response[F]), F[Either[Message, GLUpdatedProject]]] = { + case (Ok, _, resp) => + resp.as[GLUpdatedProject].map(_.asRight[Message]) + case (BadRequest, _, resp) => + resp.as[Json](MonadThrow[F], jsonOf(Async[F], errorDecoder)).map(Message.Error.fromJsonUnsafe).map(_.asLeft) } private lazy val errorDecoder: Decoder[Json] = { cur => diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLUpdatedProject.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLUpdatedProject.scala new file mode 100644 index 0000000000..6df5b95183 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLUpdatedProject.scala @@ -0,0 +1,31 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import io.circe.Decoder +import io.renku.graph.model.images.ImageUri +import io.renku.graph.model.projects + +private final case class GLUpdatedProject(image: Option[ImageUri], visibility: projects.Visibility) + +private object GLUpdatedProject { + + implicit val decoder: Decoder[GLUpdatedProject] = + Decoder.forProduct2("avatar_url", "visibility")(GLUpdatedProject.apply) +} diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index 6f1950cea0..a33a9d10df 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -42,7 +42,8 @@ private object ProjectUpdater { UserInfoFinder[F], GLProjectUpdater[F], _, - _ + _, + TGUpdatesFinder[F] ) ) } @@ -52,39 +53,43 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr userInfoFinder: UserInfoFinder[F], glProjectUpdater: GLProjectUpdater[F], tgClient: TriplesGeneratorClient[F], - renkuCoreClient: RenkuCoreClient[F] + renkuCoreClient: RenkuCoreClient[F], + tgUpdatesFinder: TGUpdatesFinder[F] ) extends ProjectUpdater[F] { override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] = if (updates.onlyGLUpdateNeeded) - updateGL(slug, updates, authUser) >> updateTG(slug, updates) + updateGL(slug, updates, authUser) + .flatMap(findTGUpdates(slug, updates, _)) + .flatMap(updateTG(slug, _)) else canPushToDefaultBranch(slug, authUser) >> { for { coreUpdates <- findCoreProjectUpdates(slug, updates, authUser) coreUri <- findCoreUri(coreUpdates, authUser) _ <- updateCore(slug, coreUri, coreUpdates, authUser) - _ <- updateGL(slug, updates, authUser) - _ <- updateTG(slug, updates) + updated <- updateGL(slug, updates, authUser) + tgUpdates <- findTGUpdates(slug, updates, updated) + _ <- updateTG(slug, tgUpdates) } yield () } - private def updateGL(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] = + private def updateGL(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Option[GLUpdatedProject]] = glProjectUpdater .updateProject(slug, updates, authUser.accessToken) .adaptError(Failure.onGLUpdate(slug, _)) - .flatMap(_.fold(errMsg => Failure.badRequestOnGLUpdate(errMsg).raiseError[F, Unit], _.pure[F])) + .flatMap(_.fold(err => Failure.badRequestOnGLUpdate(err).raiseError[F, Option[GLUpdatedProject]], _.pure[F])) + + private def findTGUpdates(slug: projects.Slug, + updates: ProjectUpdates, + maybeGLUpdatedProject: Option[GLUpdatedProject] + ) = tgUpdatesFinder + .findTGProjectUpdates(updates, maybeGLUpdatedProject) + .adaptError(Failure.onTGUpdatesFinding(slug, _)) - private def updateTG(slug: projects.Slug, updates: ProjectUpdates): F[Unit] = + private def updateTG(slug: projects.Slug, updates: TGProjectUpdates): F[Unit] = tgClient - .updateProject( - slug, - TGProjectUpdates(newDescription = updates.newDescription, - newImages = updates.newImage.map(_.toList), - newKeywords = updates.newKeywords, - newVisibility = updates.newVisibility - ) - ) + .updateProject(slug, updates) .map(_.toEither) .handleError(_.asLeft) .flatMap(_.fold(Failure.onTSUpdate(slug, _).raiseError[F, Unit], _.pure[F])) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index 2578ce2aec..9c6ca2b404 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -36,6 +36,9 @@ private final case class ProjectUpdates(newDescription: Option[Option[projects.D (newImage orElse newVisibility).isDefined && (newDescription orElse newKeywords).isEmpty + lazy val glUpdateNeeded: Boolean = + (newImage orElse newVisibility).isDefined + lazy val coreUpdateNeeded: Boolean = (newDescription orElse newKeywords).isDefined } diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinder.scala new file mode 100644 index 0000000000..73a867e154 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinder.scala @@ -0,0 +1,65 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.MonadThrow +import cats.syntax.all._ +import io.renku.graph.model.images.ImageUri +import io.renku.triplesgenerator.api.{ProjectUpdates => TGProjectUpdates} + +private trait TGUpdatesFinder[F[_]] { + def findTGProjectUpdates(updates: ProjectUpdates, + maybeGLUpdatedProject: Option[GLUpdatedProject] + ): F[TGProjectUpdates] +} + +private object TGUpdatesFinder { + def apply[F[_]: MonadThrow]: TGUpdatesFinder[F] = new TGUpdatesFinderImpl[F] + +} +private class TGUpdatesFinderImpl[F[_]: MonadThrow] extends TGUpdatesFinder[F] { + + override def findTGProjectUpdates(updates: ProjectUpdates, + maybeGLUpdatedProject: Option[GLUpdatedProject] + ): F[TGProjectUpdates] = + findNewImages(updates, maybeGLUpdatedProject).map(maybeNewImages => + TGProjectUpdates( + newDescription = updates.newDescription, + newImages = maybeNewImages, + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility + ) + ) + + private def findNewImages(updates: ProjectUpdates, + maybeGLUpdatedProject: Option[GLUpdatedProject] + ): F[Option[List[ImageUri]]] = + updates.newImage match { + case None => + Option.empty[List[ImageUri]].pure[F] + case Some(_) if maybeGLUpdatedProject.isEmpty => + new Exception("No info about updated values in GL").raiseError + case Some(None) if maybeGLUpdatedProject.flatMap(_.image).nonEmpty => + new Exception("Image not deleted in GL").raiseError + case Some(Some(_)) if maybeGLUpdatedProject.flatMap(_.image).isEmpty => + new Exception("Image not updated in GL").raiseError + case _ => + maybeGLUpdatedProject.map(_.image.toList).pure[F] + } +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index f1064f129b..c6a35aed4c 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -25,8 +25,9 @@ import cats.syntax.all._ import eu.timepit.refined.api.Refined import eu.timepit.refined.auto._ import eu.timepit.refined.collection.NonEmpty -import io.circe.Json import io.circe.literal._ +import io.circe.syntax._ +import io.circe.{Encoder, Json} import io.renku.data.Message import io.renku.generators.CommonGraphGenerators.accessTokens import io.renku.generators.Generators.Implicits._ @@ -55,15 +56,16 @@ class GLProjectUpdaterSpec with EitherValues with GitLabClientTools[IO] { - it should s"call GL's PUT gl/projects/:slug and return unit on success" in { + it should s"call GL's PUT gl/projects/:slug and return updated values on success" in { - val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne - val accessToken = accessTokens.generateOne + val slug = projectSlugs.generateOne + val newValues = projectUpdatesGen.suchThat(_.glUpdateNeeded).generateOne + val accessToken = accessTokens.generateOne + val updatedProject = glUpdatedProjectsGen.generateOne - givenEditProjectAPICall(slug, newValues, accessToken, returning = ().asRight.pure[IO]) + givenEditProjectAPICall(slug, newValues, accessToken, returning = updatedProject.asRight.pure[IO]) - finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe ()) + finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe updatedProject.some) } it should s"do nothing if neither new image nor visibility is set in the update" in { @@ -72,7 +74,7 @@ class GLProjectUpdaterSpec val newValues = projectUpdatesGen.generateOne.copy(newImage = None, newVisibility = None) val accessToken = accessTokens.generateOne - finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe ()) + finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe None) } it should s"call GL's PUT gl/projects/:slug and return GL message if returned" in { @@ -89,8 +91,10 @@ class GLProjectUpdaterSpec .asserting(_.left.value shouldBe error) } - it should "succeed if PUT gl/projects/:slug returns 200 OK" in { - mapResponse(Ok, Request[IO](), Response[IO]()).asserting(_.value shouldBe ()) + it should "succeed and return updated values if PUT gl/projects/:slug returns 200 OK" in { + val updatedProject = glUpdatedProjectsGen.generateOne + mapResponse(Ok, Request[IO](), Response[IO]().withEntity(updatedProject.asJson)) + .asserting(_.value shouldBe updatedProject) } it should "return left if PUT gl/projects/:slug returns 400 BAD_REQUEST with an error" in { @@ -115,11 +119,11 @@ class GLProjectUpdaterSpec private def givenEditProjectAPICall(slug: projects.Slug, newValues: ProjectUpdates, accessToken: AccessToken, - returning: IO[Either[Message, Unit]] + returning: IO[Either[Message, GLUpdatedProject]] ) = { val endpointName: String Refined NonEmpty = "edit-project" (glClient - .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Message, Unit]])( + .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Message, GLUpdatedProject]])( _: Option[AccessToken] )) .expects(uri"projects" / slug, endpointName, toUrlForm(newValues), *, accessToken.some) @@ -135,7 +139,7 @@ class GLProjectUpdaterSpec ) } - private lazy val mapResponse: ResponseMappingF[IO, Either[Message, Unit]] = + private lazy val mapResponse: ResponseMappingF[IO, Either[Message, GLUpdatedProject]] = captureMapping(glClient)( finder .updateProject(projectSlugs.generateOne, @@ -143,7 +147,15 @@ class GLProjectUpdaterSpec accessTokens.generateOne ) .unsafeRunSync(), - ().asRight[Message], + glUpdatedProjectsGen.generateOne.asRight[Message], method = PUT ) + + private implicit lazy val responseEncoder: Encoder[GLUpdatedProject] = Encoder.instance { + case GLUpdatedProject(image, visibility) => + json"""{ + "avatar_url": $image, + "visibility": $visibility + }""" + } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala index e512018539..b3533bea5b 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala @@ -31,4 +31,10 @@ private object Generators { maybeNewKeywords <- projectKeywords.toGeneratorOfSet().toGeneratorOfOptions maybeNewVisibility <- projectVisibilities.toGeneratorOfOptions } yield ProjectUpdates(maybeNewDesc, maybeNewImage, maybeNewKeywords, maybeNewVisibility) + + val glUpdatedProjectsGen: Gen[GLUpdatedProject] = + for { + maybeNewImage <- imageUris.toGeneratorOfOptions + maybeNewVisibility <- projectVisibilities + } yield GLUpdatedProject(maybeNewImage, maybeNewVisibility) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 348ef6db71..191d3a1fc9 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -33,6 +33,7 @@ import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger import io.renku.interpreters.TestLogger.Level.Error import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesgenerator.api.Generators.{projectUpdatesGen => tgUpdatesGen} import io.renku.triplesgenerator.api.{TriplesGeneratorClient, ProjectUpdates => TGProjectUpdates} import org.scalamock.scalatest.AsyncMockFactory import org.scalatest.flatspec.AsyncFlatSpec @@ -50,8 +51,12 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .suchThat(_.onlyGLUpdateNeeded) .generateOne - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) - givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + val glUpdated = glUpdatedProjectsGen.generateSome + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = glUpdated.asRight.pure[IO]) + + val tgUpdates = tgUpdatesGen.generateOne + givenTGUpdatesCalculation(updates, glUpdated, returning = tgUpdates.pure[IO]) + givenSendingUpdateToTG(slug, tgUpdates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) updater.updateProject(slug, updates, authUser).assertNoException } @@ -78,8 +83,12 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul authUser.accessToken, returning = resultSuccesses(()).generateOne ) - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) - givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + + val glUpdated = glUpdatedProjectsGen.generateSome + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = glUpdated.asRight.pure[IO]) + val tgUpdates = tgUpdatesGen.generateOne + givenTGUpdatesCalculation(updates, glUpdated, returning = tgUpdates.pure[IO]) + givenSendingUpdateToTG(slug, tgUpdates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) updater.updateProject(slug, updates, authUser).assertNoException } @@ -223,8 +232,11 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul authUser.accessToken, returning = failedResult ) - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) - givenSendingUpdateToTG(slug, updates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) + val glUpdated = glUpdatedProjectsGen.generateSome + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = glUpdated.asRight.pure[IO]) + val tgUpdates = tgUpdatesGen.generateOne + givenTGUpdatesCalculation(updates, glUpdated, returning = tgUpdates.pure[IO]) + givenSendingUpdateToTG(slug, tgUpdates, returning = TriplesGeneratorClient.Result.success(()).pure[IO]) updater.updateProject(slug, updates, authUser).assertNoException >> { logger.waitFor(Error(show"Updating project $slug failed", failedResult)) @@ -261,7 +273,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul givenUpdatingProjectInGL(slug, updates, authUser.accessToken, - returning = exception.raiseError[IO, Either[Message, Unit]] + returning = exception.raiseError[IO, Either[Message, Option[GLUpdatedProject]]] ) updater @@ -269,7 +281,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .assertThrowsError[Exception](_ shouldBe Failure.onGLUpdate(slug, exception)) } - it should "fail if updating TG failed" in { + it should "fail if finding TG updates fails" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne @@ -278,9 +290,32 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul .suchThat(_.onlyGLUpdateNeeded) .generateOne - givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = ().asRight.pure[IO]) + val glUpdated = glUpdatedProjectsGen.generateSome + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = glUpdated.asRight.pure[IO]) + val exception = TriplesGeneratorClient.Result.Failure(exceptions.generateOne.getMessage) - givenSendingUpdateToTG(slug, updates, returning = exception.pure[IO]) + givenTGUpdatesCalculation(updates, glUpdated, returning = exception.raiseError[IO, Nothing]) + + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onTGUpdatesFinding(slug, exception)) + } + + it should "fail if updating TG fails" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen + .map(_.copy(newDescription = None, newKeywords = None)) + .suchThat(_.onlyGLUpdateNeeded) + .generateOne + + val glUpdated = glUpdatedProjectsGen.generateSome + givenUpdatingProjectInGL(slug, updates, authUser.accessToken, returning = glUpdated.asRight.pure[IO]) + val tgUpdates = tgUpdatesGen.generateOne + givenTGUpdatesCalculation(updates, glUpdated, returning = tgUpdates.pure[IO]) + val exception = TriplesGeneratorClient.Result.Failure(exceptions.generateOne.getMessage) + givenSendingUpdateToTG(slug, tgUpdates, returning = exception.pure[IO]) updater .updateProject(slug, updates, authUser) @@ -294,12 +329,14 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul private val glProjectUpdater = mock[GLProjectUpdater[IO]] private val tgClient = mock[TriplesGeneratorClient[IO]] private val renkuCoreClient = mock[RenkuCoreClient[IO]] + private val tgUpdatesFinder = mock[TGUpdatesFinder[IO]] private lazy val updater = new ProjectUpdaterImpl[IO](branchProtectionCheck, projectGitUrlFinder, userInfoFinder, glProjectUpdater, tgClient, - renkuCoreClient + renkuCoreClient, + tgUpdatesFinder ) private def givenBranchProtectionChecking(slug: projects.Slug, at: UserAccessToken, returning: IO[Boolean]) = @@ -322,23 +359,16 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul private def givenUpdatingProjectInGL(slug: projects.Slug, updates: ProjectUpdates, at: UserAccessToken, - returning: IO[Either[Message, Unit]] + returning: IO[Either[Message, Option[GLUpdatedProject]]] ) = (glProjectUpdater.updateProject _) .expects(slug, updates, at) .returning(returning) private def givenSendingUpdateToTG(slug: projects.Slug, - updates: ProjectUpdates, + updates: TGProjectUpdates, returning: IO[TriplesGeneratorClient.Result[Unit]] ) = (tgClient.updateProject _) - .expects( - slug, - TGProjectUpdates(newDescription = updates.newDescription, - newImages = updates.newImage.map(_.toList), - newKeywords = updates.newKeywords, - newVisibility = updates.newVisibility - ) - ) + .expects(slug, updates) .returning(returning) private def givenFindingCoreUri(gitUrl: projects.GitHttpUrl, @@ -356,4 +386,11 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul ) = (renkuCoreClient.updateProject _) .expects(coreUri, updates, at) .returning(returning.pure[IO]) + + private def givenTGUpdatesCalculation(updates: ProjectUpdates, + maybeGLUpdatedProject: Option[GLUpdatedProject], + returning: IO[TGProjectUpdates] + ) = (tgUpdatesFinder.findTGProjectUpdates _) + .expects(updates, maybeGLUpdatedProject) + .returning(returning) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala index 76bde0657e..72c215d469 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala @@ -48,6 +48,22 @@ class ProjectUpdatesSpec extends AnyWordSpec with should.Matchers with ScalaChec } } + "glUpdateNeeded" should { + + "return true if at least image and/or visibility is updated" in { + forAll( + projectUpdatesGen + .suchThat(u => (u.newImage orElse u.newVisibility).isDefined) + )(_.glUpdateNeeded shouldBe true) + } + + "return false otherwise" in { + forAll(projectUpdatesGen.map(_.copy(newImage = None, newVisibility = None)))( + _.glUpdateNeeded shouldBe false + ) + } + } + "coreUpdateNeeded" should { "return true if at least description and/or keywords is updated" in { diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinderSpec.scala new file mode 100644 index 0000000000..862f35f37e --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/TGUpdatesFinderSpec.scala @@ -0,0 +1,94 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import Generators._ +import cats.syntax.all._ +import io.renku.generators.Generators.Implicits._ +import io.renku.triplesgenerator.api.{ProjectUpdates => TGProjectUpdates} +import org.scalatest.TryValues +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should + +import scala.util.Try + +class TGUpdatesFinderSpec extends AnyFlatSpec with should.Matchers with TryValues { + + private val updatesFinder = TGUpdatesFinder[Try] + import updatesFinder.findTGProjectUpdates + + it should "map values from the Project updates if no GL updated values given" in { + + val updates = projectUpdatesGen.suchThat(_.newImage.isEmpty).generateOne + + findTGProjectUpdates(updates, maybeGLUpdatedProject = None).success.value shouldBe TGProjectUpdates( + newDescription = updates.newDescription, + newImages = None, + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility + ) + } + + it should "map values from the Project updates taking the image uri from GL update" in { + + val updates = projectUpdatesGen.suchThat(_.newImage.flatten.nonEmpty).generateOne + val glUpdated = glUpdatedProjectsGen.suchThat(_.image.nonEmpty).generateOne + + findTGProjectUpdates(updates, glUpdated.some).success.value shouldBe TGProjectUpdates( + newDescription = updates.newDescription, + newImages = glUpdated.image.toList.some, + newKeywords = updates.newKeywords, + newVisibility = updates.newVisibility + ) + } + + it should "fail if there's a new image update but no GL updated values" in { + + val updates = projectUpdatesGen.suchThat(_.newImage.flatten.nonEmpty).generateOne + + findTGProjectUpdates(updates, maybeGLUpdatedProject = None).failure.exception.getMessage shouldBe + "No info about updated values in GL" + } + + it should "fail if there's a new image update but GL updated values without the uri" in { + + val updates = projectUpdatesGen.suchThat(_.newImage.flatten.nonEmpty).generateOne + val glUpdated = glUpdatedProjectsGen.suchThat(_.image.isEmpty).generateSome + + findTGProjectUpdates(updates, glUpdated).failure.exception.getMessage shouldBe + "Image not updated in GL" + } + + it should "fail if there's an image deletion but no GL updated values" in { + + val updates = projectUpdatesGen.generateOne.copy(newImage = Some(None)) + + findTGProjectUpdates(updates, maybeGLUpdatedProject = None).failure.exception.getMessage shouldBe + "No info about updated values in GL" + } + + it should "fail if there's an image deletion but GL updated values contain some uri" in { + + val updates = projectUpdatesGen.generateOne.copy(newImage = Some(None)) + val glUpdated = glUpdatedProjectsGen.suchThat(_.image.nonEmpty).generateSome + + findTGProjectUpdates(updates, glUpdated).failure.exception.getMessage shouldBe + "Image not deleted in GL" + } +} From 7d1e74c56a438034f403e2a64afcd45ba927d14c Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 23 Aug 2023 16:03:06 +0200 Subject: [PATCH 40/49] feat: GL project update with multipart request --- .../io/renku/http/client/GitLabClient.scala | 21 +++--- .../io/renku/http/client/RestClient.scala | 1 + .../renku/http/client/GitLabClientSpec.scala | 54 +++++++-------- .../renku/testtools/GitLabClientTools.scala | 5 +- knowledge-graph/README.md | 11 ++- .../io/renku/knowledgegraph/docs/model.scala | 5 +- .../projects/update/Endpoint.scala | 25 +++++-- .../projects/update/EndpointDocs.scala | 14 +++- .../projects/update/GLProjectUpdater.scala | 46 ++++++++++--- .../projects/update/ProjectUpdates.scala | 18 ++--- .../projects/update/EndpointSpec.scala | 8 +-- .../update/GLProjectUpdaterSpec.scala | 68 ++++++++++++------- .../projects/update/Generators.scala | 13 +++- .../projects/update/ProjectUpdatesSpec.scala | 20 +----- 14 files changed, 192 insertions(+), 117 deletions(-) diff --git a/graph-commons/src/main/scala/io/renku/http/client/GitLabClient.scala b/graph-commons/src/main/scala/io/renku/http/client/GitLabClient.scala index 65d38b64d6..bb7df4b56b 100644 --- a/graph-commons/src/main/scala/io/renku/http/client/GitLabClient.scala +++ b/graph-commons/src/main/scala/io/renku/http/client/GitLabClient.scala @@ -35,7 +35,8 @@ import io.renku.http.rest.paging.model.{Page, Total} import io.renku.metrics.{GitLabApiCallRecorder, MetricsRegistry} import org.http4s.Method.{DELETE, GET, HEAD, POST, PUT} import org.http4s.circe.{jsonEncoder, jsonEncoderOf} -import org.http4s.{EntityEncoder, Method, Response, Uri, UrlForm} +import org.http4s.multipart.Multipart +import org.http4s.{EntityEncoder, Method, Response, Uri} import org.typelevel.ci._ import org.typelevel.log4cats.Logger @@ -55,7 +56,7 @@ trait GitLabClient[F[_]] { mapResponse: ResponseMappingF[F, ResultType] )(implicit maybeAccessToken: Option[AccessToken]): F[ResultType] - def put[ResultType](path: Uri, endpointName: String Refined NonEmpty, payload: UrlForm)( + def put[ResultType](path: Uri, endpointName: String Refined NonEmpty, payload: Multipart[F])( mapResponse: ResponseMappingF[F, ResultType] )(implicit maybeAccessToken: Option[AccessToken]): F[ResultType] @@ -104,13 +105,17 @@ final class GitLabClientImpl[F[_]: Async: Logger]( result <- super.send(request)(mapResponse) } yield result - override def put[ResultType](path: Uri, endpointName: String Refined NonEmpty, payload: UrlForm)( + override def put[ResultType](path: Uri, endpointName: String Refined NonEmpty, payload: Multipart[F])( mapResponse: ResponseMappingF[F, ResultType] - )(implicit maybeAccessToken: Option[AccessToken]): F[ResultType] = for { - uri <- validateUri(show"$gitLabApiUrl/$path") - request <- secureNamedRequest(PUT, uri, endpointName, payload) - result <- super.send(request)(mapResponse) - } yield result + )(implicit maybeAccessToken: Option[AccessToken]): F[ResultType] = + validateUri(show"$gitLabApiUrl/$path") + .flatMap( + secureNamedRequest(PUT, _, endpointName) + .map(req => req.copy(request = req.request.withEntity(payload).putHeaders(payload.headers))) + ) + .flatMap( + send(_: NamedRequest[F])(mapResponse) + ) override def delete[ResultType](path: Uri, endpointName: Refined[String, NonEmpty])( mapResponse: ResponseMappingF[F, ResultType] diff --git a/graph-commons/src/main/scala/io/renku/http/client/RestClient.scala b/graph-commons/src/main/scala/io/renku/http/client/RestClient.scala index c1a918e46f..480ff5e167 100644 --- a/graph-commons/src/main/scala/io/renku/http/client/RestClient.scala +++ b/graph-commons/src/main/scala/io/renku/http/client/RestClient.scala @@ -251,6 +251,7 @@ abstract class RestClient[F[_]: Async: Logger, ThrottlingTarget]( new MultipartBuilder(request, parts).build() class MultipartBuilder private[RequestOps] (request: Request[F], parts: Vector[Part[F]] = Vector.empty[Part[F]]) { + def addPart[PartType](name: String, value: PartType)(implicit encoder: PartEncoder[PartType] ): MultipartBuilder = diff --git a/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala b/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala index fbc91a35f2..ff753d93b8 100644 --- a/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala +++ b/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala @@ -36,10 +36,12 @@ import io.renku.logging.TestExecutionTimeRecorder import io.renku.metrics.GitLabApiCallRecorder import io.renku.stubbing.ExternalServiceStubbing import io.renku.testtools.IOSpec -import org.http4s.Method.{GET, _} +import org.http4s.MediaType.multipart +import org.http4s.Method._ import org.http4s.Status.{Accepted, NotFound, Ok, Unauthorized} import org.http4s.circe.CirceEntityCodec.circeEntityDecoder -import org.http4s.{Header, Method, Request, Response, Status, Uri, UrlForm} +import org.http4s.multipart.{Multiparts, Part} +import org.http4s.{Header, Method, Request, Response, Status, Uri} import org.scalacheck.Gen import org.scalamock.scalatest.MockFactory import org.scalatest.TryValues @@ -226,46 +228,40 @@ class GitLabClientSpec "put" should { - val mapPutResponse: PartialFunction[(Status, Request[IO], Response[IO]), IO[Unit]] = { - case (Accepted, _, _) => ().pure[IO] - case (Unauthorized, _, _) => UnauthorizedException.raiseError[IO, Unit] - } - forAll(tokenScenarios) { (tokenType, accessToken: AccessToken) => - s"send form data with the $tokenType to the endpoint" in new TestCase { + s"send the given multipart request with the $tokenType to the endpoint" in new TestCase { + + implicit val mat: Option[AccessToken] = accessToken.some + val partName = nonEmptyStrings().generateOne + val partValue = nonEmptyStrings().generateOne - val propName = nonEmptyStrings().generateOne - val propValue = nonEmptyStrings().generateOne + val multipartPayload = Multiparts + .forSync[IO] + .flatMap(_.multipart(Vector(Part.formData[IO](partName, partValue)))) + .unsafeRunSync() stubFor { put(s"/api/v4/$path") .withAccessToken(accessToken.some) - .withRequestBody(equalTo(s"$propName=$propValue")) + .withMultipartRequestBody( + aMultipart(partName).withBody(equalTo(partValue)) + ) + .withHeader( + "Content-Type", + containing(s"${multipart.`form-data`.mainType}/${multipart.`form-data`.subType}") + ) + .withHeader( + "Content-Type", + containing(s"""boundary="${multipartPayload.boundary.value}"""") + ) .willReturn(aResponse().withStatus(Accepted.code)) } client - .put(path, endpointName, UrlForm(propName -> propValue))(mapPutResponse)(accessToken.some) + .put(path, endpointName, multipartPayload) { case (Accepted, _, _) => ().pure[IO] } .unsafeRunSync() shouldBe () } } - - "return an UnauthorizedException if remote client responds with UNAUTHORIZED" in new TestCase { - - val propName = nonEmptyStrings().generateOne - val propValue = nonEmptyStrings().generateOne - - stubFor { - put(s"/api/v4/$path") - .withAccessToken(maybeAccessToken) - .withRequestBody(equalTo(s"$propName=$propValue")) - .willReturn(unauthorized()) - } - - intercept[Exception] { - client.put(path, endpointName, UrlForm(propName -> propValue))(mapPutResponse).unsafeRunSync() - } shouldBe UnauthorizedException - } } "delete" should { diff --git a/graph-commons/src/test/scala/io/renku/testtools/GitLabClientTools.scala b/graph-commons/src/test/scala/io/renku/testtools/GitLabClientTools.scala index 8a24bea863..000d3ce015 100644 --- a/graph-commons/src/test/scala/io/renku/testtools/GitLabClientTools.scala +++ b/graph-commons/src/test/scala/io/renku/testtools/GitLabClientTools.scala @@ -27,7 +27,8 @@ import io.renku.generators.Generators.Implicits._ import io.renku.http.client.RestClient.ResponseMappingF import io.renku.http.client.{AccessToken, GitLabClient} import org.http4s.Method.{DELETE, GET, HEAD, POST, PUT} -import org.http4s.{Method, Uri, UrlForm} +import org.http4s.multipart.Multipart +import org.http4s.{Method, Uri} import org.scalacheck.Gen import org.scalamock.clazz.Mock import org.scalamock.function.MockFunctions @@ -73,7 +74,7 @@ trait GitLabClientTools[F[_]] { .repeat(expectedNumberOfCalls) case PUT => (gitLabClient - .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[F, ResultType])( + .put(_: Uri, _: String Refined NonEmpty, _: Multipart[F])(_: ResponseMappingF[F, ResultType])( _: Option[AccessToken] )) .expects(*, maybeEndpointName.map(new MockParameter(_)).getOrElse(*), *, capture(responseMapping), *) diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index 5a653b2fdf..71a5606aed 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -815,7 +815,8 @@ Response body example for `Accept: application/ld+json`: API to update project data. -Each of the properties can be either set to a new value or omitted in case there's no new value. +Each of the properties can be either set to a new value or omitted in case there's no new value for it. +The new values should be sent as a `multipart/form-data` in case there's new image or as JSON in case no update for an image is needed. The properties that can be updated are: * description - possible values are: @@ -823,7 +824,7 @@ The properties that can be updated are: * any non-blank String value * image - possible values are: * `null` for removing the current image - * any relative or absolute link to the image + * any file content * keywords - an array of String values; an empty array removes all the keywords * visibility - possible values are: `public`, `internal`, `private` @@ -835,15 +836,19 @@ The endpoint requires an authorization token to be passed. Supported headers are **Request** +* case when there's no update for the image (with `Content-Type: application/json`) + ```json { "description": "a new project description", - "image": "image.png", "keywords": ["keyword1", "keyword2"], "visibility": "public|internal|private" } ``` +* case when there's an update for the image (with `Content-Type: multipart/form-data`) each parameter has to be listed as a form data field. +The `image` field has to contain the file data. + **Response** | Status | Description | diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala index 33d685fa23..7264f81e92 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/docs/model.scala @@ -22,7 +22,7 @@ import cats.Show import cats.syntax.all._ import io.circe.{Encoder, Json} import io.renku.jsonld.JsonLD -import io.renku.knowledgegraph.docs.model.Example.{JsonExample, JsonLDExample} +import io.renku.knowledgegraph.docs.model.Example.{JsonExample, JsonLDExample, StringExample} import io.renku.knowledgegraph.docs.model.OAuthFlows.OAuthFlow import io.renku.knowledgegraph.docs.model.Path.OpMapping @@ -340,6 +340,9 @@ object model { def `application/json`: MediaType = MediaType.WithoutSchema("application/json", Map.empty) + + def `multipart/form-data`(exampleName: String, example: String): MediaType = + MediaType.WithoutSchema("multipart/form-data", Map(exampleName -> StringExample(example))) } final case class Response( diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 34518439e6..6086afb3f7 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -28,8 +28,10 @@ import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry +import org.http4s.MediaType.application import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl +import org.http4s.headers.`Content-Type` import org.http4s.{Request, Response} import org.typelevel.log4cats.Logger @@ -56,13 +58,22 @@ private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F .merge .handleErrorWith(relevantError(slug)(_)) - private lazy val decodePayload: Request[F] => EitherT[F, Response[F], ProjectUpdates] = req => - EitherT { - req - .as[ProjectUpdates] - .map(_.asRight[Response[F]]) - .handleError(badRequest(_).asLeft[ProjectUpdates]) - } + private lazy val decodePayload: Request[F] => EitherT[F, Response[F], ProjectUpdates] = { + case req if req.contentType contains `Content-Type`(application.json) => + EitherT { + req + .as[ProjectUpdates] + .map(_.asRight[Response[F]]) + .handleError(badRequest(_).asLeft[ProjectUpdates]) + } + case req => + EitherT { + Response[F](InternalServerError) + .withEntity(Message.Error.unsafeApply(s"'${req.contentType}' not supported")) + .asLeft + .pure[F] + } + } private def badRequest: Throwable => Response[F] = { _ => Response[F](BadRequest).withEntity(Message.Error("Invalid payload")) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index bb9a8218e6..9817e1bf47 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -34,7 +34,8 @@ object EndpointDocs extends docs.EndpointDocs { "Project Update", """|API to update project data. | - |Each of the properties can be either set to a new value or omitted in case there's no new value. + |Each of the properties can be either set to a new value or omitted in case there's no new value for it. + |The new values should be sent as a `multipart/form-data` in case there's new image or as JSON in case no update for an image is needed. | |The properties that can be updated are: |* description - possible values are: @@ -42,7 +43,7 @@ object EndpointDocs extends docs.EndpointDocs { | * any non-blank String value |* image - possible values are: | * `null` for removing the current image - | * any relative or absolute link to the image + | * any file content |* keywords - an array of String values; an empty array removes all the keywords |* visibility - possible values are: `public`, `internal`, `private` | @@ -57,10 +58,17 @@ object EndpointDocs extends docs.EndpointDocs { Schema.`Object`(properties = Map("visibility" -> Schema.EnumString(projects.Visibility.all.map(_.value)))), json"""{ "description": "a new project description", - "image": "image.png", "keywords": ["keyword1", "keyword2"], "visibility": "public|internal|private" }""" + ), + MediaType.`multipart/form-data`( + "Form data example", + """|'description="a new project description"'; + |'keywords=[keyword1,keyword2]'; + |'visibility="public|internal|private"'; + |'image=@"path-to-my-image"' + |""".stripMargin ) ) ), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index 46cbf3e7bc..c0ea95be58 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -18,10 +18,12 @@ package io.renku.knowledgegraph.projects.update +import ProjectUpdates.Image import cats.MonadThrow import cats.effect.Async import cats.syntax.all._ import eu.timepit.refined.auto._ +import fs2.Chunk import io.circe.{Decoder, Json} import io.renku.data.Message import io.renku.graph.model.projects @@ -30,8 +32,11 @@ import io.renku.http.tinytypes.TinyTypeURIEncoder._ import org.http4s.Status._ import org.http4s.circe.CirceEntityDecoder._ import org.http4s.circe.jsonOf +import org.http4s.headers.{`Content-Disposition`, `Content-Type`} import org.http4s.implicits._ -import org.http4s.{Request, Response, Status, UrlForm} +import org.http4s.multipart.{Multipart, Multiparts, Part} +import org.http4s.{Headers, MediaType, Request, Response, Status} +import org.typelevel.ci._ private trait GLProjectUpdater[F[_]] { def updateProject(slug: projects.Slug, @@ -50,17 +55,38 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU updates: ProjectUpdates, at: AccessToken ): F[Either[Message, Option[GLUpdatedProject]]] = - if (updates.glUpdateNeeded) - GitLabClient[F] - .put(uri"projects" / slug, "edit-project", toUrlForm(updates))(mapResponse)(at.some) - .map(_.map(_.some)) - else + if (updates.glUpdateNeeded) { + implicit val token: Option[AccessToken] = at.some + toMultipart(updates) + .flatMap( + GitLabClient[F] + .put(uri"projects" / slug, "edit-project", _)(mapResponse) + .map(_.map(_.some)) + ) + } else Option.empty[GLUpdatedProject].asRight[Message].pure[F] - private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => - UrlForm.empty - .updateFormField("avatar", newImage.map(_.fold[String](ifEmpty = null)(_.value))) - .updateFormField("visibility", newVisibility.map(_.value)) + private def toMultipart: ProjectUpdates => F[Multipart[F]] = { case ProjectUpdates(_, newImage, _, newVisibility) => + val maybeVisibilityPart = + newVisibility.map(v => Part.formData[F]("visibility", v.value)) + + val maybeAvatarPart = + newImage.map( + _.fold( + Part[F]( + Headers(`Content-Disposition`("form-data", Map(ci"name" -> "avatar")), + `Content-Type`(MediaType.image.jpeg) + ), + fs2.Stream.empty + ) + ) { case Image(name, mediaType, data) => + Part.fileData[F]("avatar", name, fs2.Stream.chunk(Chunk.byteVector(data)), `Content-Type`(mediaType)) + } + ) + + Multiparts + .forSync[F] + .flatMap(_.multipart(Vector(maybeVisibilityPart, maybeAvatarPart).flatten)) } private lazy val mapResponse diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index 9c6ca2b404..b6b7828994 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -18,16 +18,18 @@ package io.renku.knowledgegraph.projects.update +import ProjectUpdates.Image import cats.syntax.all._ import io.circe.syntax._ import io.circe.{Decoder, DecodingFailure, Encoder, Json} -import io.renku.graph.model.images.ImageUri import io.renku.graph.model.projects import io.renku.tinytypes.json.TinyTypeDecoders._ import io.renku.tinytypes.{From, TinyType} +import org.http4s.MediaType +import scodec.bits.ByteVector private final case class ProjectUpdates(newDescription: Option[Option[projects.Description]], - newImage: Option[Option[ImageUri]], + newImage: Option[Option[Image]], newKeywords: Option[Set[projects.Keyword]], newVisibility: Option[projects.Visibility] ) { @@ -47,19 +49,20 @@ private object ProjectUpdates { lazy val empty: ProjectUpdates = ProjectUpdates(None, None, None, None) - implicit val encoder: Encoder[ProjectUpdates] = Encoder.instance { - case ProjectUpdates(newDescription, newImage, newKeywords, newVisibility) => + final case class Image(name: String, mediaType: MediaType, data: ByteVector) + + implicit val jsonEncoder: Encoder[ProjectUpdates] = Encoder.instance { + case ProjectUpdates(newDescription, _, newKeywords, newVisibility) => Json.obj( List( newDescription.map(v => "description" -> v.fold(Json.Null)(_.asJson)), - newImage.map(v => "image" -> v.fold(Json.Null)(_.asJson)), newKeywords.map(v => "keywords" -> v.asJson), newVisibility.map(v => "visibility" -> v.asJson) ).flatten: _* ) } - implicit val decoder: Decoder[ProjectUpdates] = + implicit val jsonDecoder: Decoder[ProjectUpdates] = Decoder.instance { cur => def toOptionOfOption[T <: TinyType { type V = String }](prop: String, ttFactory: From[T]) = cur @@ -71,9 +74,8 @@ private object ProjectUpdates { for { newDesc <- toOptionOfOption("description", projects.Description) - newImage <- toOptionOfOption("image", ImageUri) newKeywords <- cur.downField("keywords").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] - } yield ProjectUpdates(newDesc, newImage, newKeywords, newVisibility) + } yield ProjectUpdates(newDesc, newImage = None, newKeywords, newVisibility) } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 4680cf3cf2..58bee59ec7 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -42,11 +42,11 @@ import org.scalatest.matchers.should class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matchers with AsyncMockFactory { - it should "update the project and return Accepted on success" in { + it should "decode the JSON payload, update the project and return Accepted on success" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen.suchThat(_.newImage.isEmpty).generateOne givenUpdatingProject(slug, updates, authUser, returning = ().pure[IO]) @@ -75,7 +75,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen.suchThat(_.newImage.isEmpty).generateOne val failure = Gen .oneOf( @@ -96,7 +96,7 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.generateOne + val updates = projectUpdatesGen.suchThat(_.newImage.isEmpty).generateOne val exception = exceptions.generateOne givenUpdatingProject(slug, updates, authUser, returning = exception.raiseError[IO, Nothing]) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala index c6a35aed4c..95ef6947b8 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdaterSpec.scala @@ -42,11 +42,14 @@ import org.http4s.Method.PUT import org.http4s.Status.{BadRequest, Ok} import org.http4s.circe.CirceEntityEncoder._ import org.http4s.implicits._ -import org.http4s.{Request, Response, Uri, UrlForm} +import org.http4s.multipart.Multipart +import org.http4s.{Request, Response, Uri} +import org.scalamock.matchers.ArgCapture.CaptureOne import org.scalamock.scalatest.AsyncMockFactory -import org.scalatest.EitherValues import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.matchers.should +import org.scalatest.{EitherValues, OptionValues, Succeeded} +import scodec.bits.ByteVector class GLProjectUpdaterSpec extends AsyncFlatSpec @@ -54,6 +57,7 @@ class GLProjectUpdaterSpec with AsyncMockFactory with should.Matchers with EitherValues + with OptionValues with GitLabClientTools[IO] { it should s"call GL's PUT gl/projects/:slug and return updated values on success" in { @@ -63,9 +67,12 @@ class GLProjectUpdaterSpec val accessToken = accessTokens.generateOne val updatedProject = glUpdatedProjectsGen.generateOne - givenEditProjectAPICall(slug, newValues, accessToken, returning = updatedProject.asRight.pure[IO]) + val multipartCaptor = givenEditProjectAPICall(slug, accessToken, returning = updatedProject.asRight.pure[IO]) - finder.updateProject(slug, newValues, accessToken).asserting(_.value shouldBe updatedProject.some) + finder + .updateProject(slug, newValues, accessToken) + .asserting(_.value shouldBe updatedProject.some) + .flatMap(_ => verifyRequest(multipartCaptor, newValues)) } it should s"do nothing if neither new image nor visibility is set in the update" in { @@ -80,15 +87,13 @@ class GLProjectUpdaterSpec it should s"call GL's PUT gl/projects/:slug and return GL message if returned" in { val slug = projectSlugs.generateOne - val newValues = projectUpdatesGen.suchThat(u => u.newImage.orElse(u.newVisibility).isDefined).generateOne + val newValues = projectUpdatesGen.suchThat(u => u.glUpdateNeeded).generateOne val accessToken = accessTokens.generateOne val error = Message.Error.fromJsonUnsafe(jsons.generateOne) - givenEditProjectAPICall(slug, newValues, accessToken, returning = error.asLeft.pure[IO]) + givenEditProjectAPICall(slug, accessToken, returning = error.asLeft.pure[IO]) - finder - .updateProject(slug, newValues, accessToken) - .asserting(_.left.value shouldBe error) + finder.updateProject(slug, newValues, accessToken).asserting(_.left.value shouldBe error) } it should "succeed and return updated values if PUT gl/projects/:slug returns 200 OK" in { @@ -117,26 +122,18 @@ class GLProjectUpdaterSpec private lazy val finder = new GLProjectUpdaterImpl[IO] private def givenEditProjectAPICall(slug: projects.Slug, - newValues: ProjectUpdates, accessToken: AccessToken, returning: IO[Either[Message, GLUpdatedProject]] ) = { + val multipartCaptor = CaptureOne[Multipart[IO]]() val endpointName: String Refined NonEmpty = "edit-project" (glClient - .put(_: Uri, _: String Refined NonEmpty, _: UrlForm)(_: ResponseMappingF[IO, Either[Message, GLUpdatedProject]])( - _: Option[AccessToken] - )) - .expects(uri"projects" / slug, endpointName, toUrlForm(newValues), *, accessToken.some) + .put(_: Uri, _: String Refined NonEmpty, _: Multipart[IO])( + _: ResponseMappingF[IO, Either[Message, GLUpdatedProject]] + )(_: Option[AccessToken])) + .expects(uri"projects" / slug, endpointName, capture(multipartCaptor), *, accessToken.some) .returning(returning) - } - - private def toUrlForm: ProjectUpdates => UrlForm = { case ProjectUpdates(_, newImage, _, newVisibility) => - UrlForm( - List( - newImage.map("avatar" -> _.fold[String](null)(_.value)), - newVisibility.map("visibility" -> _.value) - ).flatten: _* - ) + multipartCaptor } private lazy val mapResponse: ResponseMappingF[IO, Either[Message, GLUpdatedProject]] = @@ -151,6 +148,31 @@ class GLProjectUpdaterSpec method = PUT ) + private def verifyRequest(multipartCaptor: CaptureOne[Multipart[IO]], newValues: ProjectUpdates) = { + + val parts = multipartCaptor.value.parts + + def findPart(name: String) = + parts + .find(_.name.value == name) + .getOrElse(fail(s"No '$name' part")) + + val visCheck = newValues.newVisibility + .map(v => findPart("visibility").as[String].asserting(_ shouldBe v.value)) + .getOrElse(Succeeded.pure[IO]) + + val imageCheck = newValues.newImage + .map { + case None => + findPart("avatar").body.covary[IO].compile.toList.asserting(_ shouldBe Nil) + case Some(v) => + findPart("avatar").as[ByteVector].asserting(_ shouldBe v.data) + } + .getOrElse(Succeeded.pure[IO]) + + visCheck >> imageCheck + } + private implicit lazy val responseEncoder: Encoder[GLUpdatedProject] = Encoder.instance { case GLUpdatedProject(image, visibility) => json"""{ diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala index b3533bea5b..ffccdce6b6 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/Generators.scala @@ -19,19 +19,30 @@ package io.renku.knowledgegraph.projects.update import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.nonEmptyStrings import io.renku.graph.model.RenkuTinyTypeGenerators.{imageUris, projectDescriptions, projectKeywords, projectVisibilities} +import io.renku.knowledgegraph.projects.update.ProjectUpdates.Image +import org.http4s.MediaType._ import org.scalacheck.Gen +import scodec.bits.ByteVector private object Generators { val projectUpdatesGen: Gen[ProjectUpdates] = for { maybeNewDesc <- projectDescriptions.toGeneratorOfOptions.toGeneratorOfOptions - maybeNewImage <- imageUris.toGeneratorOfOptions.toGeneratorOfOptions + maybeNewImage <- images.toGeneratorOfOptions.toGeneratorOfOptions maybeNewKeywords <- projectKeywords.toGeneratorOfSet().toGeneratorOfOptions maybeNewVisibility <- projectVisibilities.toGeneratorOfOptions } yield ProjectUpdates(maybeNewDesc, maybeNewImage, maybeNewKeywords, maybeNewVisibility) + lazy val images: Gen[Image] = + for { + name <- nonEmptyStrings() + mediaType <- Gen.oneOf(image.png, image.jpeg, image.gif, image.bmp, image.tiff, image.`vnd.microsoft.icon`) + data <- nonEmptyStrings().map(v => ByteVector(v.getBytes)) + } yield Image(name, mediaType, data) + val glUpdatedProjectsGen: Gen[GLUpdatedProject] = for { maybeNewImage <- imageUris.toGeneratorOfOptions diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala index 72c215d469..8f09616254 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdatesSpec.scala @@ -80,10 +80,10 @@ class ProjectUpdatesSpec extends AnyWordSpec with should.Matchers with ScalaChec } } - "encode/decode" should { + "JSON encode/decode" should { "encode/decode all standard cases" in { - forAll(projectUpdatesGen) { updates => + forAll(projectUpdatesGen.suchThat(_.newImage.isEmpty)) { updates => updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } } @@ -121,21 +121,5 @@ class ProjectUpdatesSpec extends AnyWordSpec with should.Matchers with ScalaChec updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates } - - "image = null to be considered as image removal" in { - - val updates = ProjectUpdates.empty.copy(newImage = Some(None)) - - updates.asJson shouldBe json"""{"image": null}""" - - updates.asJson.hcursor.as[ProjectUpdates].value shouldBe updates - } - - "image with a blank value to be considered as image removal" in { - - val json = json"""{"image": ${blankStrings().generateOne}}""" - - json.asJson.hcursor.as[ProjectUpdates].value shouldBe ProjectUpdates.empty.copy(newImage = Some(None)) - } } } From 56ba328d376cb60b74916df1b2561dfd767be7b5 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 25 Aug 2023 12:42:13 +0200 Subject: [PATCH 41/49] feat: GL project update to work with multipart request --- .../projects/update/Endpoint.scala | 42 ++-- .../projects/update/GLProjectUpdater.scala | 50 +++-- .../update/MultipartRequestCodec.scala | 195 ++++++++++++++++++ .../projects/update/ProjectUpdates.scala | 48 +++-- .../projects/update/EndpointSpec.scala | 18 +- .../update/MultipartRequestCodecSpec.scala | 96 +++++++++ 6 files changed, 387 insertions(+), 62 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodec.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodecSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 6086afb3f7..780ae0d01d 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -28,10 +28,11 @@ import io.renku.graph.model.projects import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser import io.renku.metrics.MetricsRegistry -import org.http4s.MediaType.application +import org.http4s.MediaType.{application, multipartType} import org.http4s.circe.CirceEntityDecoder._ import org.http4s.dsl.Http4sDsl import org.http4s.headers.`Content-Type` +import org.http4s.multipart.Multipart import org.http4s.{Request, Response} import org.typelevel.log4cats.Logger @@ -49,30 +50,33 @@ private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F with Endpoint[F] { override def `PATCH /projects/:slug`(slug: projects.Slug, request: Request[F], authUser: AuthUser): F[Response[F]] = - decodePayload(request) - .semiflatMap( + EitherT(decodePayload(request)) + .semiflatMap { updates => projectUpdater - .updateProject(slug, _, authUser) + .updateProject(slug, updates, authUser) .as(Response[F](Accepted).withEntity(Message.Info("Project update accepted"))) - ) + .flatTap(_ => Logger[F].info(show"Project $slug updated with $updates")) + } .merge - .handleErrorWith(relevantError(slug)(_)) + .handleErrorWith(relevantError(slug)) - private lazy val decodePayload: Request[F] => EitherT[F, Response[F], ProjectUpdates] = { + private lazy val decodePayload: Request[F] => F[Either[Response[F], ProjectUpdates]] = { case req if req.contentType contains `Content-Type`(application.json) => - EitherT { - req - .as[ProjectUpdates] - .map(_.asRight[Response[F]]) - .handleError(badRequest(_).asLeft[ProjectUpdates]) - } + req + .as[ProjectUpdates] + .map(_.asRight[Response[F]]) + .handleError(badRequest(_).asLeft[ProjectUpdates]) + case req if req.contentType.map(_.mediaType).exists(_.satisfies(multipartType("form-data"))) => + req + .as[Multipart[F]] + .flatMap(MultipartRequestDecoder[F].decode) + .map(_.asRight[Response[F]]) + .handleError(badRequest(_).asLeft[ProjectUpdates]) case req => - EitherT { - Response[F](InternalServerError) - .withEntity(Message.Error.unsafeApply(s"'${req.contentType}' not supported")) - .asLeft - .pure[F] - } + Response[F](InternalServerError) + .withEntity(Message.Error.unsafeApply(s"'${req.contentType}' not supported")) + .asLeft[ProjectUpdates] + .pure[F] } private def badRequest: Throwable => Response[F] = { _ => diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala index c0ea95be58..6f6fdff83c 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/GLProjectUpdater.scala @@ -57,37 +57,35 @@ private class GLProjectUpdaterImpl[F[_]: Async: GitLabClient] extends GLProjectU ): F[Either[Message, Option[GLUpdatedProject]]] = if (updates.glUpdateNeeded) { implicit val token: Option[AccessToken] = at.some - toMultipart(updates) - .flatMap( - GitLabClient[F] - .put(uri"projects" / slug, "edit-project", _)(mapResponse) - .map(_.map(_.some)) - ) + toMultipart(updates).flatMap( + GitLabClient[F] + .put(uri"projects" / slug, "edit-project", _)(mapResponse) + .map(_.map(_.some)) + ) } else Option.empty[GLUpdatedProject].asRight[Message].pure[F] - private def toMultipart: ProjectUpdates => F[Multipart[F]] = { case ProjectUpdates(_, newImage, _, newVisibility) => - val maybeVisibilityPart = - newVisibility.map(v => Part.formData[F]("visibility", v.value)) + private lazy val toMultipart: ProjectUpdates => F[Multipart[F]] = { + case ProjectUpdates(_, newImage, _, newVisibility) => + Multiparts + .forSync[F] + .flatMap(_.multipart(Vector(maybeVisibilityPart(newVisibility), maybeAvatarPart(newImage)).flatten)) + } - val maybeAvatarPart = - newImage.map( - _.fold( - Part[F]( - Headers(`Content-Disposition`("form-data", Map(ci"name" -> "avatar")), - `Content-Type`(MediaType.image.jpeg) - ), - fs2.Stream.empty - ) - ) { case Image(name, mediaType, data) => - Part.fileData[F]("avatar", name, fs2.Stream.chunk(Chunk.byteVector(data)), `Content-Type`(mediaType)) - } - ) + private def maybeVisibilityPart(newVisibility: Option[projects.Visibility]) = + newVisibility.map(v => Part.formData[F]("visibility", v.value)) - Multiparts - .forSync[F] - .flatMap(_.multipart(Vector(maybeVisibilityPart, maybeAvatarPart).flatten)) - } + private def maybeAvatarPart(newImage: Option[Option[Image]]) = + newImage.map( + _.fold( + Part[F]( + Headers(`Content-Disposition`("form-data", Map(ci"name" -> "avatar")), `Content-Type`(MediaType.image.jpeg)), + fs2.Stream.empty + ) + ) { case Image(name, mediaType, data) => + Part.fileData[F]("avatar", name, fs2.Stream.chunk(Chunk.byteVector(data)), `Content-Type`(mediaType)) + } + ) private lazy val mapResponse : PartialFunction[(Status, Request[F], Response[F]), F[Either[Message, GLUpdatedProject]]] = { diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodec.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodec.scala new file mode 100644 index 0000000000..72c1ad1940 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodec.scala @@ -0,0 +1,195 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import MultipartRequestCodec.PartName +import cats.MonadThrow +import cats.effect.{Async, Sync} +import cats.syntax.all._ +import eu.timepit.refined.api.{RefType, Refined} +import eu.timepit.refined.collection.NonEmpty +import fs2.{Chunk, Stream} +import io.renku.graph.model.projects +import io.renku.knowledgegraph.projects.update.ProjectUpdates.Image +import io.renku.tinytypes.{From, TinyType} +import org.http4s.MediaType.image +import org.http4s.headers.{`Content-Disposition`, `Content-Type`} +import org.http4s.multipart.{Multipart, Multiparts, Part} +import org.http4s.{Headers, MediaType} +import org.typelevel.ci.CIStringSyntax +import scodec.bits.ByteVector + +private trait MultipartRequestCodec[F[_]] extends MultipartRequestEncoder[F] with MultipartRequestDecoder[F] + +private object MultipartRequestCodec { + + def apply[F[_]: Async](implicit + encoder: MultipartRequestEncoder[F], + decoder: MultipartRequestDecoder[F] + ): MultipartRequestCodec[F] = new MultipartRequestCodec[F] { + override def encode(updates: ProjectUpdates): F[Multipart[F]] = encoder.encode(updates) + override def decode(multipart: Multipart[F]): F[ProjectUpdates] = decoder.decode(multipart) + } + + implicit def forAsync[F[_]](implicit F: Async[F]): MultipartRequestCodec[F] = apply[F] + + private[update] object PartName { + val description = "description" + val image = "image" + val keywords = "keywords" + val visibility = "visibility" + } +} + +private trait MultipartRequestEncoder[F[_]] { + def encode(updates: ProjectUpdates): F[Multipart[F]] +} + +private object MultipartRequestEncoder { + def apply[F[_]: Sync]: MultipartRequestEncoder[F] = forSync(implicitly[Sync[F]]) + implicit def forSync[F[_]](implicit F: Sync[F]): MultipartRequestEncoder[F] = new MultipartRequestEncoderImpl[F] +} + +private class MultipartRequestEncoderImpl[F[_]: Sync] extends MultipartRequestEncoder[F] { + + override def encode(updates: ProjectUpdates): F[Multipart[F]] = { + val ProjectUpdates(newDesc, newImage, newKeywords, newVisibility) = updates + + val parts = + Vector(maybeDescPart(newDesc), maybeImagePart(newImage), maybeVisibilityPart(newVisibility)).flatten + .appendedAll(maybeKeywordsParts(newKeywords)) + + Multiparts.forSync[F].flatMap(_.multipart(parts)) + } + + private def maybeDescPart(newDesc: Option[Option[projects.Description]]) = + newDesc.map( + _.fold(Part.formData[F](PartName.description, "").copy(body = Stream.empty): Part[F])(v => + Part.formData[F](PartName.description, v.value) + ) + ) + + private def maybeImagePart(newImage: Option[Option[Image]]) = + newImage.map( + _.fold( + Part[F]( + Headers(`Content-Disposition`("form-data", Map(ci"name" -> PartName.image)), `Content-Type`(image.jpeg)), + Stream.empty + ) + ) { case Image(name, mediaType, data) => + Part.fileData[F](PartName.image, name, fs2.Stream.chunk(Chunk.byteVector(data)), `Content-Type`(mediaType)) + } + ) + + private def maybeKeywordsParts(newKeywords: Option[Set[projects.Keyword]]) = + newKeywords + .map { + case set if set.isEmpty => Vector(Part.formData[F](s"${PartName.keywords}[]", "")) + case set => set.toVector.map(v => Part.formData[F](s"${PartName.keywords}[]", v.value)) + } + .getOrElse(Vector.empty) + + private def maybeVisibilityPart(newVisibility: Option[projects.Visibility]) = + newVisibility.map(v => Part.formData[F](PartName.visibility, v.value)) +} + +private trait MultipartRequestDecoder[F[_]] { + def decode(multipart: Multipart[F]): F[ProjectUpdates] +} + +private object MultipartRequestDecoder { + def apply[F[_]: Async]: MultipartRequestDecoder[F] = forAsync[F] + implicit def forAsync[F[_]](implicit F: Async[F]): MultipartRequestDecoder[F] = new MultipartRequestDecoderImpl[F] +} + +private class MultipartRequestDecoderImpl[F[_]: Async] extends MultipartRequestDecoder[F] { + + override def decode(multipart: Multipart[F]): F[ProjectUpdates] = + (maybeDesc(multipart), maybeImage(multipart), maybeKeywords(multipart), maybeVisibility(multipart)) + .mapN(ProjectUpdates.apply) + + private lazy val maybeDesc: Multipart[F] => F[Option[Option[projects.Description]]] = + _.parts + .find(_.name contains PartName.description) + .map(_.as[String].flatMap(blankStringToNone(projects.Description))) + .sequence + + private lazy val maybeImage: Multipart[F] => F[Option[Option[Image]]] = + _.parts + .find(_.name contains PartName.image) + .map { part => + findImageData(part) >>= { + case None => Option.empty[Image].pure[F] + case Some(data) => (findImageFilename(part), findImageMediaType(part)).mapN(Image.apply(_, _, data).some) + } + } + .sequence + + private lazy val findImageData: Part[F] => F[Option[ByteVector]] = + _.as[ByteVector].map { + case bv if bv.isEmpty => Option.empty[ByteVector] + case bv => Some(bv) + } + + private lazy val findImageFilename: Part[F] => F[String] = + _.headers + .get[`Content-Disposition`] + .flatMap(_.parameters.get(ci"filename")) + .fold(new Exception(s"No filename on the ${PartName.image} part").raiseError[F, String])(_.pure[F]) + + private lazy val findImageMediaType: Part[F] => F[MediaType] = + _.headers + .get[`Content-Type`] + .fold(new Exception(s"No media type on the ${PartName.image} part").raiseError[F, MediaType])(_.mediaType.pure[F]) + + private lazy val maybeKeywords: Multipart[F] => F[Option[Set[projects.Keyword]]] = + _.parts.filter(_.name exists (_ startsWith PartName.keywords)) match { + case prts if prts.isEmpty => Option.empty[Set[projects.Keyword]].pure[F] + case prts => + prts + .map(_.as[String].flatMap(blankStringToNone(projects.Keyword))) + .sequence + .map(_.flatten.toSet.some) + } + + private lazy val maybeVisibility: Multipart[F] => F[Option[projects.Visibility]] = + _.parts + .find(_.name contains PartName.visibility) + .map(_.as[String].flatMap(projects.Visibility.from(_).fold(_.raiseError[F, projects.Visibility], _.pure[F]))) + .sequence + + private type NonBlank = String Refined NonEmpty + + private def blankStringToNone[TT <: TinyType { type V = String }](implicit + tinyTypeFactory: From[TT] + ): String => F[Option[TT]] = + (blankToNone andThen toOption[TT])(_) + + private lazy val blankToNone: String => Option[NonBlank] = _.trim match { + case "" => None + case nonBlank => RefType.applyRef[NonBlank](nonBlank).fold(_ => None, Option.apply) + } + + private def toOption[TT <: TinyType { type V = String }](implicit + ttFactory: From[TT] + ): Option[NonBlank] => F[Option[TT]] = { + case Some(nonBlank) => MonadThrow[F].fromEither(ttFactory.from(nonBlank.value).map(Option.apply)) + case None => Option.empty[TT].pure[F] + } +} diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala index b6b7828994..be38ae5d73 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdates.scala @@ -19,6 +19,7 @@ package io.renku.knowledgegraph.projects.update import ProjectUpdates.Image +import cats.Show import cats.syntax.all._ import io.circe.syntax._ import io.circe.{Decoder, DecodingFailure, Encoder, Json} @@ -50,6 +51,11 @@ private object ProjectUpdates { lazy val empty: ProjectUpdates = ProjectUpdates(None, None, None, None) final case class Image(name: String, mediaType: MediaType, data: ByteVector) + object Image { + implicit val show: Show[Image] = Show.show { case Image(name, mediaType, _) => + show"$name ($mediaType)" + } + } implicit val jsonEncoder: Encoder[ProjectUpdates] = Encoder.instance { case ProjectUpdates(newDescription, _, newKeywords, newVisibility) => @@ -62,20 +68,32 @@ private object ProjectUpdates { ) } - implicit val jsonDecoder: Decoder[ProjectUpdates] = - Decoder.instance { cur => - def toOptionOfOption[T <: TinyType { type V = String }](prop: String, ttFactory: From[T]) = - cur - .downField(prop) - .success - .fold(Option.empty[Option[T]].asRight[DecodingFailure]) { - _.as[Option[T]](blankStringToNoneDecoder(ttFactory)).map(_.some) - } + implicit val jsonDecoder: Decoder[ProjectUpdates] = Decoder.instance { cur => + def toOptionOfOption[T <: TinyType { type V = String }](prop: String, ttFactory: From[T]) = + cur + .downField(prop) + .success + .fold(Option.empty[Option[T]].asRight[DecodingFailure]) { + _.as[Option[T]](blankStringToNoneDecoder(ttFactory)).map(_.some) + } - for { - newDesc <- toOptionOfOption("description", projects.Description) - newKeywords <- cur.downField("keywords").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) - newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] - } yield ProjectUpdates(newDesc, newImage = None, newKeywords, newVisibility) - } + for { + newDesc <- toOptionOfOption("description", projects.Description) + newKeywords <- cur.downField("keywords").as[Option[List[projects.Keyword]]].map(_.map(_.toSet)) + newVisibility <- cur.downField("visibility").as[Option[projects.Visibility]] + } yield ProjectUpdates(newDesc, newImage = None, newKeywords, newVisibility) + } + + implicit val show: Show[ProjectUpdates] = Show.show { + case ProjectUpdates(newDescription, newImage, newKeywords, newVisibility) => + def showOption[T](opt: Option[T])(implicit show: Show[T]) = + opt.fold(ifEmpty = "none")(_.show) + + List( + newDescription.map(v => s"description=${showOption(v)}"), + newImage.map(v => s"image=${showOption(v)}"), + newKeywords.map(v => s"keywords=${v.mkString(", ")}"), + newVisibility.map(v => s"visibility=$v") + ).flatten.mkString(", ") + } } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala index 58bee59ec7..7f1a799709 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/EndpointSpec.scala @@ -50,8 +50,22 @@ class EndpointSpec extends AsyncFlatSpec with CustomAsyncIOSpec with should.Matc givenUpdatingProject(slug, updates, authUser, returning = ().pure[IO]) - endpoint - .`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) >>= { response => + endpoint.`PATCH /projects/:slug`(slug, Request[IO]().withEntity(updates.asJson), authUser) >>= { response => + response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> + response.as[Message].asserting(_ shouldBe Message.Info("Project update accepted")) + } + } + + it should "decode the Multipart payload, update the project and return Accepted on success" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + val updates = projectUpdatesGen.generateOne + + givenUpdatingProject(slug, updates, authUser, returning = ().pure[IO]) + + MultipartRequestEncoder[IO].encode(updates).map(mp => Request[IO]().withEntity(mp).putHeaders(mp.headers)) >>= + (req => endpoint.`PATCH /projects/:slug`(slug, req, authUser)) >>= { response => response.pure[IO].asserting(_.status shouldBe Status.Accepted) >> response.as[Message].asserting(_ shouldBe Message.Info("Project update accepted")) } diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodecSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodecSpec.scala new file mode 100644 index 0000000000..fd2086f02c --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/MultipartRequestCodecSpec.scala @@ -0,0 +1,96 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.IO +import cats.effect.testing.scalatest.AsyncIOSpec +import cats.syntax.all._ +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuTinyTypeGenerators.{projectDescriptions, projectKeywords, projectVisibilities} +import io.renku.graph.model.projects +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class MultipartRequestCodecSpec extends AsyncFlatSpec with AsyncIOSpec with should.Matchers { + + private val codec: MultipartRequestCodec[IO] = MultipartRequestCodec[IO] + + it should "decode/encode the multipart request with multiple values" in { + + val updates = ProjectUpdates.empty + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode the multipart request for empty updates" in { + + val updates = ProjectUpdates.empty + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the description if set" in { + + val updates = ProjectUpdates.empty.copy(newDescription = projectDescriptions.generateSome.some) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the description if removed" in { + + val updates = ProjectUpdates.empty.copy(newDescription = Some(None)) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the image if set" in { + + val updates = ProjectUpdates.empty.copy(newImage = Generators.images.generateSome.some) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the image if removed" in { + + val updates = ProjectUpdates.empty.copy(newImage = Some(None)) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the keywords if set" in { + + val updates = ProjectUpdates.empty.copy(newKeywords = projectKeywords.generateSet(min = 1).some) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the keywords if removed" in { + + val updates = ProjectUpdates.empty.copy(newKeywords = Set.empty[projects.Keyword].some) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } + + it should "decode/encode new value for the visibility if set" in { + + val updates = ProjectUpdates.empty.copy(newVisibility = projectVisibilities.generateSome) + + (codec.encode(updates) >>= (MultipartRequestCodec[IO].decode(_))).asserting(_ shouldBe updates) + } +} From 4ace52f50e3e192b394d2495fc5544a2d085ed54 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 25 Aug 2023 15:46:36 +0200 Subject: [PATCH 42/49] doc: README & open api docs updated --- knowledge-graph/README.md | 37 ++++++++++++++-- .../projects/update/EndpointDocs.scala | 42 +++++++++++++++---- 2 files changed, 66 insertions(+), 13 deletions(-) diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index 71a5606aed..7edf12a476 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -836,7 +836,39 @@ The endpoint requires an authorization token to be passed. Supported headers are **Request** -* case when there's no update for the image (with `Content-Type: application/json`) +* Multipart request (preferred) +``` +PATCH /knowledge-graph/projects/jakub.chrobasik/create-test-10 HTTP/1.1 +Host: dev.renku.ch +Authorization: Bearer +Content-Length: 575 +Content-Type: multipart/form-data; boundary=----WebKitFormBoundary7MA4YWxkTrZu0gW + +------WebKitFormBoundary7MA4YWxkTrZu0gW +Content-Disposition: form-data; name="visibility" + +public +------WebKitFormBoundary7MA4YWxkTrZu0gW +Content-Disposition: form-data; name="description" + +desc test 1 +------WebKitFormBoundary7MA4YWxkTrZu0gW +Content-Disposition: form-data; name="keywords[]" + +key1 +------WebKitFormBoundary7MA4YWxkTrZu0gW +Content-Disposition: form-data; name="keywords[]" + +key2 +------WebKitFormBoundary7MA4YWxkTrZu0gW +Content-Disposition: form-data; name="image"; filename="image.png" +Content-Type: image/png + +(data) +------WebKitFormBoundary7MA4YWxkTrZu0gW-- +``` + +* JSON request (updating image not possible) ```json { @@ -846,9 +878,6 @@ The endpoint requires an authorization token to be passed. Supported headers are } ``` -* case when there's an update for the image (with `Content-Type: multipart/form-data`) each parameter has to be listed as a form data field. -The `image` field has to contain the file data. - **Response** | Status | Description | diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index 9817e1bf47..c0da8a83e8 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -39,7 +39,7 @@ object EndpointDocs extends docs.EndpointDocs { | |The properties that can be updated are: |* description - possible values are: - | * `null` for removing the current description + | * `null` or blank value for removing the current description | * any non-blank String value |* image - possible values are: | * `null` for removing the current image @@ -54,6 +54,38 @@ object EndpointDocs extends docs.EndpointDocs { "Properties with new values", required = true, Contents( + MediaType.`multipart/form-data`( + "Multipart request example", + """|PATCH /knowledge-graph/projects/namespace/name HTTP/1.1 + |Host: dev.renku.ch + |Authorization: Bearer + |Content-Length: 575 + |Content-Type: multipart/form-data; boundary=----WebKitFormBoundary7MA4YWxkTrZu0gW + | + |------WebKitFormBoundary7MA4YWxkTrZu0gW + |Content-Disposition: form-data; name="visibility" + | + |public + |------WebKitFormBoundary7MA4YWxkTrZu0gW + |Content-Disposition: form-data; name="description" + | + |desc test 1 + |------WebKitFormBoundary7MA4YWxkTrZu0gW + |Content-Disposition: form-data; name="keywords[]" + | + |key1 + |------WebKitFormBoundary7MA4YWxkTrZu0gW + |Content-Disposition: form-data; name="keywords[]" + | + |key2 + |------WebKitFormBoundary7MA4YWxkTrZu0gW + |Content-Disposition: form-data; name="image"; filename="image.png" + |Content-Type: image/png + | + |(data) + |------WebKitFormBoundary7MA4YWxkTrZu0gW-- + |""".stripMargin + ), MediaType.`application/json`( Schema.`Object`(properties = Map("visibility" -> Schema.EnumString(projects.Visibility.all.map(_.value)))), json"""{ @@ -61,14 +93,6 @@ object EndpointDocs extends docs.EndpointDocs { "keywords": ["keyword1", "keyword2"], "visibility": "public|internal|private" }""" - ), - MediaType.`multipart/form-data`( - "Form data example", - """|'description="a new project description"'; - |'keywords=[keyword1,keyword2]'; - |'visibility="public|internal|private"'; - |'image=@"path-to-my-image"' - |""".stripMargin ) ) ), From 0c4bb22991f111f9add17ced33a693642e9aa871 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 25 Aug 2023 18:12:58 +0200 Subject: [PATCH 43/49] feat: log user failures at INFO level --- .../knowledgegraph/projects/update/Endpoint.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala index 780ae0d01d..df3be0fc2a 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Endpoint.scala @@ -85,12 +85,18 @@ private class EndpointImpl[F[_]: Async: Logger](projectUpdater: ProjectUpdater[F private def relevantError(slug: projects.Slug): Throwable => F[Response[F]] = { case f: Failure => - Logger[F] - .error(f)(show"Updating project $slug failed") + logFailure(slug)(f) .as(Response[F](f.status).withEntity(f.message)) case ex: Exception => Logger[F] .error(ex)(show"Updating project $slug failed") .as(Response[F](InternalServerError).withEntity(Message.Error("Update failed"))) } + + private def logFailure(slug: projects.Slug): Failure => F[Unit] = { + case f if f.status == BadRequest || f.status == Conflict => + Logger[F].info(show"Updating project $slug failed: ${f.getMessage}") + case f => + Logger[F].error(f)(show"Updating project $slug failed") + } } From 6b8727d1eaa414376eff4192e5153bdc11eb7764 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Fri, 25 Aug 2023 19:16:53 +0200 Subject: [PATCH 44/49] feat: core client reporting improved --- knowledge-graph/src/main/resources/application.conf | 2 +- .../src/main/scala/io/renku/core/client/RenkuCoreClient.scala | 2 +- .../test/scala/io/renku/core/client/RenkuCoreClientSpec.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/knowledge-graph/src/main/resources/application.conf b/knowledge-graph/src/main/resources/application.conf index b628822f9b..0dd0a6506f 100644 --- a/knowledge-graph/src/main/resources/application.conf +++ b/knowledge-graph/src/main/resources/application.conf @@ -43,7 +43,7 @@ services { } gitlab { - url = "https://renkulab.io/gitlab" + url = "https://gitlab.dev.renku.ch" url = ${?GITLAB_BASE_URL} } } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala index 6d7b7edb69..0cc36fc05a 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/RenkuCoreClient.scala @@ -64,7 +64,7 @@ private class RenkuCoreClientImpl[F[_]: Async: Logger](coreUriForSchemaLoader: R .flatMap(_.findM(migratedAndMatchingSchema(projectUrl, accessToken))) .flatMapF[RenkuCoreUri.Versioned] { case Some(sv) => findCoreUri(sv) - case None => Result.failure(show"No API for $projectUrl. Quite likely migration required").pure[F].widen + case None => Result.failure("Project in unsupported version. Quite likely migration required").pure[F].widen } .value diff --git a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala index fa8342b8c6..b2017fc4ce 100644 --- a/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala +++ b/renku-core-client/src/test/scala/io/renku/core/client/RenkuCoreClientSpec.scala @@ -148,7 +148,7 @@ class RenkuCoreClientSpec client .findCoreUri(projectUrl, accessToken) - .asserting(_ shouldBe Result.failure(show"No API for $projectUrl. Quite likely migration required")) + .asserting(_ shouldBe Result.failure("Project in unsupported version. Quite likely migration required")) } "fail if any of the calls towards the Core API fails" in { From eeaf23de4218a4768722e891e6b2b1da3e08f617 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Tue, 29 Aug 2023 19:14:45 +0200 Subject: [PATCH 45/49] refactor: using http4s renderer for media type string representation --- .../src/test/scala/io/renku/http/client/GitLabClientSpec.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala b/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala index ff753d93b8..e0b5ff0058 100644 --- a/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala +++ b/graph-commons/src/test/scala/io/renku/http/client/GitLabClientSpec.scala @@ -41,6 +41,7 @@ import org.http4s.Method._ import org.http4s.Status.{Accepted, NotFound, Ok, Unauthorized} import org.http4s.circe.CirceEntityCodec.circeEntityDecoder import org.http4s.multipart.{Multiparts, Part} +import org.http4s.util.Renderer import org.http4s.{Header, Method, Request, Response, Status, Uri} import org.scalacheck.Gen import org.scalamock.scalatest.MockFactory @@ -248,7 +249,7 @@ class GitLabClientSpec ) .withHeader( "Content-Type", - containing(s"${multipart.`form-data`.mainType}/${multipart.`form-data`.subType}") + containing(Renderer.renderString(multipart.`form-data`)) ) .withHeader( "Content-Type", From 1efb1ac1b83974ff57d62b412e22228531a01b7c Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Tue, 29 Aug 2023 19:20:28 +0200 Subject: [PATCH 46/49] chore: API docs updated --- knowledge-graph/README.md | 2 +- .../io/renku/knowledgegraph/projects/update/EndpointDocs.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/knowledge-graph/README.md b/knowledge-graph/README.md index 7edf12a476..2bf7ff81d4 100644 --- a/knowledge-graph/README.md +++ b/knowledge-graph/README.md @@ -824,7 +824,7 @@ The properties that can be updated are: * any non-blank String value * image - possible values are: * `null` for removing the current image - * any file content + * any image file; at the moment GitLab accepts images of size 200kB max and media type of: `image/png`, `image/jpeg`, `image/gif`, `image/bmp`, `image/tiff`, `image/vnd.microsoft.icon` * keywords - an array of String values; an empty array removes all the keywords * visibility - possible values are: `public`, `internal`, `private` diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala index c0da8a83e8..3c206ed0b7 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/EndpointDocs.scala @@ -43,7 +43,7 @@ object EndpointDocs extends docs.EndpointDocs { | * any non-blank String value |* image - possible values are: | * `null` for removing the current image - | * any file content + | * any image file; at the moment GitLab accepts images of size 200kB max and media type of: `image/png`, `image/jpeg`, `image/gif`, `image/bmp`, `image/tiff`, `image/vnd.microsoft.icon` |* keywords - an array of String values; an empty array removes all the keywords |* visibility - possible values are: `public`, `internal`, `private` | From 383c2bb4676e8a791d9c790c1715bf7ff1ea62d8 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Tue, 29 Aug 2023 19:30:52 +0200 Subject: [PATCH 47/49] chore: default for renku-core-service-urls in core-client's application.conf --- renku-core-client/src/main/resources/application.conf | 1 + 1 file changed, 1 insertion(+) diff --git a/renku-core-client/src/main/resources/application.conf b/renku-core-client/src/main/resources/application.conf index 0718ffc2cc..ca347ea381 100644 --- a/renku-core-client/src/main/resources/application.conf +++ b/renku-core-client/src/main/resources/application.conf @@ -6,5 +6,6 @@ services { url = ${?RENKU_CORE_LATEST_URL} } + renku-core-service-urls = "http://renku-core-v10,http://renku-core-v9" renku-core-service-urls = ${?RENKU_CORE_SERVICE_URLS} } From e16f880b1a230cd39b11e043e60f5a843ff18e01 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 30 Aug 2023 12:49:18 +0200 Subject: [PATCH 48/49] feat: improved tailRecM for NestedF and Result --- project/Dependencies.scala | 1 - .../scala/io/renku/core/client/NestedF.scala | 19 ++++++++++++------- .../scala/io/renku/core/client/Result.scala | 12 ++++++++---- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index fd7e81921d..9922b0cf16 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -196,5 +196,4 @@ object Dependencies { val scalacheck = Seq( "org.scalacheck" %% "scalacheck" % V.scalacheck ) - } diff --git a/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala b/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala index 09960f5cff..6aa01f8444 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/NestedF.scala @@ -50,13 +50,18 @@ private class NestedF[F[_]: Monad] { fa.value.map(_.map(f)) } - override def tailRecM[A, B](a: A)(f: A => NestedF[Either[A, B]]): NestedF[B] = Nested { - f(a).value >>= { - case Result.Success(Right(vb)) => Result.success(vb).pure[F] - case Result.Success(Left(va)) => tailRecM(va)(f).value - case f: Result.Failure => f.pure[F].widen - } - } + override def tailRecM[A, B](a: A)(f: A => NestedF[Either[A, B]]): NestedF[B] = + Nested[F, Result, B]( + Monad[F].tailRecM[Result[A], Result[B]](Result.success(a)) { + case failure: Result.Failure => failure.asInstanceOf[Result[B]].asRight[Result[A]].pure[F] + case Result.Success(v) => + f(v).value.map { + case failure: Result.Failure => failure.asRight[Result[A]] + case Result.Success(Left(a)) => Left(Result.success(a)) + case Result.Success(Right(a)) => Right(Result.success(a)) + } + } + ) } implicit class NestedFOps[A](nestedF: NestedF[A]) { diff --git a/renku-core-client/src/main/scala/io/renku/core/client/Result.scala b/renku-core-client/src/main/scala/io/renku/core/client/Result.scala index 2441f673b4..6b69421b7b 100644 --- a/renku-core-client/src/main/scala/io/renku/core/client/Result.scala +++ b/renku-core-client/src/main/scala/io/renku/core/client/Result.scala @@ -68,9 +68,13 @@ object Result { } override def tailRecM[A, B](a: A)(f: A => Result[Either[A, B]]): Result[B] = - flatMap(f(a)) { - case Left(v) => tailRecM[A, B](v)(f) - case Right(b) => success(b) - } + FlatMap[Either[Throwable, *]] + .tailRecM(a)(f(_).toEither) + .fold({ + case f: Failure => f + case err => Result.failure(err.getMessage) + }, + Result.success + ) } } From 69e61fab9af87b11a93391ff964e081bf1cc2469 Mon Sep 17 00:00:00 2001 From: Jakub Chrobasik Date: Wed, 30 Aug 2023 17:23:49 +0200 Subject: [PATCH 49/49] feat: Project Update API to fail if project in non-recoverable failure --- .../projects/update/Failure.scala | 12 ++ .../projects/update/ProjectUpdater.scala | 37 ++++-- .../update/ProvisioningStatusFinder.scala | 79 +++++++++++++ .../projects/update/ProjectUpdaterSpec.scala | 86 +++++++++++--- .../update/ProvisioningStatusFinderSpec.scala | 109 ++++++++++++++++++ 5 files changed, 297 insertions(+), 26 deletions(-) create mode 100644 knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinder.scala create mode 100644 knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinderSpec.scala diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala index a149c26833..b0e5fda54e 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/Failure.scala @@ -18,6 +18,7 @@ package io.renku.knowledgegraph.projects.update +import ProvisioningStatusFinder.ProvisioningStatus.Unhealthy import cats.syntax.all._ import eu.timepit.refined.auto._ import io.renku.data.Message @@ -58,6 +59,17 @@ private object Failure { val cannotPushToBranch: Failure = Failure(Conflict, Message.Error("Updating project not possible; the user cannot push to the default branch")) + def onProvisioningNotHealthy(slug: projects.Slug, unhealthy: Unhealthy): Failure = + Failure( + Conflict, + Message.Error.unsafeApply( + show"Project $slug in unhealthy state: ${unhealthy.status}; Fix the project manually on contact administrator" + ) + ) + + def onProvisioningStatusCheck(slug: projects.Slug, cause: Throwable): Failure = + Failure(InternalServerError, Message.Error.unsafeApply(show"Check if project $slug in healthy state failed"), cause) + def onBranchAccessCheck(slug: projects.Slug, userId: persons.GitLabId, cause: Throwable): Failure = Failure(InternalServerError, Message.Error.unsafeApply(show"Check if pushing to git for $slug and $userId failed"), diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala index a33a9d10df..d66607b7c7 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProjectUpdater.scala @@ -18,6 +18,7 @@ package io.renku.knowledgegraph.projects.update +import ProvisioningStatusFinder.ProvisioningStatus.Unhealthy import cats.NonEmptyParallel import cats.effect.Async import cats.syntax.all._ @@ -35,9 +36,10 @@ private trait ProjectUpdater[F[_]] { private object ProjectUpdater { def apply[F[_]: Async: NonEmptyParallel: GitLabClient: MetricsRegistry: Logger]: F[ProjectUpdater[F]] = - (TriplesGeneratorClient[F], RenkuCoreClient[F]()) + (ProvisioningStatusFinder[F], TriplesGeneratorClient[F], RenkuCoreClient[F]()) .mapN( - new ProjectUpdaterImpl[F](BranchProtectionCheck[F], + new ProjectUpdaterImpl[F](_, + BranchProtectionCheck[F], ProjectGitUrlFinder[F], UserInfoFinder[F], GLProjectUpdater[F], @@ -48,13 +50,15 @@ private object ProjectUpdater { ) } -private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchProtectionCheck: BranchProtectionCheck[F], - projectGitUrlFinder: ProjectGitUrlFinder[F], - userInfoFinder: UserInfoFinder[F], - glProjectUpdater: GLProjectUpdater[F], - tgClient: TriplesGeneratorClient[F], - renkuCoreClient: RenkuCoreClient[F], - tgUpdatesFinder: TGUpdatesFinder[F] +private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger]( + provisioningStatusFinder: ProvisioningStatusFinder[F], + branchProtectionCheck: BranchProtectionCheck[F], + projectGitUrlFinder: ProjectGitUrlFinder[F], + userInfoFinder: UserInfoFinder[F], + glProjectUpdater: GLProjectUpdater[F], + tgClient: TriplesGeneratorClient[F], + renkuCoreClient: RenkuCoreClient[F], + tgUpdatesFinder: TGUpdatesFinder[F] ) extends ProjectUpdater[F] { override def updateProject(slug: projects.Slug, updates: ProjectUpdates, authUser: AuthUser): F[Unit] = @@ -63,7 +67,7 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr .flatMap(findTGUpdates(slug, updates, _)) .flatMap(updateTG(slug, _)) else - canPushToDefaultBranch(slug, authUser) >> { + checkPrerequisites(slug, authUser) >> { for { coreUpdates <- findCoreProjectUpdates(slug, updates, authUser) coreUri <- findCoreUri(coreUpdates, authUser) @@ -94,6 +98,19 @@ private class ProjectUpdaterImpl[F[_]: Async: NonEmptyParallel: Logger](branchPr .handleError(_.asLeft) .flatMap(_.fold(Failure.onTSUpdate(slug, _).raiseError[F, Unit], _.pure[F])) + private def checkPrerequisites(slug: projects.Slug, authUser: AuthUser): F[Unit] = + (noProvisioningFailures(slug), canPushToDefaultBranch(slug, authUser)) + .parMapN((_, _) => ()) + + private def noProvisioningFailures(slug: projects.Slug): F[Unit] = + provisioningStatusFinder + .checkHealthy(slug) + .adaptError(Failure.onProvisioningStatusCheck(slug, _)) + .flatMap { + case r @ Unhealthy(_) => Failure.onProvisioningNotHealthy(slug, r).raiseError[F, Unit] + case _ => ().pure[F] + } + private def canPushToDefaultBranch(slug: projects.Slug, authUser: AuthUser): F[Unit] = branchProtectionCheck .canPushToDefaultBranch(slug, authUser.accessToken) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinder.scala new file mode 100644 index 0000000000..26cf87b6d3 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinder.scala @@ -0,0 +1,79 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import ProvisioningStatusFinder.ProvisioningStatus +import cats.MonadThrow +import cats.effect.Async +import cats.syntax.all._ +import io.renku.eventlog.api.EventLogClient +import io.renku.eventlog.api.EventLogClient.SearchCriteria +import io.renku.graph.model.events.EventStatus +import io.renku.graph.model.events.EventStatus._ +import io.renku.graph.model.projects +import io.renku.http.rest.paging.model.PerPage +import org.typelevel.log4cats.Logger + +private trait ProvisioningStatusFinder[F[_]] { + def checkHealthy(slug: projects.Slug): F[ProvisioningStatus] +} + +private object ProvisioningStatusFinder { + + def apply[F[_]: Async: Logger]: F[ProvisioningStatusFinder[F]] = + EventLogClient[F].map(new ProvisioningStatusFinderImpl[F](_)) + + sealed trait ProvisioningStatus { + lazy val widen: ProvisioningStatus = this + } + object ProvisioningStatus { + type Healthy = Healthy.type + case object Healthy extends ProvisioningStatus + final case class Unhealthy(status: EventStatus) extends ProvisioningStatus + } +} + +private class ProvisioningStatusFinderImpl[F[_]: MonadThrow](elClient: EventLogClient[F]) + extends ProvisioningStatusFinder[F] { + + private val okProvisioningStatuses = Set( + New, + Skipped, + GeneratingTriples, + GenerationRecoverableFailure, + TriplesGenerated, + TransformingTriples, + TransformationRecoverableFailure, + TriplesStore, + AwaitingDeletion, + Deleting + ) + + override def checkHealthy(slug: projects.Slug): F[ProvisioningStatus] = + elClient + .getEvents( + SearchCriteria.forProject(slug).withPerPage(PerPage(1)).sortBy(SearchCriteria.Sort.EventDateDesc) + ) + .flatMap(r => MonadThrow[F].fromEither(r.toEither)) + .map(_.headOption.map(_.status)) + .map(_.fold(ifEmpty = ProvisioningStatus.Healthy.widen) { + case status if okProvisioningStatuses contains status => ProvisioningStatus.Healthy.widen + case status => ProvisioningStatus.Unhealthy(status) + }) +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala index 191d3a1fc9..47c34110fe 100644 --- a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProjectUpdaterSpec.scala @@ -19,6 +19,7 @@ package io.renku.knowledgegraph.projects.update import Generators._ +import ProvisioningStatusFinder.ProvisioningStatus import cats.effect.IO import cats.syntax.all._ import io.renku.core.client.Generators.{coreUrisVersioned, resultDetailedFailures, resultSuccesses, userInfos} @@ -28,6 +29,7 @@ import io.renku.generators.CommonGraphGenerators.authUsers import io.renku.generators.Generators.Implicits._ import io.renku.generators.Generators.{exceptions, jsons} import io.renku.graph.model.RenkuTinyTypeGenerators.{projectGitHttpUrls, projectSlugs} +import io.renku.graph.model.events.EventStatus import io.renku.graph.model.projects import io.renku.http.client.{AccessToken, UserAccessToken} import io.renku.interpreters.TestLogger @@ -68,7 +70,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) val projectGitUrl = projectGitHttpUrls.generateOne givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) @@ -93,14 +95,51 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul updater.updateProject(slug, updates, authUser).assertNoException } + it should "if core update needed, " + + "fail if the project is in unhealthy status from the provisioning point of view" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val provisioningStatus = ProvisioningStatus.Unhealthy(EventStatus.GenerationNonRecoverableFailure) + givenProvisioningStatusFinding(slug, returning = provisioningStatus.pure[IO]) + + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onProvisioningNotHealthy(slug, provisioningStatus)) + } + + it should "if core update needed, " + + "fail if checking the project status from the provisioning point of view fails" in { + + val authUser = authUsers.generateOne + val slug = projectSlugs.generateOne + + givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + + val exception = exceptions.generateOne + givenProvisioningStatusFinding(slug, returning = exception.raiseError[IO, Nothing]) + + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne + + updater + .updateProject(slug, updates, authUser) + .assertThrowsError[Exception](_ shouldBe Failure.onProvisioningStatusCheck(slug, exception)) + } + it should "if core update needed, " + "fail if pushing to the default branch check return false" in { val authUser = authUsers.generateOne val slug = projectSlugs.generateOne - val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne givenBranchProtectionChecking(slug, authUser.accessToken, returning = false.pure[IO]) + givenProvisioningStatusFinding(slug, returning = ProvisioningStatus.Healthy.pure[IO]) + val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne updater.updateProject(slug, updates, authUser).assertThrowsError[Exception](_ shouldBe Failure.cannotPushToBranch) } @@ -114,6 +153,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val exception = exceptions.generateOne givenBranchProtectionChecking(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) + givenProvisioningStatusFinding(slug, returning = ProvisioningStatus.Healthy.pure[IO]) updater .updateProject(slug, updates, authUser) @@ -127,7 +167,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) val exception = exceptions.generateOne givenProjectGitUrlFinding(slug, authUser.accessToken, returning = exception.raiseError[IO, Nothing]) @@ -145,7 +185,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) givenProjectGitUrlFinding(slug, authUser.accessToken, returning = None.pure[IO]) givenUserInfoFinding(authUser.accessToken, returning = userInfos.generateSome.pure[IO]) @@ -162,7 +202,8 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitHttpUrls.generateSome.pure[IO]) val exception = exceptions.generateOne givenUserInfoFinding(authUser.accessToken, returning = exception.raiseError[IO, Nothing]) @@ -179,7 +220,8 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) + givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitHttpUrls.generateSome.pure[IO]) givenUserInfoFinding(authUser.accessToken, returning = None.pure[IO]) @@ -195,7 +237,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) val projectGitUrl = projectGitHttpUrls.generateOne givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) @@ -216,7 +258,7 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul val slug = projectSlugs.generateOne val updates = projectUpdatesGen.suchThat(_.coreUpdateNeeded).generateOne - givenBranchProtectionChecking(slug, authUser.accessToken, returning = true.pure[IO]) + givenPrerequisitesCheckFine(slug, authUser.accessToken) val projectGitUrl = projectGitHttpUrls.generateOne givenProjectGitUrlFinding(slug, authUser.accessToken, returning = projectGitUrl.some.pure[IO]) @@ -323,14 +365,16 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul } private implicit lazy val logger: TestLogger[IO] = TestLogger[IO]() - private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] - private val projectGitUrlFinder = mock[ProjectGitUrlFinder[IO]] - private val userInfoFinder = mock[UserInfoFinder[IO]] - private val glProjectUpdater = mock[GLProjectUpdater[IO]] - private val tgClient = mock[TriplesGeneratorClient[IO]] - private val renkuCoreClient = mock[RenkuCoreClient[IO]] - private val tgUpdatesFinder = mock[TGUpdatesFinder[IO]] - private lazy val updater = new ProjectUpdaterImpl[IO](branchProtectionCheck, + private val provisioningStatusFinder = mock[ProvisioningStatusFinder[IO]] + private val branchProtectionCheck = mock[BranchProtectionCheck[IO]] + private val projectGitUrlFinder = mock[ProjectGitUrlFinder[IO]] + private val userInfoFinder = mock[UserInfoFinder[IO]] + private val glProjectUpdater = mock[GLProjectUpdater[IO]] + private val tgClient = mock[TriplesGeneratorClient[IO]] + private val renkuCoreClient = mock[RenkuCoreClient[IO]] + private val tgUpdatesFinder = mock[TGUpdatesFinder[IO]] + private lazy val updater = new ProjectUpdaterImpl[IO](provisioningStatusFinder, + branchProtectionCheck, projectGitUrlFinder, userInfoFinder, glProjectUpdater, @@ -339,6 +383,16 @@ class ProjectUpdaterSpec extends AsyncFlatSpec with CustomAsyncIOSpec with shoul tgUpdatesFinder ) + private def givenPrerequisitesCheckFine(slug: projects.Slug, at: UserAccessToken) = { + givenProvisioningStatusFinding(slug, returning = ProvisioningStatus.Healthy.pure[IO]) + givenBranchProtectionChecking(slug, at, returning = true.pure[IO]) + } + + private def givenProvisioningStatusFinding(slug: projects.Slug, returning: IO[ProvisioningStatus]) = + (provisioningStatusFinder.checkHealthy _) + .expects(slug) + .returning(returning) + private def givenBranchProtectionChecking(slug: projects.Slug, at: UserAccessToken, returning: IO[Boolean]) = (branchProtectionCheck.canPushToDefaultBranch _) .expects(slug, at) diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinderSpec.scala new file mode 100644 index 0000000000..6673e7f60a --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/projects/update/ProvisioningStatusFinderSpec.scala @@ -0,0 +1,109 @@ +/* + * Copyright 2023 Swiss Data Science Center (SDSC) + * A partnership between École Polytechnique Fédérale de Lausanne (EPFL) and + * Eidgenössische Technische Hochschule Zürich (ETHZ). + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.renku.knowledgegraph.projects.update + +import cats.effect.IO +import cats.effect.testing.scalatest.AsyncIOSpec +import cats.syntax.all._ +import io.renku.eventlog.api.EventLogClient +import io.renku.eventlog.api.EventLogClient.{Result, SearchCriteria} +import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators.{exceptions, nonEmptyStrings} +import io.renku.graph.model.EventContentGenerators.eventInfos +import io.renku.graph.model.RenkuTinyTypeGenerators.projectSlugs +import io.renku.graph.model.events.EventStatus.{GenerationNonRecoverableFailure, TransformationNonRecoverableFailure} +import io.renku.graph.model.events.{EventInfo, EventStatus} +import io.renku.graph.model.projects +import io.renku.http.rest.paging.model.PerPage +import io.renku.knowledgegraph.projects.update.ProvisioningStatusFinder.ProvisioningStatus.{Healthy, Unhealthy} +import org.scalamock.scalatest.AsyncMockFactory +import org.scalatest.OptionValues +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should + +class ProvisioningStatusFinderSpec + extends AsyncFlatSpec + with AsyncIOSpec + with should.Matchers + with AsyncMockFactory + with OptionValues { + + EventStatus.all - GenerationNonRecoverableFailure - TransformationNonRecoverableFailure foreach { status => + it should show"return Healthy if the latest commit is found and it's in $status status" in { + + val project = projectSlugs.generateOne + + givenELClient(project, + returning = Result.success(List(eventInfos(project).generateOne.copy(status = status))).pure[IO] + ) + + finder.checkHealthy(project).asserting(_ shouldBe Healthy) + } + } + + GenerationNonRecoverableFailure :: TransformationNonRecoverableFailure :: Nil foreach { status => + it should show"return Unhealthy if the latest commit is found and it's in $status status" in { + + val project = projectSlugs.generateOne + + givenELClient(project, + returning = Result.success(List(eventInfos(project).generateOne.copy(status = status))).pure[IO] + ) + + finder.checkHealthy(project).asserting(_ shouldBe Unhealthy(status)) + } + } + + it should "return Healthy if the latest commit is not found" in { + + val project = projectSlugs.generateOne + + givenELClient(project, returning = Result.success(Nil).pure[IO]) + + finder.checkHealthy(project).asserting(_ shouldBe Healthy) + } + + it should "fail if call to the EL returns a failure" in { + + val project = projectSlugs.generateOne + + val failure = Result.failure(nonEmptyStrings().generateOne) + givenELClient(project, returning = failure.pure[IO]) + + finder.checkHealthy(project).assertThrowsError[Exception](_ shouldBe failure) + } + + it should "fail if call to the EL fails" in { + + val project = projectSlugs.generateOne + + val failure = exceptions.generateOne + givenELClient(project, returning = failure.raiseError[IO, Nothing]) + + finder.checkHealthy(project).assertThrowsError[Exception](_ shouldBe failure) + } + + private val elClient = mock[EventLogClient[IO]] + private lazy val finder = new ProvisioningStatusFinderImpl[IO](elClient) + + private def givenELClient(project: projects.Slug, returning: IO[EventLogClient.Result[List[EventInfo]]]) = + (elClient.getEvents _) + .expects(SearchCriteria.forProject(project).withPerPage(PerPage(1)).sortBy(SearchCriteria.Sort.EventDateDesc)) + .returning(returning) +}