diff --git a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/EventLog.scala b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/EventLog.scala index 8b38ad0baf..830ef208cc 100644 --- a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/EventLog.scala +++ b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/EventLog.scala @@ -60,15 +60,19 @@ object EventLog extends TypeSerializers { session.prepare(query).flatMap(_.stream(projectId, 32).compile.toList) } - def findSyncEvents(projectId: GitLabId)(implicit ioRuntime: IORuntime): List[CategoryName] = execute { session => - val query: Query[projects.GitLabId, CategoryName] = sql""" + def findSyncEventsIO(projectId: GitLabId): IO[List[CategoryName]] = + sessionResource.flatMap(_.session).use { session => + val query: Query[projects.GitLabId, CategoryName] = sql""" SELECT category_name FROM subscription_category_sync_time WHERE project_id = $projectIdEncoder""" - .query(varchar) - .map(category => CategoryName(category)) - session.prepare(query).flatMap(_.stream(projectId, 32).compile.toList) - } + .query(varchar) + .map(category => CategoryName(category)) + session.prepare(query).flatMap(_.stream(projectId, 32).compile.toList) + } + + def findSyncEvents(projectId: GitLabId)(implicit ioRuntime: IORuntime): List[CategoryName] = + findSyncEventsIO(projectId).unsafeRunSync() def forceCategoryEventTriggering(categoryName: CategoryName, projectId: projects.GitLabId)(implicit ioRuntime: IORuntime diff --git a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/TriplesStore.scala b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/TriplesStore.scala index 9933806c64..1b8cc80b2a 100644 --- a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/TriplesStore.scala +++ b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/db/TriplesStore.scala @@ -22,6 +22,8 @@ import cats.effect.{IO, Resource, Temporal} import cats.{Applicative, Monad} import eu.timepit.refined.auto._ import io.renku.db.DBConfigProvider +import io.renku.graph.model.{RenkuUrl, projects} +import io.renku.projectauth.{ProjectAuthData, QueryFilter} import io.renku.triplesgenerator.TgLockDB.SessionResource import io.renku.triplesgenerator.{TgLockDB, TgLockDbConfigProvider} import io.renku.triplesstore._ @@ -54,4 +56,10 @@ object TriplesStore extends InMemoryJena with ProjectsDataset with MigrationsDat private def waitForReadiness(implicit logger: Logger[IO]): IO[Unit] = Monad[IO].whileM_(IO(!isRunning))(logger.info("Waiting for TS") >> (Temporal[IO] sleep (500 millis))) + def findProjectAuth( + slug: projects.Slug + )(implicit renkuUrl: RenkuUrl, sqtr: SparqlQueryTimeRecorder[IO], L: Logger[IO]): IO[Option[ProjectAuthData]] = + ProjectSparqlClient[IO](projectsDSConnectionInfo) + .map(_.asProjectAuthService) + .use(_.getAll(QueryFilter.all.withSlug(slug)).compile.last) } diff --git a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/flows/TSProvisioning.scala b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/flows/TSProvisioning.scala index 66ab6a6ca2..02c96a1ee9 100644 --- a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/flows/TSProvisioning.scala +++ b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/flows/TSProvisioning.scala @@ -16,31 +16,35 @@ * limitations under the License. */ -package io.renku.graph.acceptancetests.flows +package io.renku.graph.acceptancetests +package flows import cats.data.NonEmptyList import cats.effect.IO import cats.effect.unsafe.IORuntime +import cats.syntax.all._ import fs2.Stream +import io.renku.eventlog.events.producers.membersync.{categoryName => memberSyncCategory} +import io.renku.eventlog.events.producers.minprojectinfo.{categoryName => minProjectInfoCategory} import io.renku.events.CategoryName import io.renku.graph.acceptancetests.data -import io.renku.graph.acceptancetests.db.EventLog +import io.renku.graph.acceptancetests.db.{EventLog, TriplesStore} import io.renku.graph.acceptancetests.testing.AcceptanceTestPatience -import io.renku.graph.acceptancetests.tooling.EventLogClient.ProjectEvent -import io.renku.graph.acceptancetests.tooling.{AcceptanceSpec, ApplicationServices, ModelImplicits} import io.renku.graph.model.events.{CommitId, EventId, EventStatus, EventStatusProgress} import io.renku.graph.model.projects import io.renku.http.client.AccessToken +import io.renku.logging.TestSparqlQueryTimeRecorder import io.renku.testtools.IOSpec +import io.renku.triplesstore.SparqlQueryTimeRecorder import io.renku.webhookservice.model.HookToken import org.http4s.Status._ import org.scalatest.concurrent.Eventually import org.scalatest.matchers.should import org.scalatest.{Assertion, EitherValues} import org.typelevel.log4cats.Logger +import tooling.EventLogClient.ProjectEvent +import tooling.{AcceptanceSpec, ApplicationServices, ModelImplicits} -import java.lang.Thread.sleep -import scala.annotation.tailrec import scala.concurrent.duration._ trait TSProvisioning @@ -77,6 +81,8 @@ trait TSProvisioning // commitId is the eventId val condition = commitIds.map(e => EventId(e.value)).toList.map(_ -> EventStatus.TriplesStore) waitForAllEvents(project.id, condition: _*) + waitForSyncEvents(project.id, memberSyncCategory) + waitForProjectAuthData(project.slug) } private def projectEvents(projectId: projects.GitLabId): Stream[IO, List[ProjectEvent]] = { @@ -119,24 +125,57 @@ trait TSProvisioning lastValue.forall(_ == EventStatusProgress.Stage.Final) shouldBe true } - def `check hook cannot be found`(projectId: projects.GitLabId, accessToken: AccessToken): Assertion = eventually { - webhookServiceClient.`GET projects/:id/events/status`(projectId, accessToken).status shouldBe NotFound + def getSyncEvents(projectId: projects.GitLabId) = { + val getSyncEvents = EventLog.findSyncEventsIO(projectId) + + val waitTimes = Stream.iterate(1d)(_ * 1.5).map(_.seconds).covary[IO].evalMap(IO.sleep) + Stream + .repeatEval(getSyncEvents) + .zip(waitTimes) + .map(_._1) } - def `wait for the Fast Tract event`(projectId: projects.GitLabId)(implicit ioRuntime: IORuntime): Unit = eventually { + def waitForSyncEvents(projectId: projects.GitLabId, category1: CategoryName, categoryN: CategoryName*) = { + val expected = categoryN.toSet + category1 - val sleepTime = 1 second + val tries = + getSyncEvents(projectId) + .evalTap(l => Logger[IO].info(s"Sync events for project $projectId: ${l.mkString(", ")}")) + .takeThrough(evs => expected.intersect(evs.toSet) != expected) + .take(13) - @tailrec - def checkIfWasSent(categoryName: CategoryName, attempt: Int = 1): Unit = { - if (attempt > 20) fail(s"'$categoryName' event wasn't sent after ${(sleepTime * attempt).toSeconds}") + val lastValue = tries.compile.lastOrError.unsafeRunSync() + expected.intersect(lastValue.toSet) shouldBe expected + } - if (!EventLog.findSyncEvents(projectId).contains(categoryName)) { - sleep(sleepTime.toMillis) - checkIfWasSent(categoryName) - } - } + def getProjectAuthData(slug: projects.Slug) = { + implicit val sqtr: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder.createUnsafe + val waitTimes = Stream.iterate(1d)(_ * 1.5).map(_.seconds).covary[IO].evalMap(IO.sleep) - checkIfWasSent(CategoryName("ADD_MIN_PROJECT_INFO")) + Stream + .repeatEval(TriplesStore.findProjectAuth(slug)) + .zip(waitTimes) + .map(_._1) } + + def waitForProjectAuthData(slug: projects.Slug) = { + val tries = + getProjectAuthData(slug) + .evalTap { + case None => Logger[IO].info(show"auth data not ready for $slug") + case Some(authData) => Logger[IO].info(show"auth data ready $authData") + } + .takeThrough(_.isEmpty) + .take(15) + + val lastValue = tries.compile.lastOrError.unsafeRunSync() + lastValue.isDefined shouldBe true + } + + def `check hook cannot be found`(projectId: projects.GitLabId, accessToken: AccessToken): Assertion = eventually { + webhookServiceClient.`GET projects/:id/events/status`(projectId, accessToken).status shouldBe NotFound + } + + def `wait for the Fast Tract event`(projectId: projects.GitLabId) = + waitForSyncEvents(projectId, minProjectInfoCategory) } diff --git a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/knowledgegraph/DatasetsResourcesSpec.scala b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/knowledgegraph/DatasetsResourcesSpec.scala index 91a895f86d..572520bcc7 100644 --- a/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/knowledgegraph/DatasetsResourcesSpec.scala +++ b/acceptance-tests/src/test/scala/io/renku/graph/acceptancetests/knowledgegraph/DatasetsResourcesSpec.scala @@ -23,17 +23,17 @@ import eu.timepit.refined.auto._ import io.circe.Json import io.circe.literal._ import io.renku.generators.CommonGraphGenerators.authUsers -import io.renku.generators.Generators._ import io.renku.generators.Generators.Implicits._ +import io.renku.generators.Generators._ import io.renku.graph.acceptancetests.data import io.renku.graph.acceptancetests.data._ import io.renku.graph.acceptancetests.flows.TSProvisioning -import io.renku.graph.acceptancetests.tooling.{AcceptanceSpec, ApplicationServices} import io.renku.graph.acceptancetests.tooling.TestReadabilityTools._ -import io.renku.graph.model._ +import io.renku.graph.acceptancetests.tooling.{AcceptanceSpec, ApplicationServices} import io.renku.graph.model.EventsGenerators.commitIds -import io.renku.graph.model.testentities.{::~, creatorUsernameUpdaterInternal} +import io.renku.graph.model._ import io.renku.graph.model.testentities.generators.EntitiesGenerators._ +import io.renku.graph.model.testentities.{::~, creatorUsernameUpdaterInternal} import io.renku.http.client.UrlEncoder.urlEncode import io.renku.http.rest.Links.Rel import io.renku.http.server.EndpointTester._ @@ -57,23 +57,22 @@ class DatasetsResourcesSpec Feature("GET knowledge-graph/projects///datasets to find project's datasets") { - val (dataset1 -> dataset2 -> dataset2Modified, testProject) = - renkuProjectEntities(visibilityPublic, creatorGen = cliShapedPersons) - .modify(removeMembers()) - .addDataset(datasetEntities(provenanceInternal(cliShapedPersons))) - .addDatasetAndModification( - datasetEntities(provenanceInternal(cliShapedPersons)), - creatorGen = cliShapedPersons - ) - .generateOne - val creatorPerson = cliShapedPersons.generateOne - val project = - dataProjects(testProject) - .map(replaceCreatorFrom(creatorPerson, creator.id)) - .map(addMemberFrom(creatorPerson, creator.id) >>> addMemberWithId(user.id)) - .generateOne - Scenario("As a user I would like to find project's datasets by calling a REST endpoint") { + val (dataset1 -> dataset2 -> dataset2Modified, testProject) = + renkuProjectEntities(visibilityPublic, creatorGen = cliShapedPersons) + .modify(removeMembers()) + .addDataset(datasetEntities(provenanceInternal(cliShapedPersons))) + .addDatasetAndModification( + datasetEntities(provenanceInternal(cliShapedPersons)), + creatorGen = cliShapedPersons + ) + .generateOne + val creatorPerson = cliShapedPersons.generateOne + val project = + dataProjects(testProject) + .map(replaceCreatorFrom(creatorPerson, creator.id)) + .map(addMemberFrom(creatorPerson, creator.id) >>> addMemberWithId(user.id)) + .generateOne Given("some data in the Triples Store") gitLabStub.addAuthenticated(creator) diff --git a/build.sbt b/build.sbt index 6ece318eda..4c563ac8dd 100644 --- a/build.sbt +++ b/build.sbt @@ -108,7 +108,10 @@ lazy val graphCommons = project .in(file("graph-commons")) .withId("graph-commons") .settings(commonSettings) - .dependsOn(renkuModel % "compile->compile; test->test") + .dependsOn( + renkuModel % "compile->compile; test->test", + projectAuth % "compile->compile; test->test" + ) .enablePlugins(AutomateHeaderPlugin) lazy val eventLogApi = project diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/Authorizer.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/Authorizer.scala index b21f7c4458..976dbdab4c 100644 --- a/graph-commons/src/main/scala/io/renku/graph/http/server/security/Authorizer.scala +++ b/graph-commons/src/main/scala/io/renku/graph/http/server/security/Authorizer.scala @@ -40,7 +40,10 @@ object Authorizer { projectSlug: projects.Slug, allowedPersons: Set[persons.GitLabId] ) - trait SecurityRecordFinder[F[_], Key] extends ((Key, Option[AuthUser]) => F[List[SecurityRecord]]) + trait SecurityRecordFinder[F[_], Key] extends ((Key, Option[AuthUser]) => F[List[SecurityRecord]]) { + def asAuthorizer(implicit F: MonadThrow[F]): Authorizer[F, Key] = + Authorizer.of(this) + } final case class AuthContext[Key](maybeAuthUser: Option[AuthUser], key: Key, allowedProjects: Set[projects.Slug]) { def addAllowedProject(slug: projects.Slug): AuthContext[Key] = copy(allowedProjects = allowedProjects + slug) @@ -52,6 +55,9 @@ object Authorizer { AuthContext(None, key, allowedProjects) } + def of[F[_]: MonadThrow, K](securityRecordFinder: SecurityRecordFinder[F, K]): Authorizer[F, K] = + new AuthorizerImpl[F, K](securityRecordFinder) + def using[F[_]: Async: Logger, Key]( securityRecordsFinderFactory: F[SecurityRecordFinder[F, Key]] ): F[Authorizer[F, Key]] = securityRecordsFinderFactory.map(new AuthorizerImpl[F, Key](_)) diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinder.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinder.scala deleted file mode 100644 index 2df9cb1eab..0000000000 --- a/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinder.scala +++ /dev/null @@ -1,92 +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.graph.http.server.security - -import cats.effect.Async -import cats.syntax.all._ -import io.circe.{Decoder, DecodingFailure} -import io.renku.graph.http.server.security.Authorizer.{SecurityRecord, SecurityRecordFinder} -import io.renku.graph.model.{datasets, projects, GraphClass} -import io.renku.graph.model.entities.Person -import io.renku.graph.model.persons.GitLabId -import io.renku.graph.model.projects.Visibility -import io.renku.http.server.security.model.AuthUser -import io.renku.triplesstore._ -import io.renku.triplesstore.ResultsDecoder._ -import io.renku.triplesstore.SparqlQuery.Prefixes -import org.typelevel.log4cats.Logger - -object DatasetIdRecordsFinder { - def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]: F[SecurityRecordFinder[F, datasets.Identifier]] = - ProjectsConnectionConfig[F]().map(new DatasetIdRecordsFinderImpl(_)) -} - -private class DatasetIdRecordsFinderImpl[F[_]: Async: Logger: SparqlQueryTimeRecorder]( - storeConfig: ProjectsConnectionConfig -) extends TSClientImpl(storeConfig) - with SecurityRecordFinder[F, datasets.Identifier] { - - override def apply(id: datasets.Identifier, maybeAuthUser: Option[AuthUser]): F[List[SecurityRecord]] = - queryExpecting[List[SecurityRecord]](selectQuery = query(id))(recordsDecoder) - - import eu.timepit.refined.auto._ - import io.renku.graph.model.Schemas._ - import io.renku.triplesstore.client.syntax._ - - private lazy val rowsSeparator = '\u0000' - - private def query(id: datasets.Identifier) = SparqlQuery.of( - name = "authorise - dataset id", - Prefixes of (renku -> "renku", schema -> "schema"), - s"""|SELECT DISTINCT ?projectId ?slug ?visibility (GROUP_CONCAT(?maybeMemberGitLabId; separator='$rowsSeparator') AS ?memberGitLabIds) - |WHERE { - | GRAPH ?projectGraph { - | ?projectId a schema:Project; - | renku:hasDataset/schema:identifier ${id.asObject.asSparql.sparql}; - | renku:projectPath ?slug; - | renku:projectVisibility ?visibility - | OPTIONAL { - | ?projectId schema:member ?memberId. - | GRAPH ${GraphClass.Persons.id.sparql} { - | ?memberId schema:sameAs ?sameAsId. - | ?sameAsId schema:additionalType ${Person.gitLabSameAsAdditionalType.asTripleObject.asSparql.sparql}; - | schema:identifier ?maybeMemberGitLabId - | } - | } - | } - |} - |GROUP BY ?projectId ?slug ?visibility - |""".stripMargin - ) - - private lazy val recordsDecoder: Decoder[List[SecurityRecord]] = ResultsDecoder[List, SecurityRecord] { - implicit cur => - import Decoder._ - import io.renku.tinytypes.json.TinyTypeDecoders._ - - for { - visibility <- extract[Visibility]("visibility") - slug <- extract[projects.Slug]("slug") - userIds <- extract[Option[String]]("memberGitLabIds") - .map(_.map(_.split(rowsSeparator).toList).getOrElse(List.empty)) - .flatMap(_.map(GitLabId.parse).sequence.leftMap(ex => DecodingFailure(ex.getMessage, Nil))) - .map(_.toSet) - } yield SecurityRecord(visibility, slug, userIds) - } -} diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinder.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinder.scala deleted file mode 100644 index 84c63a87b1..0000000000 --- a/graph-commons/src/main/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinder.scala +++ /dev/null @@ -1,96 +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.graph.http.server.security - -import cats.effect.Async -import cats.syntax.all._ -import cats.MonadThrow -import io.renku.graph.http.server.security.Authorizer.{SecurityRecord, SecurityRecordFinder} -import io.renku.graph.model.{GraphClass, datasets} -import io.renku.graph.model.entities.Person -import io.renku.http.server.security.model.AuthUser -import io.renku.jsonld.EntityId -import io.renku.triplesstore.{ProjectsConnectionConfig, SparqlQueryTimeRecorder, TSClient} -import org.typelevel.log4cats.Logger - -object DatasetSameAsRecordsFinder { - def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]: F[SecurityRecordFinder[F, datasets.SameAs]] = - ProjectsConnectionConfig[F]().map(TSClient[F](_)).map(new DatasetSameAsRecordsFinderImpl(_)) -} - -private class DatasetSameAsRecordsFinderImpl[F[_]: MonadThrow](tsClient: TSClient[F]) - extends SecurityRecordFinder[F, datasets.SameAs] { - - override def apply(sameAs: datasets.SameAs, maybeAuthUser: Option[AuthUser]): F[List[SecurityRecord]] = - tsClient.queryExpecting[List[SecurityRecord]](selectQuery = query(sameAs)) - - import eu.timepit.refined.auto._ - import io.circe.{Decoder, DecodingFailure} - import io.renku.graph.model.{persons, projects} - import io.renku.graph.model.Schemas._ - import io.renku.triplesstore.{ResultsDecoder, SparqlQuery} - import io.renku.triplesstore.client.syntax._ - import io.renku.triplesstore.SparqlQuery.Prefixes - import ResultsDecoder._ - - private lazy val rowsSeparator = '\u0000' - - private def query(sameAs: datasets.SameAs) = SparqlQuery.of( - name = "authorise - dataset sameAs", - Prefixes of (renku -> "renku", schema -> "schema"), - s"""|SELECT DISTINCT ?projectId ?slug ?visibility (GROUP_CONCAT(?maybeMemberGitLabId; separator='$rowsSeparator') AS ?memberGitLabIds) - |WHERE { - | GRAPH ?projectId { - | ?projectId a schema:Project; - | renku:hasDataset/renku:topmostSameAs ${EntityId.of(sameAs.value).sparql}; - | renku:projectPath ?slug; - | renku:projectVisibility ?visibility. - | OPTIONAL { - | ?projectId schema:member ?memberId. - | GRAPH ${GraphClass.Persons.id.sparql} { - | ?memberId schema:sameAs ?sameAsId. - | ?sameAsId schema:additionalType ${Person.gitLabSameAsAdditionalType.asTripleObject.asSparql.sparql}; - | schema:identifier ?maybeMemberGitLabId - | } - | } - | } - |} - |GROUP BY ?projectId ?slug ?visibility - |""".stripMargin - ) - - private implicit lazy val recordsDecoder: Decoder[List[SecurityRecord]] = ResultsDecoder[List, SecurityRecord] { - implicit cur => - import Decoder._ - import io.renku.tinytypes.json.TinyTypeDecoders._ - - val toSetOfGitLabIds: Option[String] => Result[Set[persons.GitLabId]] = - _.map(_.split(rowsSeparator).toList) - .getOrElse(List.empty) - .map(persons.GitLabId.parse) - .sequence - .bimap(ex => DecodingFailure(ex.getMessage, Nil), _.toSet) - - for { - visibility <- extract[projects.Visibility]("visibility") - slug <- extract[projects.Slug]("slug") - userIds <- extract[Option[String]]("memberGitLabIds") >>= toSetOfGitLabIds - } yield SecurityRecord(visibility, slug, userIds) - } -} diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinder.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinder.scala new file mode 100644 index 0000000000..1282a0de6e --- /dev/null +++ b/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinder.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.graph.http.server.security + +import cats.effect.kernel.Resource +import cats.effect.{Async, Sync} +import cats.syntax.all._ +import eu.timepit.refined.auto._ +import fs2.io.net.Network +import io.renku.graph.http.server.security.Authorizer.SecurityRecordFinder +import io.renku.graph.model.{RenkuUrl, projects} +import io.renku.http.server.security.model +import io.renku.projectauth.{ProjectAuthService, QueryFilter} +import io.renku.triplesstore.{ProjectSparqlClient, ProjectsConnectionConfig, SparqlQueryTimeRecorder} +import org.typelevel.log4cats.Logger + +trait ProjectAuthRecordsFinder[F[_]] extends SecurityRecordFinder[F, projects.Slug] + +object ProjectAuthRecordsFinder { + + def create[F[_]: Async: Network: Logger: SparqlQueryTimeRecorder]( + renkuUrl: RenkuUrl, + connCfg: ProjectsConnectionConfig + ): Resource[F, ProjectAuthRecordsFinder[F]] = + ProjectSparqlClient(connCfg).map(c => apply[F](c, renkuUrl)) + + def apply[F[_]: Sync: Logger: SparqlQueryTimeRecorder]( + projectSparqlClient: ProjectSparqlClient[F], + renkuUrl: RenkuUrl + ): ProjectAuthRecordsFinder[F] = + new Impl[F](projectSparqlClient.asProjectAuthService(renkuUrl)) + + def apply[F[_]: Sync: SparqlQueryTimeRecorder: Logger]( + projectAuthService: ProjectAuthService[F] + ): ProjectAuthRecordsFinder[F] = + new Impl[F](projectAuthService) + + private final class Impl[F[_]: Sync: SparqlQueryTimeRecorder: Logger]( + projectAuthService: ProjectAuthService[F] + ) extends ProjectAuthRecordsFinder[F] { + private[this] val timeRecorder = SparqlQueryTimeRecorder[F] + override def apply(slug: projects.Slug, user: Option[model.AuthUser]): F[List[Authorizer.SecurityRecord]] = { + val filter = QueryFilter.all.withSlug(slug) + val data = projectAuthService.getAll(filter).compile.toList + timeRecorder.reportTime("project-security-records")(data).map { list => + list.map(d => + Authorizer.SecurityRecord( + visibility = d.visibility, + projectSlug = d.slug, + allowedPersons = d.members.map(_.gitLabId) + ) + ) + } + } + } +} diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinder.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinder.scala index fdaae2b0c4..de2f02c6dc 100644 --- a/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinder.scala +++ b/graph-commons/src/main/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinder.scala @@ -22,20 +22,23 @@ import cats.effect.Async import cats.syntax.all._ import cats.{MonadThrow, Parallel} import io.renku.graph.http.server.security.Authorizer.SecurityRecordFinder -import io.renku.graph.model.projects +import io.renku.graph.model.{RenkuUrl, projects} import io.renku.http.client.GitLabClient import io.renku.http.server.security.model.AuthUser -import io.renku.triplesstore.SparqlQueryTimeRecorder +import io.renku.triplesstore.{ProjectSparqlClient, SparqlQueryTimeRecorder} import org.typelevel.log4cats.Logger object ProjectSlugRecordsFinder { - def apply[F[_]: Async: Parallel: Logger: SparqlQueryTimeRecorder: GitLabClient] - : F[SecurityRecordFinder[F, projects.Slug]] = - (TSSlugRecordsFinder[F] -> GLSlugRecordsFinder[F]) + + def apply[F[_]: Async: Parallel: Logger: SparqlQueryTimeRecorder: GitLabClient]( + projectSparqlClient: ProjectSparqlClient[F], + renkuUrl: RenkuUrl + ): F[SecurityRecordFinder[F, projects.Slug]] = + (ProjectAuthRecordsFinder[F](projectSparqlClient, renkuUrl).pure[F] -> GLSlugRecordsFinder[F]) .mapN(new ProjectSlugRecordsFinderImpl[F](_, _)) } -private class ProjectSlugRecordsFinderImpl[F[_]: MonadThrow](tsSlugRecordsFinder: TSSlugRecordsFinder[F], +private class ProjectSlugRecordsFinderImpl[F[_]: MonadThrow](tsSlugRecordsFinder: ProjectAuthRecordsFinder[F], glSlugRecordsFinder: GLSlugRecordsFinder[F] ) extends SecurityRecordFinder[F, projects.Slug] { diff --git a/graph-commons/src/main/scala/io/renku/graph/http/server/security/TSSlugRecordsFinder.scala b/graph-commons/src/main/scala/io/renku/graph/http/server/security/TSSlugRecordsFinder.scala deleted file mode 100644 index 0022f6aca5..0000000000 --- a/graph-commons/src/main/scala/io/renku/graph/http/server/security/TSSlugRecordsFinder.scala +++ /dev/null @@ -1,88 +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.graph.http.server.security - -import cats.effect.Async -import cats.syntax.all._ -import io.circe.{Decoder, DecodingFailure} -import io.renku.graph.config.RenkuUrlLoader -import io.renku.graph.http.server.security.Authorizer.{SecurityRecord, SecurityRecordFinder} -import io.renku.graph.model.{GraphClass, RenkuUrl, projects} -import io.renku.graph.model.entities.Person -import io.renku.graph.model.persons.GitLabId -import io.renku.graph.model.projects.{ResourceId, Visibility} -import io.renku.graph.model.projects.Visibility._ -import io.renku.graph.model.views.RdfResource -import io.renku.http.server.security.model.AuthUser -import io.renku.triplesstore._ -import io.renku.triplesstore.ResultsDecoder._ -import io.renku.triplesstore.SparqlQuery.Prefixes -import org.typelevel.log4cats.Logger - -trait TSSlugRecordsFinder[F[_]] extends SecurityRecordFinder[F, projects.Slug] - -object TSSlugRecordsFinder { - def apply[F[_]: Async: Logger: SparqlQueryTimeRecorder]: F[TSSlugRecordsFinder[F]] = for { - implicit0(renkuUrl: RenkuUrl) <- RenkuUrlLoader[F]() - storeConfig <- ProjectsConnectionConfig[F]() - } yield new TSSlugRecordsFinderImpl[F](storeConfig) -} - -private class TSSlugRecordsFinderImpl[F[_]: Async: Logger: SparqlQueryTimeRecorder]( - storeConfig: ProjectsConnectionConfig -)(implicit renkuUrl: RenkuUrl) - extends TSClientImpl(storeConfig) - with TSSlugRecordsFinder[F] { - - override def apply(slug: projects.Slug, maybeAuthUser: Option[AuthUser]): F[List[SecurityRecord]] = - queryExpecting[List[SecurityRecord]](query(ResourceId(slug)))(recordsDecoder(slug)) - - import eu.timepit.refined.auto._ - import io.renku.graph.model.Schemas._ - - private def query(resourceId: projects.ResourceId) = SparqlQuery.of( - name = "authorise - project slug", - Prefixes of (renku -> "renku", schema -> "schema"), - s"""|SELECT DISTINCT ?projectId ?visibility (GROUP_CONCAT(?maybeMemberGitLabId; separator=',') AS ?memberGitLabIds) - |FROM <${GraphClass.Persons.id}> - |FROM ${resourceId.showAs[RdfResource]} { - | BIND (${resourceId.showAs[RdfResource]} AS ?resourceId) - | ?projectId a schema:Project; - | renku:projectVisibility ?visibility. - | OPTIONAL { - | ?projectId schema:member/schema:sameAs ?sameAsId. - | ?sameAsId schema:additionalType '${Person.gitLabSameAsAdditionalType}'; - | schema:identifier ?maybeMemberGitLabId. - | } - |} - |GROUP BY ?projectId ?visibility - |""".stripMargin - ) - - private def recordsDecoder(slug: projects.Slug): Decoder[List[SecurityRecord]] = - ResultsDecoder[List, SecurityRecord] { implicit cur => - for { - visibility <- extract[Visibility]("visibility") - maybeUserId <- extract[Option[String]]("memberGitLabIds") - .map(_.map(_.split(",").toList).getOrElse(List.empty)) - .flatMap(_.map(GitLabId.parse).sequence.leftMap(ex => DecodingFailure(ex.getMessage, Nil))) - .map(_.toSet) - } yield SecurityRecord(visibility, slug, maybeUserId) - } -} diff --git a/graph-commons/src/main/scala/io/renku/triplesstore/ProjectSparqlClient.scala b/graph-commons/src/main/scala/io/renku/triplesstore/ProjectSparqlClient.scala index 2f4c86e079..33e41e41bb 100644 --- a/graph-commons/src/main/scala/io/renku/triplesstore/ProjectSparqlClient.scala +++ b/graph-commons/src/main/scala/io/renku/triplesstore/ProjectSparqlClient.scala @@ -18,26 +18,33 @@ package io.renku.triplesstore -import cats.Monad +import cats.MonadThrow import cats.effect._ import cats.syntax.all._ import eu.timepit.refined.api.Refined -import eu.timepit.refined.collection.NonEmpty import eu.timepit.refined.auto._ +import eu.timepit.refined.collection.NonEmpty import fs2.io.net.Network +import io.renku.graph.model.RenkuUrl import io.renku.jsonld.JsonLD +import io.renku.projectauth.ProjectAuthService import io.renku.triplesstore.client.http.{Retry, SparqlClient, SparqlQuery, SparqlUpdate} import org.typelevel.log4cats.Logger /** SparQL client fixed to the `projects` dataset. */ -trait ProjectSparqlClient[F[_]] extends SparqlClient[F] +trait ProjectSparqlClient[F[_]] extends SparqlClient[F] { + def asProjectAuthService(implicit renkuUrl: RenkuUrl): ProjectAuthService[F] +} object ProjectSparqlClient { - def apply[F[_]: Monad: Logger: SparqlQueryTimeRecorder](c: SparqlClient[F]): ProjectSparqlClient[F] = - new ProjectSparqlClient[F] { + def apply[F[_]: MonadThrow: Logger: SparqlQueryTimeRecorder](c: SparqlClient[F]): ProjectSparqlClient[F] = + new ProjectSparqlClient[F] { self => private[this] val rec = SparqlQueryTimeRecorder[F].instance + def asProjectAuthService(implicit renkuUrl: RenkuUrl): ProjectAuthService[F] = + ProjectAuthService(self, renkuUrl) + override def update(request: SparqlUpdate) = { val label = histogramLabel(request) val work = c.update(request) diff --git a/graph-commons/src/main/scala/io/renku/triplesstore/SparqlQueryTimeRecorder.scala b/graph-commons/src/main/scala/io/renku/triplesstore/SparqlQueryTimeRecorder.scala index 2963feec53..b47905d4fb 100644 --- a/graph-commons/src/main/scala/io/renku/triplesstore/SparqlQueryTimeRecorder.scala +++ b/graph-commons/src/main/scala/io/renku/triplesstore/SparqlQueryTimeRecorder.scala @@ -18,16 +18,22 @@ package io.renku.triplesstore +import cats.Monad import cats.effect.Sync import cats.syntax.all._ +import eu.timepit.refined.api.Refined import eu.timepit.refined.auto._ +import eu.timepit.refined.collection.NonEmpty import io.renku.logging.ExecutionTimeRecorder import io.renku.metrics.LabeledHistogramImpl import org.typelevel.log4cats.Logger import scala.concurrent.duration._ -class SparqlQueryTimeRecorder[F[_]](val instance: ExecutionTimeRecorder[F]) +final class SparqlQueryTimeRecorder[F[_]](val instance: ExecutionTimeRecorder[F]) { + def reportTime[A](label: String Refined NonEmpty)(work: F[A])(implicit L: Logger[F], M: Monad[F]): F[A] = + instance.measureExecutionTime(work, label.some).flatMap(instance.logExecutionTime(s"Executed $label")) +} object SparqlQueryTimeRecorder { diff --git a/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinderSpec.scala b/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinderSpec.scala deleted file mode 100644 index 4f0eef67dd..0000000000 --- a/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetIdRecordsFinderSpec.scala +++ /dev/null @@ -1,100 +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.graph.http.server.security - -import cats.effect.IO -import io.renku.generators.CommonGraphGenerators.authUsers -import io.renku.generators.Generators.Implicits._ -import io.renku.graph.http.server.security.Authorizer.SecurityRecord -import io.renku.graph.model.testentities._ -import io.renku.interpreters.TestLogger -import io.renku.logging.TestSparqlQueryTimeRecorder -import io.renku.testtools.IOSpec -import io.renku.triplesstore.{InMemoryJenaForSpec, ProjectsDataset, SparqlQueryTimeRecorder} -import org.scalatest.matchers.should -import org.scalatest.wordspec.AnyWordSpec - -class DatasetIdRecordsFinderSpec - extends AnyWordSpec - with IOSpec - with InMemoryJenaForSpec - with ProjectsDataset - with should.Matchers { - - "apply" should { - - "return SecurityRecord with project visibility, slug and all project members" in new TestCase { - - val (dataset, project) = - renkuProjectEntities(anyVisibility).addDataset(datasetEntities(provenanceNonModified)).generateOne - - upload(to = projectsDataset, project) - - recordsFinder(dataset.identification.identifier, maybeAuthUser).unsafeRunSync() shouldBe List( - SecurityRecord(project.visibility, project.slug, project.members.flatMap(_.maybeGitLabId)) - ) - } - - "return SecurityRecord with project visibility, slug and no member if project has none" in new TestCase { - - val (dataset, project) = - renkuProjectEntities(anyVisibility) - .modify(removeMembers()) - .addDataset(datasetEntities(provenanceNonModified)) - .generateOne - - upload(to = projectsDataset, project) - - recordsFinder(dataset.identification.identifier, maybeAuthUser).unsafeRunSync() shouldBe List( - SecurityRecord(project.visibility, project.slug, Set.empty) - ) - } - - "return SecurityRecords with projects visibilities, slugs and members in case of forks" in new TestCase { - - val (dataset, (parentProject, project)) = - renkuProjectEntities(anyVisibility) - .modify(removeMembers()) - .addDataset(datasetEntities(provenanceNonModified)) - .forkOnce() - .generateOne - - upload(to = projectsDataset, parentProject, project) - - recordsFinder(dataset.identification.identifier, maybeAuthUser) - .unsafeRunSync() should contain theSameElementsAs List( - SecurityRecord(parentProject.visibility, parentProject.slug, parentProject.members.flatMap(_.maybeGitLabId)), - SecurityRecord(project.visibility, project.slug, project.members.flatMap(_.maybeGitLabId)) - ) - } - - "nothing if there's no project with the given slug" in new TestCase { - recordsFinder(datasetIdentifiers.generateOne, maybeAuthUser).unsafeRunSync() shouldBe Nil - } - } - - private trait TestCase { - - val maybeAuthUser = authUsers.generateOption - - private implicit val logger: TestLogger[IO] = TestLogger[IO]() - private implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() - val recordsFinder = new DatasetIdRecordsFinderImpl[IO](projectsDSConnectionInfo) - } -} diff --git a/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinderSpec.scala b/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinderSpec.scala deleted file mode 100644 index 79f83d1bfa..0000000000 --- a/graph-commons/src/test/scala/io/renku/graph/http/server/security/DatasetSameAsRecordsFinderSpec.scala +++ /dev/null @@ -1,102 +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.graph.http.server.security - -import cats.effect.IO -import io.renku.generators.CommonGraphGenerators.authUsers -import io.renku.generators.Generators.Implicits._ -import io.renku.graph.http.server.security.Authorizer.SecurityRecord -import io.renku.graph.model -import io.renku.graph.model.testentities._ -import io.renku.interpreters.TestLogger -import io.renku.logging.TestSparqlQueryTimeRecorder -import io.renku.testtools.IOSpec -import io.renku.triplesstore._ -import org.scalatest.matchers.should -import org.scalatest.wordspec.AnyWordSpec - -class DatasetSameAsRecordsFinderSpec - extends AnyWordSpec - with should.Matchers - with IOSpec - with InMemoryJenaForSpec - with ProjectsDataset { - - "apply" should { - - "return SecurityRecord with project visibility, slug and all project members" in new TestCase { - - val (dataset, project) = - anyRenkuProjectEntities.addDataset(datasetEntities(provenanceNonModified)).generateOne - - upload(to = projectsDataset, project) - - recordsFinder(model.datasets.SameAs.ofUnsafe(dataset.provenance.topmostSameAs.value), maybeAuthUser) - .unsafeRunSync() shouldBe List( - SecurityRecord(project.visibility, project.slug, project.members.flatMap(_.maybeGitLabId)) - ) - } - - "return SecurityRecord with project visibility, slug and no member if project is none" in new TestCase { - - val (dataset, project) = - renkuProjectEntities(anyVisibility) - .modify(removeMembers()) - .addDataset(datasetEntities(provenanceNonModified)) - .generateOne - - upload(to = projectsDataset, project) - - recordsFinder(model.datasets.SameAs.ofUnsafe(dataset.provenance.topmostSameAs.value), maybeAuthUser) - .unsafeRunSync() shouldBe List(SecurityRecord(project.visibility, project.slug, Set.empty)) - } - - "return SecurityRecords with projects visibilities, slugs and members in case of forks" in new TestCase { - - val (dataset, (parentProject, project)) = - renkuProjectEntities(anyVisibility) - .modify(removeMembers()) - .addDataset(datasetEntities(provenanceNonModified)) - .forkOnce() - .generateOne - - upload(to = projectsDataset, parentProject, project) - - recordsFinder(model.datasets.SameAs.ofUnsafe(dataset.provenance.topmostSameAs.value), maybeAuthUser) - .unsafeRunSync() should contain theSameElementsAs List( - SecurityRecord(parentProject.visibility, parentProject.slug, parentProject.members.flatMap(_.maybeGitLabId)), - SecurityRecord(project.visibility, project.slug, project.members.flatMap(_.maybeGitLabId)) - ) - } - - "nothing if there's no project with the given slug" in new TestCase { - recordsFinder(datasetSameAs.generateOne, maybeAuthUser).unsafeRunSync() shouldBe Nil - } - } - - private trait TestCase { - - val maybeAuthUser = authUsers.generateOption - - private implicit val logger: TestLogger[IO] = TestLogger[IO]() - private implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() - private val tsClient = TSClient[IO](projectsDSConnectionInfo) - val recordsFinder = new DatasetSameAsRecordsFinderImpl[IO](tsClient) - } -} diff --git a/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinderSpec.scala b/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinderSpec.scala new file mode 100644 index 0000000000..81572b2fd1 --- /dev/null +++ b/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectAuthRecordsFinderSpec.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.graph.http.server.security + +import cats.effect._ +import io.renku.graph.http.server.security.Authorizer.SecurityRecord +import io.renku.graph.model.RenkuUrl +import io.renku.graph.model.projects.Slug +import io.renku.interpreters.TestLogger +import io.renku.logging.TestSparqlQueryTimeRecorder +import io.renku.projectauth.{Generators, ProjectAuthServiceSupport} +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesstore.SparqlQueryTimeRecorder +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.typelevel.log4cats.Logger + +class ProjectAuthRecordsFinderSpec + extends AsyncFlatSpec + with CustomAsyncIOSpec + with ProjectAuthServiceSupport + with should.Matchers { + implicit val logger: Logger[IO] = TestLogger[IO]() + implicit val renkuUrl: RenkuUrl = RenkuUrl("http://u.rl") + implicit val qtr: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder.createUnsafe + + it should "find visibility and project members" in { + val data = Generators.projectAuthData.stream + withProjectAuthServiceData(data.take(3)).use { case (s, d) => + val finder = ProjectAuthRecordsFinder(s) + + val r = d.head + val expect = SecurityRecord(r.visibility, r.slug, r.members.map(_.gitLabId)) + + finder(r.slug, None).asserting(list => list shouldBe List(expect)) + } + } + + it should "find project without members" in { + val data = Generators.projectAuthData.withMembers().stream + withProjectAuthServiceData(data.take(1)).use { case (s, d) => + val finder = ProjectAuthRecordsFinder(s) + val r = d.head + val expect = SecurityRecord(r.visibility, r.slug, Set.empty) + finder(r.slug, None).asserting(list => list shouldBe List(expect)) + } + } + + it should "return empty when project is not found" in { + withProjectAuthService.use { s => + val finder = ProjectAuthRecordsFinder(s) + val slug = Slug("p/c") + finder(slug, None).asserting(_ shouldBe Nil) + } + } +} diff --git a/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinderSpec.scala b/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinderSpec.scala index bd589637ef..c7715c94f6 100644 --- a/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinderSpec.scala +++ b/graph-commons/src/test/scala/io/renku/graph/http/server/security/ProjectSlugRecordsFinderSpec.scala @@ -59,7 +59,7 @@ class ProjectSlugRecordsFinderSpec extends AnyWordSpec with should.Matchers with val projectSlug = projectSlugs.generateOne val maybeAuthUser = authUsers.generateOption - private val tsSlugRecordsFinder = mock[TSSlugRecordsFinder[Try]] + private val tsSlugRecordsFinder = mock[ProjectAuthRecordsFinder[Try]] private val glSlugRecordsFinder = mock[GLSlugRecordsFinder[Try]] val recordsFinder = new ProjectSlugRecordsFinderImpl[Try](tsSlugRecordsFinder, glSlugRecordsFinder) diff --git a/graph-commons/src/test/scala/io/renku/graph/http/server/security/TSSlugRecordsFinderSpec.scala b/graph-commons/src/test/scala/io/renku/graph/http/server/security/TSSlugRecordsFinderSpec.scala deleted file mode 100644 index e3d06ffa94..0000000000 --- a/graph-commons/src/test/scala/io/renku/graph/http/server/security/TSSlugRecordsFinderSpec.scala +++ /dev/null @@ -1,76 +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.graph.http.server.security - -import cats.effect.IO -import io.renku.generators.CommonGraphGenerators.authUsers -import io.renku.generators.Generators.Implicits._ -import io.renku.graph.http.server.security.Authorizer.SecurityRecord -import io.renku.graph.model.testentities.generators.EntitiesGenerators -import io.renku.interpreters.TestLogger -import io.renku.logging.TestSparqlQueryTimeRecorder -import io.renku.testtools.IOSpec -import io.renku.triplesstore.{InMemoryJenaForSpec, ProjectsDataset, SparqlQueryTimeRecorder} -import org.scalatest.matchers.should -import org.scalatest.wordspec.AnyWordSpec - -class TSSlugRecordsFinderSpec - extends AnyWordSpec - with IOSpec - with EntitiesGenerators - with InMemoryJenaForSpec - with ProjectsDataset - with should.Matchers { - - "apply" should { - - "return SecurityRecords with project visibility and all project members" in new TestCase { - val project = anyProjectEntities.generateOne - - upload(to = projectsDataset, project) - - recordsFinder(project.slug, maybeAuthUser).unsafeRunSync() shouldBe List( - SecurityRecord(project.visibility, project.slug, project.members.flatMap(_.maybeGitLabId)) - ) - } - - "return SecurityRecords with project visibility and no member is project has none" in new TestCase { - val project = renkuProjectEntities(anyVisibility).generateOne.copy(members = Set.empty) - - upload(to = projectsDataset, project) - - recordsFinder(project.slug, maybeAuthUser).unsafeRunSync() shouldBe List( - SecurityRecord(project.visibility, project.slug, Set.empty) - ) - } - - "nothing if there's no project with the given slug" in new TestCase { - recordsFinder(projectSlugs.generateOne, maybeAuthUser).unsafeRunSync() shouldBe Nil - } - } - - private trait TestCase { - - val maybeAuthUser = authUsers.generateOption - - private implicit val logger: TestLogger[IO] = TestLogger[IO]() - private implicit val timeRecorder: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder[IO].unsafeRunSync() - val recordsFinder = new TSSlugRecordsFinderImpl[IO](projectsDSConnectionInfo) - } -} diff --git a/graph-commons/src/test/scala/io/renku/logging/TestSparqlQueryTimeRecorder.scala b/graph-commons/src/test/scala/io/renku/logging/TestSparqlQueryTimeRecorder.scala index b664441028..914e3b012e 100644 --- a/graph-commons/src/test/scala/io/renku/logging/TestSparqlQueryTimeRecorder.scala +++ b/graph-commons/src/test/scala/io/renku/logging/TestSparqlQueryTimeRecorder.scala @@ -18,7 +18,8 @@ package io.renku.logging -import cats.effect.Sync +import cats.effect.unsafe.IORuntime +import cats.effect.{IO, Sync} import io.renku.metrics.{MetricsRegistry, TestMetricsRegistry} import io.renku.triplesstore.SparqlQueryTimeRecorder import org.typelevel.log4cats.Logger @@ -28,4 +29,7 @@ object TestSparqlQueryTimeRecorder { implicit val metricsRegistry: MetricsRegistry[F] = TestMetricsRegistry[F] SparqlQueryTimeRecorder.create[F]() } + + def createUnsafe(implicit logger: Logger[IO], IORuntime: IORuntime): SparqlQueryTimeRecorder[IO] = + apply[IO].unsafeRunSync() } diff --git a/graph-commons/src/test/scala/io/renku/triplesstore/ExternalJenaForSpec.scala b/graph-commons/src/test/scala/io/renku/triplesstore/ExternalJenaForSpec.scala index 83092856ca..1f752abe06 100644 --- a/graph-commons/src/test/scala/io/renku/triplesstore/ExternalJenaForSpec.scala +++ b/graph-commons/src/test/scala/io/renku/triplesstore/ExternalJenaForSpec.scala @@ -19,7 +19,6 @@ package io.renku.triplesstore import eu.timepit.refined.auto._ -import io.renku.testtools.IOSpec import io.renku.triplesstore.client.util.JenaRunMode import org.scalatest.Suite @@ -27,7 +26,7 @@ import org.scalatest.Suite * starting a container. */ trait ExternalJenaForSpec extends InMemoryJenaForSpec { - self: Suite with IOSpec => + self: Suite => /** Expect the external Jena instance to accept connections on the default port. */ override val jenaRunMode: JenaRunMode = JenaRunMode.Local(3030) diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/Microservice.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/Microservice.scala index 69fe42d23a..3c77657e1c 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/Microservice.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/Microservice.scala @@ -28,7 +28,7 @@ import io.renku.knowledgegraph.metrics.KGMetrics import io.renku.logging.ApplicationLogger import io.renku.metrics.MetricsRegistry import io.renku.microservices.{IOMicroservice, ResourceUse} -import io.renku.triplesstore.{ProjectsConnectionConfig, SparqlQueryTimeRecorder} +import io.renku.triplesstore.{ProjectSparqlClient, ProjectsConnectionConfig, SparqlQueryTimeRecorder} import org.http4s.server.Server import org.typelevel.log4cats.Logger @@ -36,20 +36,41 @@ object Microservice extends IOMicroservice { private implicit val logger: Logger[IO] = ApplicationLogger - override def run(args: List[String]): IO[ExitCode] = for { - implicit0(mr: MetricsRegistry[IO]) <- MetricsRegistry[IO]() - implicit0(sqtr: SparqlQueryTimeRecorder[IO]) <- SparqlQueryTimeRecorder.create[IO]() - projectConnConfig <- ProjectsConnectionConfig[IO]() - certificateLoader <- CertificateLoader[IO] - sentryInitializer <- SentryInitializer[IO] - kgMetrics <- KGMetrics[IO] - microserviceRoutes <- MicroserviceRoutes[IO](projectConnConfig) - termSignal <- SignallingRef.of[IO, Boolean](false) - exitCode <- microserviceRoutes.routes.use { routes => - val httpServer = HttpServer[IO](serverPort = port"9004", routes) - new MicroserviceRunner(certificateLoader, sentryInitializer, httpServer, kgMetrics).run(termSignal) - } - } yield exitCode + override def run(args: List[String]): IO[ExitCode] = + Setup.resource.use { setup => + implicit val sqtr: SparqlQueryTimeRecorder[IO] = setup.sparqlQueryTimeRecorder + implicit val mr: MetricsRegistry[IO] = setup.metricsRegistry + for { + certificateLoader <- CertificateLoader[IO] + sentryInitializer <- SentryInitializer[IO] + kgMetrics <- KGMetrics[IO] + microserviceRoutes <- MicroserviceRoutes[IO](setup.projectConnConfig, setup.projectSparqlClient) + termSignal <- SignallingRef.of[IO, Boolean](false) + exitCode <- microserviceRoutes.routes.use { routes => + val httpServer = HttpServer[IO](serverPort = port"9004", routes) + new MicroserviceRunner(certificateLoader, sentryInitializer, httpServer, kgMetrics).run( + termSignal + ) + } + } yield exitCode + } + + final case class Setup( + projectConnConfig: ProjectsConnectionConfig, + metricsRegistry: MetricsRegistry[IO], + sparqlQueryTimeRecorder: SparqlQueryTimeRecorder[IO], + projectSparqlClient: ProjectSparqlClient[IO] + ) + + object Setup { + val resource: Resource[IO, Setup] = for { + pcc <- Resource.eval(ProjectsConnectionConfig[IO]()) + + implicit0(mr: MetricsRegistry[IO]) <- Resource.eval(MetricsRegistry[IO]()) + implicit0(sqtr: SparqlQueryTimeRecorder[IO]) <- Resource.eval(SparqlQueryTimeRecorder.create[IO]()) + psc <- ProjectSparqlClient[IO](pcc) + } yield Setup(pcc, mr, sqtr, psc) + } } private class MicroserviceRunner( 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 0538fabc04..abda804b3a 100644 --- a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/MicroserviceRoutes.scala @@ -42,9 +42,10 @@ import io.renku.http.server.QueryParameterTools._ import io.renku.http.server.security.Authentication import io.renku.http.server.security.model.{AuthUser, MaybeAuthUser} import io.renku.http.server.version +import io.renku.knowledgegraph.datasets.{DatasetIdRecordsFinder, DatasetSameAsRecordsFinder} import io.renku.knowledgegraph.datasets.details.RequestedDataset import io.renku.metrics.{MetricsRegistry, RoutesMetrics} -import io.renku.triplesstore.{ProjectsConnectionConfig, SparqlQueryTimeRecorder} +import io.renku.triplesstore.{ProjectSparqlClient, ProjectsConnectionConfig, SparqlQueryTimeRecorder} import org.http4s.dsl.Http4sDsl import org.http4s.server.AuthMiddleware import org.http4s.{AuthedRoutes, ParseFailure, Request, Response, Status, Uri} @@ -339,7 +340,8 @@ private class MicroserviceRoutes[F[_]: Async]( private object MicroserviceRoutes { def apply[F[_]: Async: Parallel: Logger: MetricsRegistry: SparqlQueryTimeRecorder]( - projectConnConfig: ProjectsConnectionConfig + projectConnConfig: ProjectsConnectionConfig, + projectSparqlClient: ProjectSparqlClient[F] ): F[MicroserviceRoutes[F]] = for { implicit0(gv: GitLabClient[F]) <- GitLabClient[F]() implicit0(atf: AccessTokenFinder[F]) <- AccessTokenFinder[F]() @@ -360,10 +362,10 @@ private object MicroserviceRoutes { usersProjectsEndpoint <- users.projects.Endpoint[F] authenticator <- GitLabAuthenticator[F] authMiddleware <- Authentication.middlewareAuthenticatingIfNeeded(authenticator) - projectSlugAuthorizer <- Authorizer.using(ProjectSlugRecordsFinder[F]) - datasetIdAuthorizer <- Authorizer.using(DatasetIdRecordsFinder[F]) - datasetSameAsAuthorizer <- Authorizer.using(DatasetSameAsRecordsFinder[F]) - versionRoutes <- version.Routes[F] + projectSlugAuthorizer <- ProjectSlugRecordsFinder[F](projectSparqlClient, ru).map(_.asAuthorizer) + datasetIdAuthorizer = DatasetIdRecordsFinder[F](projectSparqlClient).asAuthorizer + datasetSameAsAuthorizer = DatasetSameAsRecordsFinder[F](projectSparqlClient).asAuthorizer + versionRoutes <- version.Routes[F] } yield new MicroserviceRoutes( datasetsSearchEndpoint, datasetDetailsEndpoint, diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinder.scala new file mode 100644 index 0000000000..d04c3c1d3d --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinder.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.knowledgegraph.datasets + +import cats.effect.kernel.Sync +import fs2.Stream +import eu.timepit.refined.auto._ +import io.renku.graph.http.server.security.Authorizer +import io.renku.graph.http.server.security.Authorizer.SecurityRecordFinder +import io.renku.graph.model.datasets +import io.renku.http.server.security.model +import io.renku.projectauth.util.ProjectAuthDataRow +import io.renku.triplesstore.{ProjectSparqlClient, SparqlQueryTimeRecorder} +import io.renku.triplesstore.client.syntax._ +import org.typelevel.log4cats.Logger + +trait DatasetIdRecordsFinder[F[_]] extends SecurityRecordFinder[F, datasets.Identifier] + +object DatasetIdRecordsFinder { + + def apply[F[_]: Sync: Logger: SparqlQueryTimeRecorder]( + projectSparqlClient: ProjectSparqlClient[F] + ): DatasetIdRecordsFinder[F] = + new Impl[F](projectSparqlClient) + + private class Impl[F[_]: Sync: Logger: SparqlQueryTimeRecorder](projectSparqlClient: ProjectSparqlClient[F]) + extends DatasetIdRecordsFinder[F] { + private[this] val timeRecorder = SparqlQueryTimeRecorder[F] + + override def apply(id: datasets.Identifier, user: Option[model.AuthUser]): F[List[Authorizer.SecurityRecord]] = + Stream + .evals(runQuery(id)) + .through(ProjectAuthDataRow.collect) + .map(p => Authorizer.SecurityRecord(p.visibility, p.slug, p.members.map(_.gitLabId))) + .compile + .toList + + private def runQuery(id: datasets.Identifier) = + timeRecorder.reportTime("security-dataset-id")( + projectSparqlClient.queryDecode[ProjectAuthDataRow](query(id)) + ) + + private def query(id: datasets.Identifier) = + sparql"""PREFIX schema: + |PREFIX renku: + | + | select distinct ?slug ?visibility ?memberRole + | where { + | bind (${id.value} as ?dsIdent). + | { + | select distinct ?projectId where { + | bind (${id.value} as ?dsIdent). + | graph ?projectId { + | ?projectId a schema:Project; + | renku:hasDataset ?datasetId. + | ?datasetId a schema:Dataset; + | schema:identifier ?dsIdent. + | } + | } + | } + | graph renku:ProjectAuth { + | ?projectId a schema:Project; + | renku:slug ?slug; + | renku:visibility ?visibility. + | Optional { + | ?projectId renku:memberRole ?memberRole. + | } + | } + | } + | ORDER BY ?slug + |""".stripMargin + } +} diff --git a/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinder.scala b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinder.scala new file mode 100644 index 0000000000..9992312f82 --- /dev/null +++ b/knowledge-graph/src/main/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinder.scala @@ -0,0 +1,82 @@ +/* + * 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.datasets + +import cats.effect.kernel.Sync +import fs2.Stream +import eu.timepit.refined.auto._ +import io.renku.graph.http.server.security.Authorizer +import io.renku.graph.model.datasets +import io.renku.http.server.security.model +import io.renku.jsonld.EntityId +import io.renku.projectauth.util.ProjectAuthDataRow +import io.renku.triplesstore.{ProjectSparqlClient, SparqlQueryTimeRecorder} +import io.renku.triplesstore.client.syntax._ +import org.typelevel.log4cats.Logger + +trait DatasetSameAsRecordsFinder[F[_]] extends Authorizer.SecurityRecordFinder[F, datasets.SameAs] + +object DatasetSameAsRecordsFinder { + + def apply[F[_]: Sync: Logger: SparqlQueryTimeRecorder]( + projectSparqlClient: ProjectSparqlClient[F] + ): DatasetSameAsRecordsFinder[F] = + new Impl[F](projectSparqlClient) + + private class Impl[F[_]: Sync: Logger: SparqlQueryTimeRecorder](projectSparqlClient: ProjectSparqlClient[F]) + extends DatasetSameAsRecordsFinder[F] { + private[this] val timeRecorder = SparqlQueryTimeRecorder[F] + + override def apply(sameAs: datasets.SameAs, user: Option[model.AuthUser]): F[List[Authorizer.SecurityRecord]] = + Stream + .evals(runQuery(sameAs)) + .through(ProjectAuthDataRow.collect) + .map(p => Authorizer.SecurityRecord(p.visibility, p.slug, p.members.map(_.gitLabId))) + .compile + .toList + + private def runQuery(sameAs: datasets.SameAs) = + timeRecorder.reportTime("security-dataset-sameAs")( + projectSparqlClient.queryDecode[ProjectAuthDataRow](query(sameAs)) + ) + + private def query(sameAs: datasets.SameAs) = + sparql"""PREFIX schema: + |PREFIX renku: + | + | select distinct ?slug ?visibility ?memberRole + | where { + | bind (${EntityId.of(sameAs.value)} as ?dsSameAs). + | graph schema:Dataset { + | ?dsSameAs a renku:DiscoverableDataset; + | renku:datasetProjectLink / renku:project ?projectId. + | } + | graph renku:ProjectAuth { + | ?projectId a schema:Project; + | renku:slug ?slug; + | renku:visibility ?visibility. + | Optional { + | ?projectId renku:memberRole ?memberRole. + | } + | } + | } + | ORDER BY ?slug + |""".stripMargin + } +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/DatasetProvision.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/DatasetProvision.scala new file mode 100644 index 0000000000..c6a7795d5c --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/DatasetProvision.scala @@ -0,0 +1,75 @@ +/* + * 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 + +import cats.effect.IO +import io.renku.entities.searchgraphs.SearchInfoDatasets +import io.renku.graph.model.entities.EntityFunctions +import io.renku.graph.model.projects.Role +import io.renku.graph.model.{RenkuUrl, entities, testentities} +import io.renku.projectauth.{ProjectAuthData, ProjectAuthService, ProjectMember} +import io.renku.triplesstore._ + +trait DatasetProvision extends SearchInfoDatasets { self: ProjectsDataset with InMemoryJena => + + def projectAuthServiceR(implicit renkuUrl: RenkuUrl) = + ProjectAuthService.resource[IO](projectsDSConnectionInfo.toCC()) + + override def provisionProject( + project: entities.Project + )(implicit + entityFunctions: EntityFunctions[entities.Project], + graphsProducer: GraphsProducer[entities.Project], + renkuUrl: RenkuUrl + ): IO[Unit] = { + val members = project.members.flatMap(p => p.maybeGitLabId.map(id => ProjectMember(id, Role.Reader))) + val authData = ProjectAuthData(project.slug, members, project.visibility) + super.provisionProject(project) *> projectAuthServiceR.use(_.update(authData)) + } + + def provisionProjectAndMembers( + project: entities.Project, + memberRole: PartialFunction[entities.Person, Role] = PartialFunction.empty + )(implicit + entityFunctions: EntityFunctions[entities.Project], + graphsProducer: GraphsProducer[entities.Project], + renkuUrl: RenkuUrl + ) = { + val members = project.members.flatMap(p => + p.maybeGitLabId.map(gid => ProjectMember(gid, memberRole.lift.apply(p).getOrElse(Role.Reader))) + ) + val authData = ProjectAuthData(project.slug, members, project.visibility) + super.provisionProject(project) *> projectAuthServiceR.use(_.update(authData)) + } + + def provisionTestProjectAndMembers( + project: testentities.Project, + memberRole: PartialFunction[testentities.Person, Role] = PartialFunction.empty + )(implicit + entityFunctions: EntityFunctions[entities.Project], + graphsProducer: GraphsProducer[entities.Project], + renkuUrl: RenkuUrl + ) = { + val members = project.members.flatMap(p => + p.maybeGitLabId.map(gid => ProjectMember(gid, memberRole.lift.apply(p).getOrElse(Role.Reader))) + ) + val authData = ProjectAuthData(project.slug, members, project.visibility) + super.provisionTestProject(project) *> projectAuthServiceR.use(_.update(authData)) + } +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinderSpec.scala new file mode 100644 index 0000000000..99418d5973 --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetIdRecordsFinderSpec.scala @@ -0,0 +1,97 @@ +/* + * 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.datasets + +import cats.effect.IO +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.projects.Role +import io.renku.graph.model.testentities.Person +import io.renku.graph.model.testentities.generators.EntitiesGenerators +import io.renku.triplesstore.ProjectSparqlClient + +class DatasetIdRecordsFinderSpec extends SecurityRecordFinderSupport { + lazy val finder = ProjectSparqlClient[IO](projectsDSConnectionInfo).map(DatasetIdRecordsFinder.apply[IO]) + + it should "find security records for a simple project with one dataset" in { + val project = projectWithDatasetAndMembers.generateOne + + val memberDef: PartialFunction[Person, Role] = { + case p if p == project.members.head => Role.Owner + } + + val dsId = project.datasets.head.identification.identifier + + for { + _ <- provisionTestProjectAndMembers(project, memberDef) + r <- finder.use(_.apply(dsId, None)) + _ = r shouldBe List(project).map(toSecRecord) + } yield () + } + + it should "find security records without members if there are none" in { + val project = projectWithDatasetAndNoMembers.generateOne + val dsId = project.datasets.head.identification.identifier + for { + _ <- provisionTestProjectAndMembers(project) + r <- finder.use(_.apply(dsId, None)) + _ = r shouldBe List(project).map(toSecRecord) + } yield () + } + + it should "find security records for a forked project" in { + val (dataset, (parentProject, project)) = projectAndFork.generateOne + + val dsId = dataset.identification.identifier + + for { + _ <- provisionTestProjectAndMembers(project) + _ <- provisionTestProjectAndMembers(parentProject) + r <- finder.use(_.apply(dsId, None)) + _ = r shouldBe List(project, parentProject).map(toSecRecord).sortBy(_.projectSlug.value) + } yield () + } + + it should "find security records for the project that has the dataset" in { + val createProjects = projectWithDatasetAndMembers.asStream.toIO + .evalTap(provisionTestProjectAndMembers(_)) + .take(3) + .compile + .toList + + for { + projects <- createProjects + dsId = projects.head.datasets.head.identification.identifier + r <- finder.use(_.apply(dsId, None)) + _ = r shouldBe projects.take(1).map(toSecRecord) + } yield () + } + + it should "find nothing if there is no project using the dataset" in { + val project = projectWithDatasetAndMembers.generateOne + val dsId = EntitiesGenerators.datasetIdentifiers + .suchThat(_ != project.datasets.head.identification.identifier) + .generateOne + + for { + _ <- provisionTestProjectAndMembers(project) + r <- finder.use(_.apply(dsId, None)) + _ = r shouldBe Nil + } yield () + } +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinderSpec.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinderSpec.scala new file mode 100644 index 0000000000..157230c839 --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/DatasetSameAsRecordsFinderSpec.scala @@ -0,0 +1,99 @@ +/* + * 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.datasets + +import cats.effect.IO +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.datasets.SameAs +import io.renku.graph.model.projects.Role +import io.renku.graph.model.testentities.Person +import io.renku.graph.model.testentities.generators.EntitiesGenerators +import io.renku.triplesstore.ProjectSparqlClient + +class DatasetSameAsRecordsFinderSpec extends SecurityRecordFinderSupport { + + lazy val finder = ProjectSparqlClient[IO](projectsDSConnectionInfo).map(DatasetSameAsRecordsFinder.apply[IO]) + + it should "find security records for a simple project with one dataset" in { + val project = projectWithDatasetAndMembers.generateOne + + val memberDef: PartialFunction[Person, Role] = { + case p if p == project.members.head => Role.Owner + } + + val dsSameAs = SameAs(project.datasets.head.provenance.topmostSameAs.value) + + for { + _ <- provisionTestProjectAndMembers(project, memberDef) + r <- finder.use(_.apply(dsSameAs, None)) + _ = r shouldBe List(project).map(toSecRecord) + } yield () + } + + it should "find security records without members if there are none" in { + val project = projectWithDatasetAndNoMembers.generateOne + val dsSameAs = SameAs(project.datasets.head.provenance.topmostSameAs.value) + for { + _ <- provisionTestProjectAndMembers(project) + r <- finder.use(_.apply(dsSameAs, None)) + _ = r shouldBe List(project).map(toSecRecord) + } yield () + } + + it should "find security records for a forked project" in { + val (dataset, (parentProject, project)) = projectAndFork.generateOne + + val dsSameAs = SameAs(dataset.provenance.topmostSameAs.value) + + for { + _ <- provisionTestProjectAndMembers(project) + _ <- provisionTestProjectAndMembers(parentProject) + r <- finder.use(_.apply(dsSameAs, None)) + _ = r shouldBe List(project, parentProject).map(toSecRecord).sortBy(_.projectSlug.value) + } yield () + } + + it should "find security records for the project that has the dataset" in { + val createProjects = projectWithDatasetAndMembers.asStream.toIO + .evalTap(provisionTestProjectAndMembers(_)) + .take(3) + .compile + .toList + + for { + projects <- createProjects + dsSameAs = projects.head.datasets.head.provenance.topmostSameAs + r <- finder.use(_.apply(SameAs(dsSameAs.value), None)) + _ = r shouldBe projects.take(1).map(toSecRecord) + } yield () + } + + it should "find nothing if there is no project using the dataset" in { + val project = projectWithDatasetAndMembers.generateOne + val dsSameAs = EntitiesGenerators.datasetSameAs + .suchThat(_.value != project.datasets.head.provenance.topmostSameAs.value) + .generateOne + + for { + _ <- provisionTestProjectAndMembers(project) + r <- finder.use(_.apply(dsSameAs, None)) + _ = r shouldBe Nil + } yield () + } +} diff --git a/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/SecurityRecordFinderSupport.scala b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/SecurityRecordFinderSupport.scala new file mode 100644 index 0000000000..bd8c6bf73b --- /dev/null +++ b/knowledge-graph/src/test/scala/io/renku/knowledgegraph/datasets/SecurityRecordFinderSupport.scala @@ -0,0 +1,78 @@ +/* + * 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.datasets + +import cats.effect.IO +import io.renku.graph.http.server.security.Authorizer +import io.renku.graph.model.testentities.generators.EntitiesGenerators +import io.renku.graph.model.{GitLabApiUrl, RenkuUrl, testentities} +import io.renku.interpreters.TestLogger +import io.renku.knowledgegraph.DatasetProvision +import io.renku.logging.TestSparqlQueryTimeRecorder +import io.renku.testtools.CustomAsyncIOSpec +import io.renku.triplesstore.{InMemoryJenaForSpec, ProjectsDataset, SparqlQueryTimeRecorder} +import org.scalatest.flatspec.AsyncFlatSpec +import org.scalatest.matchers.should +import org.typelevel.log4cats.Logger + +abstract class SecurityRecordFinderSupport + extends AsyncFlatSpec + with CustomAsyncIOSpec + with should.Matchers + with InMemoryJenaForSpec + // with ExternalJenaForSpec + with ProjectsDataset + with DatasetProvision { + + implicit val renkuUrl: RenkuUrl = RenkuUrl("http://u.rl") + implicit val gitlabUrl: GitLabApiUrl = GitLabApiUrl("http://gitl.ab") + implicit val ioLogger: Logger[IO] = TestLogger() + implicit val sqtr: SparqlQueryTimeRecorder[IO] = TestSparqlQueryTimeRecorder.createUnsafe + + def toSecRecord(p: testentities.Project) = + Authorizer.SecurityRecord(p.visibility, p.slug, p.members.flatMap(_.maybeGitLabId)) + + def projectWithDatasetAndMembers = + EntitiesGenerators + .renkuProjectEntities( + visibilityGen = EntitiesGenerators.anyVisibility, + creatorGen = EntitiesGenerators.personEntities(EntitiesGenerators.withGitLabId) + ) + .withDatasets( + EntitiesGenerators.datasetEntities(EntitiesGenerators.provenanceInternal()) + ) + .suchThat(_.members.nonEmpty) + + def projectWithDatasetAndNoMembers = + EntitiesGenerators + .renkuProjectEntities( + visibilityGen = EntitiesGenerators.anyVisibility, + creatorGen = EntitiesGenerators.personEntities(EntitiesGenerators.withGitLabId) + ) + .modify(EntitiesGenerators.removeMembers()) + .withDatasets( + EntitiesGenerators.datasetEntities(EntitiesGenerators.provenanceInternal()) + ) + + def projectAndFork = + EntitiesGenerators + .renkuProjectEntities(EntitiesGenerators.anyVisibility) + .addDataset(EntitiesGenerators.datasetEntities(EntitiesGenerators.provenanceNonModified)) + .forkOnce() +} diff --git a/project-auth/src/main/scala/io/renku/projectauth/ProjectAuth.scala b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuth.scala new file mode 100644 index 0000000000..d3c0b7a24c --- /dev/null +++ b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuth.scala @@ -0,0 +1,26 @@ +/* + * 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.projectauth + +import io.renku.graph.model.Schemas + +object ProjectAuth { + + def graph = Schemas.renku / "ProjectAuth" +} diff --git a/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthData.scala b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthData.scala index c31fb7b794..73592320c9 100644 --- a/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthData.scala +++ b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthData.scala @@ -18,6 +18,8 @@ package io.renku.projectauth +import cats.Show +import cats.syntax.all._ import io.renku.graph.model.projects.{ResourceId, Slug, Visibility} import io.renku.graph.model.{RenkuUrl, Schemas} import io.renku.jsonld.JsonLD.JsonLDArray @@ -31,6 +33,7 @@ final case class ProjectAuthData( ) object ProjectAuthData { + implicit def jsonLDEncoder(implicit renkuUrl: RenkuUrl): JsonLDEncoder[ProjectAuthData] = JsonLDEncoder.instance { data => JsonLD.entity( @@ -42,4 +45,8 @@ object ProjectAuthData { Schemas.renku / "memberRole" -> JsonLDArray(data.members.map(_.encoded.asJsonLD).toSeq) ) } + + implicit val show: Show[ProjectAuthData] = Show.show { case ProjectAuthData(slug, members, visibility) => + show"slug = $slug, visibility = $visibility, members = [${members.toList.mkString_(", ")}]" + } } diff --git a/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthService.scala b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthService.scala index 113be3e8c1..d2d0bfcd92 100644 --- a/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthService.scala +++ b/project-auth/src/main/scala/io/renku/projectauth/ProjectAuthService.scala @@ -24,13 +24,13 @@ import cats.effect._ import cats.syntax.all._ import fs2.io.net.Network import fs2.{Pipe, Stream} -import io.circe.Decoder -import io.renku.graph.model.projects.{Slug, Visibility} +import io.renku.graph.model.projects.Slug import io.renku.graph.model.{RenkuUrl, Schemas} import io.renku.jsonld.NamedGraph import io.renku.jsonld.syntax._ -import io.renku.tinytypes.json.TinyTypeDecoders._ -import io.renku.triplesstore.client.http.{ConnectionConfig, RowDecoder, SparqlClient} +import io.renku.projectauth.util.ProjectAuthDataRow +import io.renku.triplesstore.client.http.{ConnectionConfig, SparqlClient} +import io.renku.triplesstore.client.sparql.Fragment import io.renku.triplesstore.client.syntax._ import org.typelevel.log4cats.Logger @@ -46,7 +46,7 @@ trait ProjectAuthService[F[_]] { def remove(slugs: NonEmptyList[Slug]): F[Unit] def remove(slug: Slug, more: Slug*): F[Unit] = remove(NonEmptyList(slug, more.toList)) - def getAll(chunkSize: Int = 100): Stream[F, ProjectAuthData] + def getAll(filter: QueryFilter, chunkSize: Int = 100): Stream[F, ProjectAuthData] } object ProjectAuthService { @@ -62,12 +62,12 @@ object ProjectAuthService { private final class Impl[F[_]: MonadThrow](sparqlClient: SparqlClient[F], renkuUrl: RenkuUrl) extends ProjectAuthService[F] { - private[this] val graph = Schemas.renku / "ProjectAuth" + private[this] val graph = ProjectAuth.graph private implicit val rUrl: RenkuUrl = renkuUrl override def remove(slugs: NonEmptyList[Slug]): F[Unit] = - sparqlClient.update(sparql"""PREFIX schema: - |PREFIX renku: + sparqlClient.update(sparql"""${"schema" -> Schemas.schema} + |${"renku" -> Schemas.renku} | |DELETE { Graph $graph {?s ?p ?o} } |WHERE { @@ -97,25 +97,19 @@ object ProjectAuthService { .evalMap(sparqlClient.upload) .drain - override def getAll(chunkSize: Int): Stream[F, ProjectAuthData] = - streamAll(chunkSize) + override def getAll(filter: QueryFilter, chunkSize: Int): Stream[F, ProjectAuthData] = + streamAll(filter, chunkSize) - private def streamAll(chunkSize: Int) = + private def streamAll(filter: QueryFilter, chunkSize: Int) = Stream .iterate(0)(_ + chunkSize) - .evalMap(offset => getChunk(chunkSize, offset)) + .evalMap(offset => getChunk(filter, chunkSize, offset)) .takeWhile(_.nonEmpty) .flatMap(Stream.emits) - .groupAdjacentBy(_._1) - .map { case (slug, rest) => - val members = rest.toList.flatMap(_._3) - val vis = rest.head.map(_._2) - vis.map(v => ProjectAuthData(slug, members.toSet, v)) - } - .unNone - - private def getChunk(limit: Int, offset: Int) = - sparqlClient.queryDecode[(Slug, Visibility, Option[ProjectMember])]( + .through(ProjectAuthDataRow.collect) + + private def getChunk(filter: QueryFilter, limit: Int, offset: Int): F[List[ProjectAuthDataRow]] = + sparqlClient.queryDecode[ProjectAuthDataRow]( sparql"""PREFIX schema: |PREFIX renku: | @@ -128,6 +122,8 @@ object ProjectAuthService { | OPTIONAL { | ?project renku:memberRole ?memberRole. | } + | ${slugFilter(filter)} + | ${memberFilter(filter)} | } |} |ORDER BY ?slug @@ -136,12 +132,20 @@ object ProjectAuthService { |""".stripMargin ) - private implicit val projectMemberDecoder: Decoder[ProjectMember] = - Decoder.decodeString.emap(ProjectMember.fromEncoded) - - private implicit val tupleRowDecoder: RowDecoder[(Slug, Visibility, Option[ProjectMember])] = - RowDecoder.forProduct3("slug", "visibility", "memberRole")( - Tuple3.apply[Slug, Visibility, Option[ProjectMember]] - ) + private def slugFilter(f: QueryFilter) = + f.slug.map(s => sparql"?project renku:slug ${s.value}.").getOrElse(Fragment.empty) + + private def memberFilter(f: QueryFilter) = + f.member match { + case s if s.isEmpty => Fragment.empty + case s if s.size == 1 => sparql"?project renku:memberId ${s.head.value}" + case s => + sparql""" + |?project renku:memberId ?memberId. + |VALUES (?memberId) { + | ${s.map(_.value)} + |}. + |""".stripMargin + } } } diff --git a/project-auth/src/main/scala/io/renku/projectauth/ProjectMember.scala b/project-auth/src/main/scala/io/renku/projectauth/ProjectMember.scala index 8bd8de36c8..59c43f5e0f 100644 --- a/project-auth/src/main/scala/io/renku/projectauth/ProjectMember.scala +++ b/project-auth/src/main/scala/io/renku/projectauth/ProjectMember.scala @@ -18,6 +18,7 @@ package io.renku.projectauth +import cats.Show import io.renku.graph.model.persons.GitLabId import io.renku.graph.model.projects.Role @@ -31,6 +32,7 @@ final case class ProjectMember( } object ProjectMember { + private[projectauth] def fromEncoded(str: String): Either[String, ProjectMember] = str.split(':').toList match { case idStr :: roleStr :: Nil => @@ -45,4 +47,6 @@ object ProjectMember { def fromGitLabData(gitLabId: GitLabId, accessLevel: Int): ProjectMember = ProjectMember(gitLabId, Role.fromGitLabAccessLevel(accessLevel)) + + implicit val show: Show[ProjectMember] = Show.show(_.encoded) } diff --git a/project-auth/src/main/scala/io/renku/projectauth/QueryFilter.scala b/project-auth/src/main/scala/io/renku/projectauth/QueryFilter.scala new file mode 100644 index 0000000000..7445ac7b6f --- /dev/null +++ b/project-auth/src/main/scala/io/renku/projectauth/QueryFilter.scala @@ -0,0 +1,39 @@ +/* + * 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.projectauth + +import io.renku.graph.model.persons.GitLabId +import io.renku.graph.model.projects.Slug + +final case class QueryFilter( + slug: Option[Slug], + member: Set[GitLabId] +) { + + def withSlug(slug: Slug): QueryFilter = + copy(slug = Some(slug)) + + def withMember(id: GitLabId): QueryFilter = + copy(member = member + id) +} + +object QueryFilter { + + def all: QueryFilter = QueryFilter(None, Set.empty) +} diff --git a/project-auth/src/main/scala/io/renku/projectauth/util/ProjectAuthDataRow.scala b/project-auth/src/main/scala/io/renku/projectauth/util/ProjectAuthDataRow.scala new file mode 100644 index 0000000000..50a1fc2103 --- /dev/null +++ b/project-auth/src/main/scala/io/renku/projectauth/util/ProjectAuthDataRow.scala @@ -0,0 +1,46 @@ +/* + * 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.projectauth.util + +import fs2.Pipe +import io.circe.Decoder +import io.renku.graph.model.projects.{Slug, Visibility} +import io.renku.projectauth.{ProjectAuthData, ProjectMember} +import io.renku.triplesstore.client.http.RowDecoder +import io.renku.tinytypes.json.TinyTypeDecoders._ + +final case class ProjectAuthDataRow(slug: Slug, visibility: Visibility, memberRole: Option[ProjectMember]) + +object ProjectAuthDataRow { + + private implicit val projectMemberDecoder: Decoder[ProjectMember] = + Decoder.decodeString.emap(ProjectMember.fromEncoded) + + implicit val tupleRowDecoder: RowDecoder[ProjectAuthDataRow] = + RowDecoder.forProduct3("slug", "visibility", "memberRole")(ProjectAuthDataRow.apply) + + def collect[F[_]]: Pipe[F, ProjectAuthDataRow, ProjectAuthData] = + _.groupAdjacentBy(_.slug) + .map { case (slug, rest) => + val members = rest.toList.flatMap(_.memberRole) + val vis = rest.head.map(_.visibility) + vis.map(v => ProjectAuthData(slug, members.toSet, v)) + } + .unNone +} diff --git a/project-auth/src/test/scala/io/renku/projectauth/Generators.scala b/project-auth/src/test/scala/io/renku/projectauth/Generators.scala index 5524a8632a..cb70be2da2 100644 --- a/project-auth/src/test/scala/io/renku/projectauth/Generators.scala +++ b/project-auth/src/test/scala/io/renku/projectauth/Generators.scala @@ -18,7 +18,11 @@ package io.renku.projectauth +import cats.syntax.all._ +import io.renku.graph.model.persons.GitLabId +import io.renku.graph.model.projects.{Role, Slug, Visibility} import io.renku.graph.model.{RenkuTinyTypeGenerators, persons} +import io.renku.generators.Generators.Implicits._ import org.scalacheck.Gen object Generators { @@ -31,9 +35,34 @@ object Generators { id <- gitLabIds } yield ProjectMember(id, role) - val projectAuthDataGen: Gen[ProjectAuthData] = for { - slug <- RenkuTinyTypeGenerators.projectSlugs - members <- Gen.choose(0, 150).flatMap(n => Gen.listOfN(n, memberGen)) - visibility <- RenkuTinyTypeGenerators.projectVisibilities - } yield ProjectAuthData(slug, members.toSet, visibility) + def projectAuthData: ProjectAuthDataBuilder = ProjectAuthDataBuilder() + + val projectAuthDataGen: Gen[ProjectAuthData] = + projectAuthData.build + + final case class ProjectAuthDataBuilder( + slug: Gen[Slug] = RenkuTinyTypeGenerators.projectSlugs, + members: Gen[Set[ProjectMember]] = Gen.choose(0, 150).flatMap(n => Gen.listOfN(n, memberGen)).map(_.toSet), + visibility: Gen[Visibility] = RenkuTinyTypeGenerators.projectVisibilities + ) { + def withMembers(members: (Int, Role)*): ProjectAuthDataBuilder = + copy(members = Gen.const(members.map(t => ProjectMember(GitLabId(t._1), t._2)).toSet)) + + def withRoles(roles: Role*): ProjectAuthDataBuilder = + copy(members = roles.traverse(r => gitLabIds.map(id => ProjectMember(id, r))).map(_.toSet)) + + def withSlug(slug: Slug): ProjectAuthDataBuilder = + copy(slug = Gen.const(slug)) + + def withVisibility(v: Visibility): ProjectAuthDataBuilder = + copy(visibility = Gen.const(v)) + + val build = for { + s <- slug + m <- members + v <- visibility + } yield ProjectAuthData(s, m, v) + + val stream = build.asStream + } } diff --git a/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSpec.scala b/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSpec.scala index f402ff03f6..67afc2b73f 100644 --- a/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSpec.scala +++ b/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSpec.scala @@ -20,121 +20,114 @@ package io.renku.projectauth import cats.data.NonEmptyList import cats.effect.IO -import cats.effect.kernel.Resource import cats.effect.testing.scalatest.AsyncIOSpec -import fs2.Stream import io.renku.generators.Generators.Implicits._ -import io.renku.graph.model.RenkuUrl import io.renku.graph.model.persons.GitLabId -import io.renku.graph.model.projects.{Role, Visibility} -import io.renku.triplesstore.client.util.JenaContainerSupport +import io.renku.graph.model.projects.{Role, Slug, Visibility} +import io.renku.graph.model.{RenkuTinyTypeGenerators, RenkuUrl} import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.matchers.should import org.typelevel.log4cats.Logger import org.typelevel.log4cats.slf4j.Slf4jLogger -class ProjectAuthServiceSpec extends AsyncFlatSpec with AsyncIOSpec with JenaContainerSupport with should.Matchers { +class ProjectAuthServiceSpec + extends AsyncFlatSpec + with AsyncIOSpec + with ProjectAuthServiceSupport + with should.Matchers { implicit val logger: Logger[IO] = Slf4jLogger.getLogger[IO] implicit val renkuUrl: RenkuUrl = RenkuUrl("http://localhost/renku") - def withProjectAuthService: Resource[IO, ProjectAuthService[IO]] = - withDataset("projectauth").map(ProjectAuthService[IO](_, renkuUrl)) + def randomData(num: Int) = Generators.projectAuthDataGen.asStream.take(num) it should "add data" in { - withProjectAuthService.use { s => + withProjectAuthServiceData(randomData(20)).use { case (s, data) => for { - data <- Generators.projectAuthDataGen.asStream.toIO.take(20).compile.toVector - _ <- Stream - .emits(data) - .through(s.updateAll) - .compile - .drain - - n <- s.getAll(15).compile.toVector + n <- s.getAll(QueryFilter.all, 15).compile.toVector _ = n shouldBe data.sortBy(_.slug) } yield () } } it should "work with no members" in { - withProjectAuthService.use { s => + withProjectAuthServiceData(randomData(2)).use { case (s, data) => for { - data <- Generators.projectAuthDataGen.asStream.toIO - .take(2) - .map(_.copy(members = Set.empty)) - .compile - .toVector - _ <- Stream - .emits(data) - .through(s.updateAll) - .compile - .drain - - n <- s.getAll().compile.toVector + n <- s.getAll(QueryFilter.all).compile.toVector _ = n shouldBe data.sortBy(_.slug) } yield () } } it should "remove projects" in { - withProjectAuthService.use { s => + val data = Generators.projectAuthData.withVisibility(Visibility.Internal).stream + withProjectAuthServiceData(data.take(1)).use { case (s, original) => for { - original <- Generators.projectAuthDataGen.asStream.toIO - .take(1) - .map(_.copy(visibility = Visibility.Internal)) - .compile - .lastOrError - _ <- s.update(original) - - n <- s.getAll().compile.toVector - _ = n.head shouldBe original - - _ <- s.remove(original.slug) - _ <- s.getAll().compile.toVector.asserting(v => v shouldBe Vector.empty) + n <- s.getAll(QueryFilter.all).compile.toList + _ = n shouldBe original + + _ <- s.remove(original.head.slug) + _ <- s.getAll(QueryFilter.all).compile.toVector.asserting(v => v shouldBe Vector.empty) } yield () } } it should "remove selectively" in { - withProjectAuthService.use { s => + withProjectAuthServiceData(randomData(6)).use { case (s, data) => + val (toremove, tokeep) = data.splitAt(3) for { - data <- Generators.projectAuthDataGen.asStream.toIO - .take(6) - .compile - .toVector - _ <- Stream.emits(data).through(s.updateAll).compile.drain - - (toremove, tokeep) = data.splitAt(3) _ <- s.remove(NonEmptyList.fromListUnsafe(toremove.map(_.slug).toList)) - - _ <- s.getAll().compile.toVector.asserting(v => v shouldBe tokeep.sortBy(_.slug)) + _ <- s.getAll(QueryFilter.all).compile.toVector.asserting(v => v shouldBe tokeep.sortBy(_.slug)) } yield () } } it should "update new properties" in { - withProjectAuthService.use { s => + val data = Generators.projectAuthData.withVisibility(Visibility.Internal).stream + withProjectAuthServiceData(data.take(1)).use { case (s, original) => for { - original <- Generators.projectAuthDataGen.asStream.toIO - .take(1) - .map(_.copy(visibility = Visibility.Internal)) - .compile - .lastOrError - _ <- s.update(original) + n <- s.getAll(QueryFilter.all).compile.toList + _ = n shouldBe original - n <- s.getAll().compile.toVector - _ = n.head shouldBe original - - second = original.copy(visibility = Visibility.Public) + second = original.head.copy(visibility = Visibility.Public) _ <- s.update(second) - n2 <- s.getAll().compile.toVector + n2 <- s.getAll(QueryFilter.all).compile.toVector _ = n2.head shouldBe second third = second.copy(members = second.members + ProjectMember(GitLabId(43), Role.Reader)) _ <- s.update(third) - n3 <- s.getAll().compile.toVector + n3 <- s.getAll(QueryFilter.all).compile.toVector _ = n3.head shouldBe third } yield () } } + + it should "search for a specific project by slug" in { + withProjectAuthServiceData(randomData(1)).use { case (s, original) => + for { + found <- s.getAll(QueryFilter.all.withSlug(original.head.slug)).compile.lastOrError + nf <- s.getAll(QueryFilter.all.withSlug(Slug(original.head.slug.value + "x"))).compile.last + + _ = found shouldBe original.head + _ = nf shouldBe None + } yield () + } + } + + it should "search by member id" in { + withProjectAuthServiceData(randomData(1)).use { case (s, original) => + for { + found <- s.getAll(QueryFilter.all.withMember(original.head.members.head.gitLabId)).compile.lastOrError + + gId <- IO( + RenkuTinyTypeGenerators.personGitLabIds + .suchThat(id => !original.head.members.map(_.gitLabId).contains(id)) + .generateOne + ) + nf <- s.getAll(QueryFilter.all.withMember(gId)).compile.last + + _ = found shouldBe original.head + _ = nf shouldBe None + } yield () + } + } } diff --git a/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSupport.scala b/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSupport.scala new file mode 100644 index 0000000000..653b33e267 --- /dev/null +++ b/project-auth/src/test/scala/io/renku/projectauth/ProjectAuthServiceSupport.scala @@ -0,0 +1,46 @@ +/* + * 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.projectauth + +import cats.effect._ +import fs2.Stream +import io.renku.generators.Generators.Implicits._ +import io.renku.graph.model.RenkuUrl +import io.renku.triplesstore.client.util.JenaContainerSupport +import org.scalacheck.Gen +import org.scalatest.Suite +import org.typelevel.log4cats.Logger + +trait ProjectAuthServiceSupport extends JenaContainerSupport { self: Suite => + + val datasetName: String = "projectauth" + + def withProjectAuthService(implicit renkuUrl: RenkuUrl, L: Logger[IO]): Resource[IO, ProjectAuthService[IO]] = + withDataset(datasetName).map(ProjectAuthService[IO](_, renkuUrl)) + + def withProjectAuthServiceData(data: Stream[Gen, ProjectAuthData])(implicit + renkuUrl: RenkuUrl, + L: Logger[IO] + ) = + withProjectAuthService.flatMap { s => + val genData = data.toIO.compile.toList + val insert = genData.flatMap(d => Stream.emits(d).through(s.updateAll).compile.drain.as(d)) + Resource.eval(insert).map(data => (s, data)) + } +} diff --git a/renku-model/src/test/scala/io/renku/graph/model/testentities/generators/ProjectEntitiesGenerators.scala b/renku-model/src/test/scala/io/renku/graph/model/testentities/generators/ProjectEntitiesGenerators.scala index 2ca91839b2..22d80f6848 100644 --- a/renku-model/src/test/scala/io/renku/graph/model/testentities/generators/ProjectEntitiesGenerators.scala +++ b/renku-model/src/test/scala/io/renku/graph/model/testentities/generators/ProjectEntitiesGenerators.scala @@ -64,9 +64,9 @@ trait ProjectEntitiesGenerators { def creatorLens[P <: Project]: Lens[P, Option[Person]] = Lens[P, Option[Person]](_.maybeCreator) { maybeCreator => _.fold(_.copy(maybeCreator = maybeCreator), - _.copy(maybeCreator = maybeCreator), - _.copy(maybeCreator = maybeCreator), - _.copy(maybeCreator = maybeCreator) + _.copy(maybeCreator = maybeCreator), + _.copy(maybeCreator = maybeCreator), + _.copy(maybeCreator = maybeCreator) ).asInstanceOf[P] } diff --git a/triples-store-client/src/main/scala/io/renku/triplesstore/client/sparql/StringInterpolator.scala b/triples-store-client/src/main/scala/io/renku/triplesstore/client/sparql/StringInterpolator.scala index 9ff462aa28..e32394bab5 100644 --- a/triples-store-client/src/main/scala/io/renku/triplesstore/client/sparql/StringInterpolator.scala +++ b/triples-store-client/src/main/scala/io/renku/triplesstore/client/sparql/StringInterpolator.scala @@ -18,7 +18,7 @@ package io.renku.triplesstore.client.sparql -import io.renku.jsonld.{EntityId, Property} +import io.renku.jsonld.{EntityId, Property, Schema} import io.renku.triplesstore.client.model.{Quad, Triple, TripleObject} import io.renku.triplesstore.client.syntax._ @@ -36,26 +36,27 @@ class StringInterpolator(private val sc: StringContext) { } private lazy val makeValue: ((Any, Int)) => String = { - case (a: LuceneQuery, _) => a.asSparql.sparql - case (a: String, _) => a.asTripleObject.asSparql.sparql - case (a: Char, _) => a.toString.asTripleObject.asSparql.sparql - case (a: Float, _) => a.asTripleObject.asSparql.sparql - case (a: Int, _) => a.asTripleObject.asSparql.sparql - case (a: Long, _) => a.asTripleObject.asSparql.sparql - case (a: Double, _) => a.asTripleObject.asSparql.sparql - case (a: Boolean, _) => a.asTripleObject.asSparql.sparql - case (a: Instant, _) => a.asTripleObject.asSparql.sparql - case (a: LocalDate, _) => a.asTripleObject.asSparql.sparql - case (a: EntityId, _) => a.asSparql.sparql - case (a: Triple, _) => a.asSparql.sparql - case (a: Quad, _) => a.asSparql.sparql - case (a: TripleObject, _) => a.asSparql.sparql - case (a: Property, _) => a.asSparql.sparql - case (a: Fragment, _) => a.sparql - case (a: VarName, _) => a.name - case (it: Iterable[Any], idx) => resolveIterable(it, idx) - case (opt: Option[Any], idx) => opt.map(makeValue(_, idx)).getOrElse("") - case (arg, _) => sys.error(s"Unsupported value type '${arg.getClass}: $arg'") + case (a: LuceneQuery, _) => a.asSparql.sparql + case (a: String, _) => a.asTripleObject.asSparql.sparql + case (a: Char, _) => a.toString.asTripleObject.asSparql.sparql + case (a: Float, _) => a.asTripleObject.asSparql.sparql + case (a: Int, _) => a.asTripleObject.asSparql.sparql + case (a: Long, _) => a.asTripleObject.asSparql.sparql + case (a: Double, _) => a.asTripleObject.asSparql.sparql + case (a: Boolean, _) => a.asTripleObject.asSparql.sparql + case (a: Instant, _) => a.asTripleObject.asSparql.sparql + case (a: LocalDate, _) => a.asTripleObject.asSparql.sparql + case (a: EntityId, _) => a.asSparql.sparql + case (a: Triple, _) => a.asSparql.sparql + case (a: Quad, _) => a.asSparql.sparql + case (a: TripleObject, _) => a.asSparql.sparql + case (a: Property, _) => a.asSparql.sparql + case (a: Fragment, _) => a.sparql + case (a: VarName, _) => a.name + case ((n: String, s: Schema), _) => s.asPrefix(n) + case (it: Iterable[Any], idx) => resolveIterable(it, idx) + case (opt: Option[Any], idx) => opt.map(makeValue(_, idx)).getOrElse("") + case (arg, _) => sys.error(s"Unsupported value type '${arg.getClass}: $arg'") } private def resolveIterable(it: Iterable[Any], idx: Int) = {