diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownload.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownload.scala index 8ea91c610d..43a7115c8c 100644 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownload.scala +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownload.scala @@ -35,6 +35,7 @@ import monix.execution.Scheduler import java.nio.ByteBuffer import java.nio.charset.StandardCharsets +import akka.stream.alpakka.file.ArchiveMetadata /** * Archive download functionality. @@ -55,10 +56,9 @@ trait ArchiveDownload { * @param caller * the caller to be used for checking for access */ - def apply[M]( + def apply( value: ArchiveValue, project: ProjectRef, - format: ArchiveFormat[M], ignoreNotFound: Boolean )(implicit caller: Caller, scheduler: Scheduler): IO[ArchiveRejection, AkkaSource] @@ -96,18 +96,17 @@ object ArchiveDownload { private val printer = Printer.spaces2.copy(dropNullValues = true) private val sourcePrinter = Printer.spaces2.copy(dropNullValues = false) - override def apply[M]( + override def apply( value: ArchiveValue, project: ProjectRef, - format: ArchiveFormat[M], ignoreNotFound: Boolean )(implicit caller: Caller, scheduler: Scheduler): IO[ArchiveRejection, AkkaSource] = { for { references <- value.resources.toList.traverse(toFullReference) _ <- checkResourcePermissions(references, project) - contentStream <- resolveReferencesAsStream(references, project, ignoreNotFound, format) + contentStream <- resolveReferencesAsStream(references, project, ignoreNotFound) } yield { - Source.fromGraph(StreamConverter(contentStream)).via(format.writeFlow) + Source.fromGraph(StreamConverter(contentStream)).via(Zip.writeFlow) } } @@ -124,34 +123,29 @@ object ArchiveDownload { } } - private def resolveReferencesAsStream[M]( + private def resolveReferencesAsStream( references: List[FullArchiveReference], project: ProjectRef, - ignoreNotFound: Boolean, - format: ArchiveFormat[M] - )(implicit caller: Caller): IO[ArchiveRejection, Stream[Task, (M, AkkaSource)]] = { + ignoreNotFound: Boolean + )(implicit caller: Caller): IO[ArchiveRejection, Stream[Task, (ArchiveMetadata, AkkaSource)]] = { references .traverseFilter { - case ref: FileReference => fileEntry(ref, project, format, ignoreNotFound) - case ref: ResourceReference => resourceEntry(ref, project, format, ignoreNotFound) + case ref: FileReference => fileEntry(ref, project, ignoreNotFound) + case ref: ResourceReference => resourceEntry(ref, project, ignoreNotFound) } - .map(sortWith(format)) + .map(sortWith) .map(asStream) } - private def sortWith[M]( - format: ArchiveFormat[M] - )(list: List[(M, Task[AkkaSource])]): List[(M, Task[AkkaSource])] = { - list.sortBy { case (entry, _) => - entry - }(format.ordering) - } + private def sortWith(list: List[(ArchiveMetadata, Task[AkkaSource])]): List[(ArchiveMetadata, Task[AkkaSource])] = + list.sortBy { case (entry, _) => entry }(Zip.ordering) - private def asStream[M](list: List[(M, Task[AkkaSource])]) = { - Stream.iterable(list).evalMap[Task, (M, AkkaSource)] { case (metadata, source) => + private def asStream( + list: List[(ArchiveMetadata, Task[AkkaSource])] + ): Stream[Task, (ArchiveMetadata, AkkaSource)] = + Stream.iterable(list).evalMap { case (metadata, source) => source.map(metadata -> _) } - } private def checkResourcePermissions( refs: List[FullArchiveReference], @@ -166,14 +160,13 @@ object ArchiveDownload { ) .void - private def fileEntry[Metadata]( + private def fileEntry( ref: FileReference, project: ProjectRef, - format: ArchiveFormat[Metadata], ignoreNotFound: Boolean )(implicit caller: Caller - ): IO[ArchiveRejection, Option[(Metadata, Task[AkkaSource])]] = { + ): IO[ArchiveRejection, Option[(ArchiveMetadata, Task[AkkaSource])]] = { val refProject = ref.project.getOrElse(project) // the required permissions are checked for each file content fetch val entry = fetchFileContent(ref.ref, refProject, caller) @@ -184,21 +177,19 @@ object ArchiveDownload { case FileRejection.AuthorizationFailed(addr, perm) => AuthorizationFailed(addr, perm) case other => WrappedFileRejection(other) } - .flatMap { case FileResponse(fileMetadata, content) => - IO.fromEither( - pathOf(ref, project, format, fileMetadata.filename).map { path => - val archiveMetadata = format.metadata(path, fileMetadata.bytes) - val contentTask: Task[AkkaSource] = content - .tapError(response => - UIO.delay( - logger - .error(s"Error streaming file '${fileMetadata.filename}' for archive: ${response.value.value}") - ) - ) - .mapError(response => ArchiveDownloadError(fileMetadata.filename, response)) - Some((archiveMetadata, contentTask)) - } - ) + .map { case FileResponse(fileMetadata, content) => + val path = pathOf(ref, project, fileMetadata.filename) + val archiveMetadata = Zip.metadata(path) + val contentTask: Task[AkkaSource] = content + .tapError(response => + UIO.delay( + logger + .error(s"Error streaming file '${fileMetadata.filename}' for archive: ${response.value.value}") + ) + ) + .mapError(response => ArchiveDownloadError(fileMetadata.filename, response)) + Some((archiveMetadata, contentTask)) + } if (ignoreNotFound) entry.onErrorRecover { case _: ResourceNotFound => None } else entry @@ -207,27 +198,21 @@ object ArchiveDownload { private def pathOf( ref: FileReference, project: ProjectRef, - format: ArchiveFormat[_], filename: String - ): Either[FilenameTooLong, String] = - ref.path.map { p => Right(p.value.toString) }.getOrElse { + ): String = + ref.path.map(_.value.toString).getOrElse { val p = ref.project.getOrElse(project) - Either.cond( - format != ArchiveFormat.Tar || filename.length < 100, - s"$p/file/$filename", - FilenameTooLong(ref.ref.original, p, filename) - ) + s"$p/file/$filename" } - private def resourceEntry[Metadata]( + private def resourceEntry( ref: ResourceReference, project: ProjectRef, - format: ArchiveFormat[Metadata], ignoreNotFound: Boolean - ): IO[ArchiveRejection, Option[(Metadata, Task[AkkaSource])]] = { + ): IO[ArchiveRejection, Option[(ArchiveMetadata, Task[AkkaSource])]] = { val archiveEntry = resourceRefToByteString(ref, project).map { content => val path = pathOf(ref, project) - val metadata = format.metadata(path, content.length.toLong) + val metadata = Zip.metadata(path) Some((metadata, Task.pure(Source.single(content)))) } if (ignoreNotFound) archiveEntry.onErrorHandle { _: ResourceNotFound => None } diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/Archives.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/Archives.scala index e451e6fdde..c99b3d5a04 100644 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/Archives.scala +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/Archives.scala @@ -166,13 +166,12 @@ class Archives( def download( id: IdSegment, project: ProjectRef, - format: ArchiveFormat[_], ignoreNotFound: Boolean )(implicit caller: Caller, scheduler: Scheduler): IO[ArchiveRejection, AkkaSource] = (for { resource <- fetch(id, project) value = resource.value - source <- archiveDownload(value.value, project, format, ignoreNotFound) + source <- archiveDownload(value.value, project, ignoreNotFound) } yield source).span("downloadArchive") private def eval(cmd: CreateArchive): IO[ArchiveRejection, ArchiveResource] = diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveFormat.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveFormat.scala deleted file mode 100644 index 378e39942f..0000000000 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveFormat.scala +++ /dev/null @@ -1,94 +0,0 @@ -package ch.epfl.bluebrain.nexus.delta.plugins.archive.model - -import akka.NotUsed -import akka.http.scaladsl.model.{ContentType, HttpRequest, MediaTypes} -import akka.stream.alpakka.file.scaladsl.Archive -import akka.stream.alpakka.file.{ArchiveMetadata, TarArchiveMetadata} -import akka.stream.scaladsl.{Flow, Source} -import akka.util.ByteString -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveFormat.WriteFlow -import ch.epfl.bluebrain.nexus.delta.sdk.utils.HeadersUtils - -/** - * Available format to download the archive - */ -sealed trait ArchiveFormat[Metadata] extends Product with Serializable { - - /** - * Content type - */ - def contentType: ContentType - - /** - * File extension - */ - def fileExtension: String - - /** - * How to build the metadata for the archive entry - */ - def metadata(filename: String, size: Long): Metadata - - /** - * Ordering for the archive entries - */ - def ordering: Ordering[Metadata] = Ordering.by(filePath) - - /** - * How to extract the file path from the archive metadata - */ - def filePath(metadata: Metadata): String - - /** - * Flow to create an archive - */ - def writeFlow: WriteFlow[Metadata] -} - -object ArchiveFormat { - - type WriteFlow[Metadata] = Flow[(Metadata, Source[ByteString, _]), ByteString, NotUsed] - - /** - * Tar format - * @see - * https://en.wikipedia.org/wiki/Tar_(computing)#Limitations for the limitations - */ - final case object Tar extends ArchiveFormat[TarArchiveMetadata] { - override def contentType: ContentType = MediaTypes.`application/x-tar` - - override def fileExtension: String = "tar" - - override def metadata(filename: String, size: Long): TarArchiveMetadata = - TarArchiveMetadata.create(filename, size) - - override def filePath(metadata: TarArchiveMetadata): String = metadata.filePath - - override def writeFlow: WriteFlow[TarArchiveMetadata] = Archive.tar() - } - - /** - * Zip format - * - * @see - * https://en.wikipedia.org/wiki/ZIP_(file_format)#Limits for the limitations - */ - final case object Zip extends ArchiveFormat[ArchiveMetadata] { - override def contentType: ContentType = MediaTypes.`application/zip` - - override def fileExtension: String = "zip" - - override def metadata(filename: String, size: Long): ArchiveMetadata = - ArchiveMetadata.create(filename) - - override def filePath(metadata: ArchiveMetadata): String = metadata.filePath - - override def writeFlow: WriteFlow[ArchiveMetadata] = Archive.zip() - } - - private val availableFormats = List(Tar, Zip) - - def apply(req: HttpRequest): Option[ArchiveFormat[_]] = availableFormats.find { format => - HeadersUtils.matches(req.headers, format.contentType.mediaType) - } -} diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveRejection.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveRejection.scala index f278888d2d..49637d95b6 100644 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveRejection.scala +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/ArchiveRejection.scala @@ -69,11 +69,6 @@ object ArchiveRejection { )).mkString("\n") ) - final case class FilenameTooLong(id: Iri, project: ProjectRef, fileName: String) - extends ArchiveRejection( - s"File '$id' in project '$project' has a file name '$fileName' exceeding the 100 character limit for a tar file." - ) - /** * Rejection returned when an archive doesn't exist. * @@ -201,7 +196,6 @@ object ArchiveRejection { HttpResponseFields { case ResourceAlreadyExists(_, _) => StatusCodes.Conflict case InvalidResourceCollection(_, _, _) => StatusCodes.BadRequest - case FilenameTooLong(_, _, _) => StatusCodes.BadRequest case ArchiveNotFound(_, _) => StatusCodes.NotFound case InvalidArchiveId(_) => StatusCodes.BadRequest case ProjectContextRejection(rejection) => rejection.status diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/Zip.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/Zip.scala new file mode 100644 index 0000000000..b09f21a5de --- /dev/null +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/model/Zip.scala @@ -0,0 +1,33 @@ +package ch.epfl.bluebrain.nexus.delta.plugins.archive.model + +import akka.NotUsed +import akka.http.scaladsl.model.{ContentType, HttpRequest, MediaTypes} +import akka.stream.alpakka.file.scaladsl.Archive +import akka.stream.alpakka.file.ArchiveMetadata +import akka.stream.scaladsl.{Flow, Source} +import akka.util.ByteString +import ch.epfl.bluebrain.nexus.delta.sdk.utils.HeadersUtils + +/** + * Zip archive format + * + * @see + * https://en.wikipedia.org/wiki/ZIP_(file_format)#Limits for the limitations + */ +object Zip { + type WriteFlow[Metadata] = Flow[(Metadata, Source[ByteString, _]), ByteString, NotUsed] + + def apply(req: HttpRequest): Option[Zip.type] = + if (HeadersUtils.matches(req.headers, Zip.contentType.mediaType)) Some(Zip) else None + + def contentType: ContentType = MediaTypes.`application/zip` + + def fileExtension: String = "zip" + + def metadata(filename: String): ArchiveMetadata = + ArchiveMetadata.create(filename) + + def writeFlow: WriteFlow[ArchiveMetadata] = Archive.zip() + + def ordering: Ordering[ArchiveMetadata] = Ordering.by(md => md.filePath) +} diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/routes/ArchiveRoutes.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/routes/ArchiveRoutes.scala index 7153dcc6ce..960faab676 100644 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/routes/ArchiveRoutes.scala +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/routes/ArchiveRoutes.scala @@ -4,7 +4,8 @@ import akka.http.scaladsl.model.StatusCodes.{Created, SeeOther} import akka.http.scaladsl.server.Directives._ import akka.http.scaladsl.server.{Directive1, Route} import ch.epfl.bluebrain.nexus.delta.plugins.archive.Archives -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.{permissions, ArchiveFormat} +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.permissions +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.Zip import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolution import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering import ch.epfl.bluebrain.nexus.delta.sdk.AkkaSource @@ -76,14 +77,14 @@ class ArchiveRoutes( (get & pathEndOrSingleSlash) { authorizeFor(ref, permissions.read).apply { archiveResponse { - case Some(format) => + case Some(_) => parameter("ignoreNotFound".as[Boolean] ? false) { ignoreNotFound => - val response = archives.download(id, ref, format, ignoreNotFound).map { source => - sourceToFileResponse(source, format) + val response = archives.download(id, ref, ignoreNotFound).map { source => + sourceToFileResponse(source) } emit(response) } - case None => emit(archives.fetch(id, ref)) + case None => emit(archives.fetch(id, ref)) } } } @@ -96,9 +97,9 @@ class ArchiveRoutes( } } - private def sourceToFileResponse(source: AkkaSource, format: ArchiveFormat[_]): FileResponse = - FileResponse(s"archive.${format.fileExtension}", format.contentType, 0L, source) + private def sourceToFileResponse(source: AkkaSource): FileResponse = + FileResponse(s"archive.${Zip.fileExtension}", Zip.contentType, 0L, source) - private def archiveResponse: Directive1[Option[ArchiveFormat[_]]] = - extractRequest.map(ArchiveFormat(_)) + private def archiveResponse: Directive1[Option[Zip.type]] = + extractRequest.map(Zip(_)) } diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownloadSpec.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownloadSpec.scala index a549e747de..f9337095e4 100644 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownloadSpec.scala +++ b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveDownloadSpec.scala @@ -10,9 +10,10 @@ import cats.data.NonEmptySet import ch.epfl.bluebrain.nexus.delta.kernel.utils.UrlUtils.encode import ch.epfl.bluebrain.nexus.delta.plugins.archive.FileSelf.ParsingError import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveReference.{FileReference, FileSelfReference, ResourceReference} -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveRejection.{AuthorizationFailed, FilenameTooLong, InvalidFileSelf, ResourceNotFound} +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveRejection.{AuthorizationFailed, InvalidFileSelf, ResourceNotFound} import ch.epfl.bluebrain.nexus.delta.sdk.model.ResourceRepresentation.{CompactedJsonLd, Dot, ExpandedJsonLd, NQuads, NTriples, SourceJson} -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.{ArchiveFormat, ArchiveRejection, ArchiveValue} +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.{ArchiveRejection, ArchiveValue} +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.Zip import ch.epfl.bluebrain.nexus.delta.plugins.storage.RemoteContextResolutionFixture import ch.epfl.bluebrain.nexus.delta.plugins.storage.files.model.FileAttributes.FileAttributesOrigin.Client import ch.epfl.bluebrain.nexus.delta.plugins.storage.files.model.FileRejection.FileNotFound @@ -50,7 +51,7 @@ import java.util.UUID import scala.concurrent.ExecutionContext import scala.reflect.ClassTag -abstract class ArchiveDownloadSpec +class ArchiveDownloadSpec extends TestKit(ActorSystem()) with AnyWordSpecLike with Inspectors @@ -81,9 +82,7 @@ abstract class ArchiveDownloadSpec private val permissions = Set(Permissions.resources.read) private val aclCheck = AclSimpleCheck((subject, AclAddress.Root, permissions)).accepted - def format: ArchiveFormat[_] - - def sourceToMap(source: AkkaSource): Map[String, String] + def sourceToMap(source: AkkaSource): Map[String, String] = fromZip(source).map { case (k, v) => k -> v.utf8String } "An ArchiveDownload" should { val storageRef = ResourceRef.Revision(iri"http://localhost/${genString()}", 5) @@ -147,20 +146,20 @@ abstract class ArchiveDownloadSpec ) def downloadAndExtract(value: ArchiveValue, ignoreNotFound: Boolean) = { - archiveDownload(value, project.ref, format, ignoreNotFound).map(sourceToMap).accepted + archiveDownload(value, project.ref, ignoreNotFound).map(sourceToMap).accepted } def failToDownload[R <: ArchiveRejection: ClassTag](value: ArchiveValue, ignoreNotFound: Boolean) = { - archiveDownload(value, project.ref, format, ignoreNotFound).rejectedWith[R] + archiveDownload(value, project.ref, ignoreNotFound).rejectedWith[R] } def rejectedAccess(value: ArchiveValue) = { archiveDownload - .apply(value, project.ref, format, ignoreNotFound = true)(Caller.Anonymous, global) + .apply(value, project.ref, ignoreNotFound = true)(Caller.Anonymous, global) .rejectedWith[AuthorizationFailed] } - s"provide a ${format.fileExtension} for both resources and files" in { + s"provide a ${Zip.fileExtension} for both resources and files" in { val value = ArchiveValue.unsafe( NonEmptySet.of( ResourceReference(Latest(id1), None, None, None), @@ -175,7 +174,7 @@ abstract class ArchiveDownloadSpec result shouldEqual expected } - s"provide a ${format.fileExtension} for file selfs" in { + s"provide a ${Zip.fileExtension} for file selfs" in { val value = ArchiveValue.unsafe( NonEmptySet.of( FileSelfReference(file1Self, None) @@ -188,12 +187,12 @@ abstract class ArchiveDownloadSpec result shouldEqual expected } - s"fail to provide a ${format.fileExtension} for file selfs which do not resolve" in { + s"fail to provide a ${Zip.fileExtension} for file selfs which do not resolve" in { val value = ArchiveValue.unsafe(NonEmptySet.of(FileSelfReference("http://wrong.file/self", None))) failToDownload[InvalidFileSelf](value, ignoreNotFound = false) } - s"provide a ${format.fileExtension} for both resources and files with different paths and formats" in { + s"provide a ${Zip.fileExtension} for both resources and files with different paths and formats" in { val list = List( SourceJson -> file1.value.asJson.sort.spaces2, CompactedJsonLd -> file1.toCompactedJsonLd.accepted.json.sort.spaces2, @@ -226,39 +225,17 @@ abstract class ArchiveDownloadSpec } } - if (format == ArchiveFormat.Tar) { - "fail to provide a tar if the file name is too long and no path is provided" in { - val value = ArchiveValue.unsafe( - NonEmptySet.of( - FileReference(Latest(id2), None, None) - ) - ) - failToDownload[FilenameTooLong](value, ignoreNotFound = false) - } - - "provide a tar if the file name is too long but a path is provided" in { - val filePath = AbsolutePath.apply(s"/${genString()}/file.txt").rightValue - val value = ArchiveValue.unsafe( - NonEmptySet.of( - FileReference(Latest(id2), None, Some(filePath)) - ) - ) - - downloadAndExtract(value, ignoreNotFound = false) should contain key filePath.value.toString - } - } else { - "provide a zip if the file name is long" in { - val value = ArchiveValue.unsafe( - NonEmptySet.of( - FileReference(Latest(id2), None, None) - ) + "provide a zip if the file name is long" in { + val value = ArchiveValue.unsafe( + NonEmptySet.of( + FileReference(Latest(id2), None, None) ) - val file2Path = s"${project.ref.toString}/file/${file2.value.attributes.filename}" - downloadAndExtract(value, ignoreNotFound = false) should contain key file2Path - } + ) + val file2Path = s"${project.ref.toString}/file/${file2.value.attributes.filename}" + downloadAndExtract(value, ignoreNotFound = false) should contain key file2Path } - s"fail to provide a ${format.fileExtension} when a resource is not found" in { + s"fail to provide a ${Zip.fileExtension} when a resource is not found" in { val value = ArchiveValue.unsafe( NonEmptySet.of( ResourceReference(Latest(iri"http://localhost/${genString()}"), None, None, None), @@ -268,7 +245,7 @@ abstract class ArchiveDownloadSpec failToDownload[ResourceNotFound](value, ignoreNotFound = false) } - s"fail to provide a ${format.fileExtension} when a file is not found" in { + s"fail to provide a ${Zip.fileExtension} when a file is not found" in { val value = ArchiveValue.unsafe( NonEmptySet.of( ResourceReference(Latest(id1), None, None, None), @@ -306,14 +283,14 @@ abstract class ArchiveDownloadSpec result shouldEqual expected } - s"fail to provide a ${format.fileExtension} when access to a resource is not found" in { + s"fail to provide a ${Zip.fileExtension} when access to a resource is not found" in { val value = ArchiveValue.unsafe( NonEmptySet.of(ResourceReference(Latest(id1), None, None, None)) ) rejectedAccess(value) } - s"fail to provide a ${format.fileExtension} when access to a file is not found" in { + s"fail to provide a ${Zip.fileExtension} when access to a file is not found" in { val value = ArchiveValue.unsafe( NonEmptySet.of(FileReference(Latest(id1), None, None)) ) diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveRoutesSpec.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveRoutesSpec.scala index 1081083c4a..15da49e499 100644 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveRoutesSpec.scala +++ b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchiveRoutesSpec.scala @@ -2,7 +2,7 @@ package ch.epfl.bluebrain.nexus.delta.plugins.archive import akka.http.scaladsl.model.ContentTypes.`text/plain(UTF-8)` import akka.http.scaladsl.model.MediaRanges.`*/*` -import akka.http.scaladsl.model.MediaTypes.{`application/x-tar`, `application/zip`} +import akka.http.scaladsl.model.MediaTypes.`application/zip` import akka.http.scaladsl.model.headers.{`Content-Type`, Accept, Location, OAuth2BearerToken} import akka.http.scaladsl.model.{ContentTypes, StatusCodes, Uri} import akka.http.scaladsl.server.Route @@ -286,36 +286,6 @@ class ArchiveRoutesSpec extends BaseRouteSpec with StorageFixtures with TryValue } } - "fetch a tar archive ignoring not found" in { - forAll(List(Accept(`application/x-tar`), acceptAll)) { accept => - Get(s"/v1/archives/$projectRef/$uuid?ignoreNotFound=true") ~> asSubject ~> accept ~> routes ~> check { - status shouldEqual StatusCodes.OK - header[`Content-Type`].value.value() shouldEqual `application/x-tar`.value - val result = fromTar(responseEntity.dataBytes) - - result.keySet shouldEqual Set( - s"${project.ref}/file/file.txt", - s"${project.ref}/compacted/${encode(fileId.toString)}.json" - ) - - val expectedContent = fileContent - val actualContent = result.entryAsString(s"${project.ref}/file/file.txt") - actualContent shouldEqual expectedContent - - val expectedMetadata = FilesRoutesSpec.fileMetadata( - projectRef, - fileId, - file.value.attributes, - storageRef, - createdBy = subject, - updatedBy = subject - ) - val actualMetadata = result.entryAsJson(s"${project.ref}/compacted/${encode(fileId.toString)}.json") - actualMetadata shouldEqual expectedMetadata - } - } - } - "fetch a zip archive ignoring not found" in { Get(s"/v1/archives/$projectRef/$uuid?ignoreNotFound=true") ~> asSubject ~> Accept( `application/zip` diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivesSpec.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivesSpec.scala index 5d1c49abc5..273d3ee9dd 100644 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivesSpec.scala +++ b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivesSpec.scala @@ -5,7 +5,7 @@ import cats.data.NonEmptySet import ch.epfl.bluebrain.nexus.delta.kernel.utils.UUIDF import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveReference.{FileReference, ResourceReference} import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveRejection.{ArchiveNotFound, ProjectContextRejection} -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.{Archive, ArchiveFormat, ArchiveRejection, ArchiveValue} +import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.{Archive, ArchiveRejection, ArchiveValue} import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.{nxv, schema} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdJavaApi} import ch.epfl.bluebrain.nexus.delta.sdk.AkkaSource @@ -63,8 +63,7 @@ class ArchivesSpec private val cfg = ArchivePluginConfig(1, EphemeralLogConfig(5.seconds, 5.hours)) private val download = new ArchiveDownload { - override def apply[M](value: ArchiveValue, project: ProjectRef, format: ArchiveFormat[M], ignoreNotFound: Boolean)( - implicit + override def apply(value: ArchiveValue, project: ProjectRef, ignoreNotFound: Boolean)(implicit caller: Caller, scheduler: Scheduler ): IO[ArchiveRejection, AkkaSource] = @@ -250,7 +249,7 @@ class ArchivesSpec resource.value shouldEqual Archive(id, project.ref, value.resources, 5.hours.toSeconds) } - "download an existing archive as zip and tar" in { + "download an existing archive as zip" in { val id = iri"http://localhost/base/${genString()}" val resourceId = iri"http://localhost/${genString()}" val fileId = iri"http://localhost/${genString()}" @@ -261,8 +260,7 @@ class ArchivesSpec ) ) archives.create(id, project.ref, value).accepted - archives.download(id, project.ref, ArchiveFormat.Tar, ignoreNotFound = true).accepted - archives.download(id, project.ref, ArchiveFormat.Zip, ignoreNotFound = true).accepted + archives.download(id, project.ref, ignoreNotFound = true).accepted } "return not found for unknown archives" in { diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/TarDownloadSpec.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/TarDownloadSpec.scala deleted file mode 100644 index 456b4bc266..0000000000 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/TarDownloadSpec.scala +++ /dev/null @@ -1,15 +0,0 @@ -package ch.epfl.bluebrain.nexus.delta.plugins.archive -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveFormat -import ch.epfl.bluebrain.nexus.delta.sdk.AkkaSource - -import scala.concurrent.duration.DurationInt - -class TarDownloadSpec extends ArchiveDownloadSpec { - - implicit override def patienceConfig: PatienceConfig = PatienceConfig(3.seconds, 10.millis) - override def format: ArchiveFormat[_] = ArchiveFormat.Tar - - override def sourceToMap(source: AkkaSource): Map[String, String] = - fromTar(source).map { case (k, v) => k -> v.utf8String } - -} diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ZipDownloadSpec.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ZipDownloadSpec.scala deleted file mode 100644 index dafffbb2c9..0000000000 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ZipDownloadSpec.scala +++ /dev/null @@ -1,12 +0,0 @@ -package ch.epfl.bluebrain.nexus.delta.plugins.archive - -import ch.epfl.bluebrain.nexus.delta.plugins.archive.model.ArchiveFormat -import ch.epfl.bluebrain.nexus.delta.sdk.AkkaSource - -class ZipDownloadSpec extends ArchiveDownloadSpec { - override def format: ArchiveFormat[_] = ArchiveFormat.Zip - - override def sourceToMap(source: AkkaSource): Map[String, String] = - fromZip(source).map { case (k, v) => k -> v.utf8String } - -} diff --git a/delta/testkit/src/main/scala/ch/epfl/bluebrain/nexus/testkit/archive/ArchiveHelpers.scala b/delta/testkit/src/main/scala/ch/epfl/bluebrain/nexus/testkit/archive/ArchiveHelpers.scala index 7f281024d6..081af7900d 100644 --- a/delta/testkit/src/main/scala/ch/epfl/bluebrain/nexus/testkit/archive/ArchiveHelpers.scala +++ b/delta/testkit/src/main/scala/ch/epfl/bluebrain/nexus/testkit/archive/ArchiveHelpers.scala @@ -30,31 +30,6 @@ trait ArchiveHelpers extends ScalaFutures with EitherValuable with OptionValues } } - def fromTar(byteString: ByteString)(implicit m: Materializer, e: ExecutionContext): ArchiveContent = - fromTar(Source.single(byteString)) - - def fromTar(source: Source[ByteString, Any])(implicit m: Materializer, e: ExecutionContext): ArchiveContent = { - val path = JFiles.createTempFile("test", ".tar") - source.runWith(FileIO.toPath(path)).futureValue - val result = FileIO - .fromPath(path) - .via(Archive.tarReader()) - .mapAsync(1) { case (metadata, source) => - source - .runFold(ByteString.empty) { case (bytes, elem) => - bytes ++ elem - } - .map { bytes => - (metadata.filePath, bytes) - } - } - .runFold(Map.empty[String, ByteString]) { case (map, elem) => - map + elem - } - .futureValue - result - } - def fromZip(byteString: ByteString)(implicit m: Materializer, e: ExecutionContext): ArchiveContent = fromZip(Source.single(byteString)) diff --git a/tests/src/test/scala/ch/epfl/bluebrain/nexus/tests/kg/ArchiveSpec.scala b/tests/src/test/scala/ch/epfl/bluebrain/nexus/tests/kg/ArchiveSpec.scala index 6c5b5df548..90a41cec9a 100644 --- a/tests/src/test/scala/ch/epfl/bluebrain/nexus/tests/kg/ArchiveSpec.scala +++ b/tests/src/test/scala/ch/epfl/bluebrain/nexus/tests/kg/ArchiveSpec.scala @@ -231,25 +231,6 @@ class ArchiveSpec extends BaseSpec with ArchiveHelpers with CirceEq { } } - "succeed returning tar" in { - val prefix = "https%3A%2F%2Fdev.nexus.test.com%2Fsimplified-resource%2F" - deltaClient.get[ByteString](s"/archives/$fullId/test-resource:archive", Tweety, acceptAll) { - (byteString, response) => - contentType(response) shouldEqual MediaTypes.`application/x-tar`.toContentType - response.status shouldEqual StatusCodes.OK - - val result = fromTar(byteString) - - val actualContent1 = result.entryAsJson(s"$fullId/compacted/${prefix}1%3Frev%3D1.json") - val actualContent2 = result.entryAsJson(s"$fullId2/compacted/${prefix}2.json") - val actualDigest3 = result.entryDigest("/some/other/nexus-logo.png") - - filterMetadataKeys(actualContent1) should equalIgnoreArrayOrder(payloadResponse1) - filterMetadataKeys(actualContent2) should equalIgnoreArrayOrder(payloadResponse2) - actualDigest3 shouldEqual nexusLogoDigest - } - } - "succeed returning zip" in { val prefix = "https%3A%2F%2Fdev.nexus.test.com%2Fsimplified-resource%2F" deltaClient.get[ByteString](s"/archives/$fullId/test-resource:archive", Tweety, acceptZip) { @@ -300,11 +281,6 @@ class ArchiveSpec extends BaseSpec with ArchiveHelpers with CirceEq { response.status shouldEqual StatusCodes.Created } downloadLink = s"/archives/$fullId/test-resource:archive-not-found?ignoreNotFound=true" - _ <- deltaClient.get[ByteString](downloadLink, Tweety, acceptAll) { (byteString, response) => - contentType(response) shouldEqual MediaTypes.`application/x-tar`.toContentType - response.status shouldEqual StatusCodes.OK - assertContent(fromTar(byteString)) - } _ <- deltaClient.get[ByteString](downloadLink, Tweety, acceptZip) { (byteString, response) => contentType(response) shouldEqual MediaTypes.`application/zip`.toContentType response.status shouldEqual StatusCodes.OK