Implement byte-based file operations (#1795)

This commit is contained in:
Ara Adkins 2021-06-11 14:48:28 +01:00 committed by GitHub
parent 0b363e3e85
commit 90c020d666
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
30 changed files with 2763 additions and 794 deletions

View File

@ -36,6 +36,9 @@
verify the integrity of files that it has transferred. The checksum is verify the integrity of files that it has transferred. The checksum is
calculated in a streaming fashion so the checksummed file need not be resident calculated in a streaming fashion so the checksummed file need not be resident
in memory all at once. in memory all at once.
- Added support for reading and writing byte ranges in files remotely
([#1795](https://github.com/enso-org/enso/pull/1795)). This allows the IDE to
transfer files to a remote back-end in a streaming fashion.
## Libraries ## Libraries

View File

@ -1644,8 +1644,12 @@ This method will create a file if no file is present at `path`.
length of the file. length of the file.
- The `byteOffset` property is zero-indexed. To append to the file you begin - The `byteOffset` property is zero-indexed. To append to the file you begin
writing at index `file.length`. writing at index `file.length`.
- If `byteOffset` is less than the length of the file and `overwriteExisting` is
set, it will truncate the file to length `byteOffset + bytes.length`.
- If `byteOffset > file.length`, the bytes in the range - If `byteOffset > file.length`, the bytes in the range
`[file.length, byteOffset)` will be filled with null bytes. `[file.length, byteOffset)` will be filled with null bytes. Please note that,
in this case, the checksum in the response will also be calculated on the null
bytes.
#### Parameters #### Parameters
@ -4055,7 +4059,7 @@ Signals that the requested file read was out of bounds for the file's size.
"code" : 1009 "code" : 1009
"message" : "Read is out of bounds for the file" "message" : "Read is out of bounds for the file"
"data" : { "data" : {
fileLength : 0 "fileLength" : 0
} }
} }
``` ```

View File

@ -23,6 +23,7 @@ A few key requirements:
<!-- MarkdownTOC levels="2,3" autolink="true" indent=" " --> <!-- MarkdownTOC levels="2,3" autolink="true" indent=" " -->
- [Control](#control) - [Control](#control)
- [Concurrency](#concurrency)
- [UX](#ux) - [UX](#ux)
<!-- /MarkdownTOC --> <!-- /MarkdownTOC -->
@ -50,6 +51,19 @@ used.
Resumption of transfers is also handled by the IDE, which may keep track of what Resumption of transfers is also handled by the IDE, which may keep track of what
portions of a file have been written or read. portions of a file have been written or read.
### Concurrency
The language server natively supports running these file operations in parallel
as it spawns a separate request-handler actor for each operation. It does,
however, not provide any _intrinsic_ guarantees to its operation. As _all_ file
operations are evaluated in parallel, coordinating them for consistency is up to
the IDE.
For example, if you want to write bytes to a file `f1` and then checksum the
resulting file, you need to wait for the `WriteBytesReply` to come back before
sending `file/checksum(f1)`. Otherwise, there is no guarantee that the write has
completed by the time the checksum is calculated.
## UX ## UX
The IDE wants to be able to provide two major UX benefits to users as part of The IDE wants to be able to provide two major UX benefits to users as part of

View File

@ -4,6 +4,7 @@ import akka.actor.{Actor, Props}
import akka.routing.SmallestMailboxPool import akka.routing.SmallestMailboxPool
import akka.pattern.pipe import akka.pattern.pipe
import com.typesafe.scalalogging.LazyLogging import com.typesafe.scalalogging.LazyLogging
import org.bouncycastle.util.encoders.Hex
import org.enso.languageserver.effect._ import org.enso.languageserver.effect._
import org.enso.languageserver.data.Config import org.enso.languageserver.data.Config
import org.enso.languageserver.monitoring.MonitoringProtocol.{Ping, Pong} import org.enso.languageserver.monitoring.MonitoringProtocol.{Ping, Pong}
@ -195,14 +196,48 @@ class FileManager(
.pipeTo(sender()) .pipeTo(sender())
() ()
case FileManagerProtocol.ChecksumRequest(path) => case FileManagerProtocol.ChecksumFileRequest(path) =>
val getChecksum = for { val getChecksum = for {
rootPath <- IO.fromEither(config.findContentRoot(path.rootId)) rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
checksum <- fs.digest(path.toFile(rootPath)) checksum <- fs.digest(path.toFile(rootPath))
} yield checksum } yield checksum
exec exec
.execTimed(config.fileManager.timeout, getChecksum) .execTimed(config.fileManager.timeout, getChecksum)
.map(FileManagerProtocol.ChecksumResponse) .map(x =>
FileManagerProtocol.ChecksumFileResponse(
x.map(digest => Hex.toHexString(digest.bytes))
)
)
.pipeTo(sender())
case FileManagerProtocol.ChecksumBytesRequest(segment) =>
val getChecksum = for {
rootPath <- IO.fromEither(config.findContentRoot(segment.path.rootId))
checksum <- fs.digestBytes(segment.toApiSegment(rootPath))
} yield checksum
exec
.execTimed(config.fileManager.timeout, getChecksum)
.map(x => FileManagerProtocol.ChecksumBytesResponse(x.map(_.bytes)))
.pipeTo(sender())
case FileManagerProtocol.WriteBytesRequest(path, off, overwrite, bytes) =>
val doWrite = for {
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
response <- fs.writeBytes(path.toFile(rootPath), off, overwrite, bytes)
} yield response
exec
.execTimed(config.fileManager.timeout, doWrite)
.map(x => FileManagerProtocol.WriteBytesResponse(x.map(_.bytes)))
.pipeTo(sender())
case FileManagerProtocol.ReadBytesRequest(segment) =>
val doRead = for {
rootPath <- IO.fromEither(config.findContentRoot(segment.path.rootId))
response <- fs.readBytes(segment.toApiSegment(rootPath))
} yield response
exec
.execTimed(config.fileManager.timeout, doRead)
.map(FileManagerProtocol.ReadBytesResponse)
.pipeTo(sender()) .pipeTo(sender())
} }
} }

View File

@ -1,5 +1,7 @@
package org.enso.languageserver.filemanager package org.enso.languageserver.filemanager
import io.circe.Json
import io.circe.literal.JsonStringContext
import org.enso.jsonrpc.{Error, HasParams, HasResult, Method, Unused} import org.enso.jsonrpc.{Error, HasParams, HasResult, Method, Unused}
/** The file manager JSON RPC API provided by the language server. /** The file manager JSON RPC API provided by the language server.
@ -177,6 +179,19 @@ object FileManagerApi {
case object NotFileError extends Error(1007, "Path is not a file") case object NotFileError extends Error(1007, "Path is not a file")
case object CannotOverwriteError
extends Error(
1008,
"Cannot overwrite the file without `overwriteExisting` set"
)
case class ReadOutOfBoundsError(length: Long)
extends Error(1009, "Read is out of bounds for the file") {
override def payload: Option[Json] = Some(
json""" { "fileLength" : $length }"""
)
}
case object CannotDecodeError case object CannotDecodeError
extends Error(1010, "Cannot decode the project configuration") extends Error(1010, "Cannot decode the project configuration")

View File

@ -182,12 +182,82 @@ object FileManagerProtocol {
* *
* @param path to the file system object * @param path to the file system object
*/ */
case class ChecksumRequest(path: Path) case class ChecksumFileRequest(path: Path)
/** Returns the checksum of the file system object in question. /** Returns the checksum of the file system object in question.
* *
* @param checksum either a FS failure or the checksum as a base64-encoded * @param checksum either a FS failure or the checksum as a base64-encoded
* string * string
*/ */
case class ChecksumResponse(checksum: Either[FileSystemFailure, String]) case class ChecksumFileResponse(checksum: Either[FileSystemFailure, String])
/** Requests that the file manager provide the checksum of the specified bytes
* in a file.
*
* @param segment a description of the bytes in a file to checksum.
*/
case class ChecksumBytesRequest(segment: Data.FileSegment)
/** Returns the checksum of the bytes in question.
*
* @param checksum either a FS failure or the checksum as an array of bytes
*/
case class ChecksumBytesResponse(
checksum: Either[FileSystemFailure, Array[Byte]]
)
/** Requests that the file manager writes the provided `bytes` to the file at
* `path`.
*
* @param path the file to write to
* @param byteOffset the offset in the file to begin writing from
* @param overwriteExisting whether or not the request can overwrite existing
* data
* @param bytes the bytes to write
*/
case class WriteBytesRequest(
path: Path,
byteOffset: Long,
overwriteExisting: Boolean,
bytes: Array[Byte]
)
/** Returns the checksum of the bytes that were written to disk.
*
* @param checksum either a FS failure or the checksum as an array of bytes
*/
case class WriteBytesResponse(
checksum: Either[FileSystemFailure, Array[Byte]]
)
/** Requests to read the bytes in the file identified by `segment`.
*
* @param segment an identification of where the bytes should be read from
*/
case class ReadBytesRequest(segment: Data.FileSegment)
/** Returns the requested bytes and their checksum.
*
* @param result either a FS failure or the checksum and corresponding bytes
* that were read
*/
case class ReadBytesResponse(
result: Either[FileSystemFailure, FileSystemApi.ReadBytesResult]
)
/** Data types for the protocol. */
object Data {
/** A representation of a segment in the file.
*
* @param path the path to the file in question
* @param byteOffset the byte offset in the file to start from
* @param length the number of bytes in the segment
*/
case class FileSegment(path: Path, byteOffset: Long, length: Long) {
def toApiSegment(rootPath: File): FileSystemApi.FileSegment = {
FileSystemApi.FileSegment(path.toFile(rootPath), byteOffset, length)
}
}
}
} }

View File

@ -1,12 +1,12 @@
package org.enso.languageserver.filemanager package org.enso.languageserver.filemanager
import org.apache.commons.io.{FileExistsException, FileUtils} import org.apache.commons.io.{FileExistsException, FileUtils}
import org.bouncycastle.util.encoders.Hex
import org.enso.languageserver.effect.BlockingIO import org.enso.languageserver.effect.BlockingIO
import zio._ import zio._
import zio.blocking.effectBlocking import zio.blocking.effectBlocking
import java.io.{File, FileNotFoundException} import java.io.{File, FileNotFoundException, RandomAccessFile}
import java.nio.ByteBuffer
import java.nio.file._ import java.nio.file._
import java.nio.file.attribute.BasicFileAttributes import java.nio.file.attribute.BasicFileAttributes
import java.security.MessageDigest import java.security.MessageDigest
@ -14,11 +14,14 @@ import scala.collection.mutable
import scala.util.Using import scala.util.Using
/** File manipulation facility. /** File manipulation facility.
*
* @tparam F represents target monad
*/ */
class FileSystem extends FileSystemApi[BlockingIO] { class FileSystem extends FileSystemApi[BlockingIO] {
private val tenMb: Int = 1 * 1024 * 1024 * 10
/** The stride used by the [[FileSystem]] when processing a file in chunks. */
val fileChunkSize: Int = tenMb
import FileSystemApi._ import FileSystemApi._
/** Writes textual content to a file. /** Writes textual content to a file.
@ -228,22 +231,157 @@ class FileSystem extends FileSystemApi[BlockingIO] {
* @param path the path to the filesystem object * @param path the path to the filesystem object
* @return either [[FileSystemFailure]] or the file checksum * @return either [[FileSystemFailure]] or the file checksum
*/ */
override def digest(path: File): BlockingIO[FileSystemFailure, String] = { override def digest(path: File): BlockingIO[FileSystemFailure, SHA3_224] = {
if (path.isFile) { if (path.isFile) {
effectBlocking { effectBlocking {
val messageDigest = MessageDigest.getInstance("SHA3-224") val messageDigest = MessageDigest.getInstance("SHA3-224")
Using.resource( Using.resource(
Files.newInputStream(path.toPath, StandardOpenOption.READ) Files.newInputStream(path.toPath, StandardOpenOption.READ)
) { stream => ) { stream =>
val tenMb = 1 * 1024 * 1024 * 10 var currentBytes = stream.readNBytes(fileChunkSize)
var currentBytes = stream.readNBytes(tenMb)
while (currentBytes.nonEmpty) { while (currentBytes.nonEmpty) {
messageDigest.update(currentBytes) messageDigest.update(currentBytes)
currentBytes = stream.readNBytes(tenMb) currentBytes = stream.readNBytes(fileChunkSize)
} }
Hex.toHexString(messageDigest.digest()) SHA3_224(messageDigest.digest())
}
}.mapError(errorHandling)
} else {
if (path.exists()) {
IO.fail(NotFile)
} else {
IO.fail(FileNotFound)
}
}
}
/** Returns the digest of the bytes described by `segment`.
*
* @param segment a description of the portion of a file to checksum
* @return either [[FileSystemFailure]] or the bytes representing the checksum
*/
override def digestBytes(
segment: FileSegment
): BlockingIO[FileSystemFailure, SHA3_224] = {
val path = segment.path
if (path.isFile) {
effectBlocking {
val messageDigest = MessageDigest.getInstance("SHA3-224")
Using.resource(
Files.newInputStream(path.toPath, StandardOpenOption.READ)
) { stream =>
val fileLength = Files.size(path.toPath)
val lastByteIndex = fileLength - 1
val lastSegIndex = segment.byteOffset + segment.length
if (segment.byteOffset > lastByteIndex || lastSegIndex > lastByteIndex) {
throw FileSystem.ReadOutOfBounds(fileLength)
}
var bytePosition = stream.skip(segment.byteOffset)
var bytesToRead = segment.length
do {
val readSize = Math.min(bytesToRead, fileChunkSize.toLong).toInt
val bytes = stream.readNBytes(readSize)
bytePosition += bytes.length
bytesToRead -= bytes.length
messageDigest.update(bytes)
} while (bytesToRead > 0)
SHA3_224(messageDigest.digest())
}
}.mapError(errorHandling)
} else {
if (path.exists()) {
IO.fail(NotFile)
} else {
IO.fail(FileNotFound)
}
}
}
override def writeBytes(
path: File,
byteOffset: Long,
overwriteExisting: Boolean,
bytes: Array[Byte]
): BlockingIO[FileSystemFailure, SHA3_224] = {
if (path.isDirectory) {
IO.fail(NotFile)
} else {
effectBlocking {
Using.resource(new RandomAccessFile(path, "rw")) { file =>
Using.resource(file.getChannel) { chan =>
val lock = chan.lock()
try {
val fileSize = chan.size()
val messageDigest = MessageDigest.getInstance("SHA3-224")
if (byteOffset < fileSize) {
if (overwriteExisting) {
chan.truncate(byteOffset)
} else {
throw FileSystem.CannotOverwrite
}
} else if (byteOffset > fileSize) {
chan.position(fileSize)
var nullBytesLeft = byteOffset - fileSize
do {
val numBytesInRound =
Math.min(nullBytesLeft, fileChunkSize.toLong)
val bytes = Array.fill(numBytesInRound.toInt)(0x0.toByte)
val bytesBuf = ByteBuffer.wrap(bytes)
messageDigest.update(bytes)
chan.write(bytesBuf)
nullBytesLeft -= numBytesInRound
} while (nullBytesLeft > 0)
}
chan.position(chan.size())
messageDigest.update(bytes)
chan.write(ByteBuffer.wrap(bytes))
SHA3_224(messageDigest.digest())
} finally {
lock.release()
}
}
}
}.mapError(errorHandling)
}
}
override def readBytes(
segment: FileSegment
): BlockingIO[FileSystemFailure, ReadBytesResult] = {
val path = segment.path
if (path.isFile) {
effectBlocking {
Using.resource(
Files.newInputStream(path.toPath, StandardOpenOption.READ)
) { stream =>
stream.skip(segment.byteOffset)
val fileSize = Files.size(path.toPath)
val lastByteIndex = fileSize - 1
if (lastByteIndex < segment.byteOffset) {
throw FileSystem.ReadOutOfBounds(fileSize)
}
val bytesToRead = segment.length
val bytes = stream.readNBytes(bytesToRead.toInt)
val digest = MessageDigest.getInstance("SHA3-224").digest(bytes)
ReadBytesResult(SHA3_224(digest), bytes)
} }
}.mapError(errorHandling) }.mapError(errorHandling)
} else { } else {
@ -260,12 +398,25 @@ class FileSystem extends FileSystemApi[BlockingIO] {
case _: NoSuchFileException => FileNotFound case _: NoSuchFileException => FileNotFound
case _: FileExistsException => FileExists case _: FileExistsException => FileExists
case _: AccessDeniedException => AccessDenied case _: AccessDeniedException => AccessDenied
case FileSystem.ReadOutOfBounds(l) => ReadOutOfBounds(l)
case FileSystem.CannotOverwrite => CannotOverwrite
case ex => GenericFileSystemFailure(ex.getMessage) case ex => GenericFileSystemFailure(ex.getMessage)
} }
} }
object FileSystem { object FileSystem {
/** An exception for when a file segment read goes out of bounds.
*
* @param length the true length of the file
*/
case class ReadOutOfBounds(length: Long) extends Throwable
/** An exception for when overwriting would be required but the corresponding
* flag is not set.
*/
case object CannotOverwrite extends Throwable
import FileSystemApi._ import FileSystemApi._
/** Represent a depth limit when recursively traversing a directory. /** Represent a depth limit when recursively traversing a directory.

View File

@ -4,7 +4,6 @@ import java.io.File
import java.nio.file.Path import java.nio.file.Path
import java.nio.file.attribute.{BasicFileAttributes, FileTime} import java.nio.file.attribute.{BasicFileAttributes, FileTime}
import java.time.{OffsetDateTime, ZoneOffset} import java.time.{OffsetDateTime, ZoneOffset}
import org.enso.languageserver.effect.BlockingIO import org.enso.languageserver.effect.BlockingIO
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
@ -135,11 +134,62 @@ trait FileSystemApi[F[_, _]] {
* @param path the path to the filesystem object * @param path the path to the filesystem object
* @return either [[FileSystemFailure]] or the file checksum * @return either [[FileSystemFailure]] or the file checksum
*/ */
def digest(path: File): F[FileSystemFailure, String] def digest(path: File): F[FileSystemFailure, SHA3_224]
/** Returns the digest for the bytes in the file described by `segment`.
*
* @param segment a description of the portion of a file to checksum
* @return either [[FileSystemFailure]] or the bytes representing the checksum
*/
def digestBytes(segment: FileSegment): F[FileSystemFailure, SHA3_224]
/** Writes the provided `bytes` to the file at `path` on disk.
*
* @param path the path to the file into which the bytes will be written
* @param byteOffset the offset in the file to start writing from
* @param overwriteExisting whether existing bytes can be overwritten
* @param bytes the bytes to write to the file
* @return either a [[FileSystemFailure]] or the checksum of the `bytes` as
* they were written to disk
*/
def writeBytes(
path: File,
byteOffset: Long,
overwriteExisting: Boolean,
bytes: Array[Byte]
): F[FileSystemFailure, SHA3_224]
/** Reads the bytes specified by `segment` from the specified `segment.file`.
*
* @param segment a description of the portion of a file to checksum
* @return either [[FileSystemFailure]] or the bytes representing the checksum
*/
def readBytes(segment: FileSegment): F[FileSystemFailure, ReadBytesResult]
} }
object FileSystemApi { object FileSystemApi {
/** A SHA3-224 digest on the filesystem.
*
* @param bytes the bytes that represent the value of the digest
*/
case class SHA3_224(bytes: Array[Byte])
/** The bytes read from the file.
*
* @param checksum the checksum of `bytes`
* @param bytes the bytes that were read
*/
case class ReadBytesResult(checksum: SHA3_224, bytes: Array[Byte])
/** A representation of a segment in the file.
*
* @param path the path to the file in question
* @param byteOffset the byte offset in the file to start from
* @param length the number of bytes in the segment
*/
case class FileSegment(path: File, byteOffset: Long, length: Long)
/** An object representing abstract file system entry. /** An object representing abstract file system entry.
*/ */
sealed trait Entry { sealed trait Entry {

View File

@ -2,7 +2,11 @@ package org.enso.languageserver.filemanager
/** Represents file system failures. /** Represents file system failures.
*/ */
sealed trait FileSystemFailure sealed trait FileSystemFailure {
/** Tells whether the error has additional data. */
def hasData: Boolean = false
}
/** Informs that the requested content root cannot be found. /** Informs that the requested content root cannot be found.
*/ */
@ -31,8 +35,21 @@ case object NotDirectory extends FileSystemFailure
/** Signal that the provided path is not a file. */ /** Signal that the provided path is not a file. */
case object NotFile extends FileSystemFailure case object NotFile extends FileSystemFailure
/** Signals that the file cannot be overwritten. */
case object CannotOverwrite extends FileSystemFailure
/** Signals that the provided file cannot be read at the requested offset.
*
* @param fileLength the actual length of the file.
*/
case class ReadOutOfBounds(fileLength: Long) extends FileSystemFailure {
override def hasData: Boolean = true
}
/** Signals file system specific errors. /** Signals file system specific errors.
* *
* @param reason a reason of failure * @param reason a reason of failure
*/ */
case class GenericFileSystemFailure(reason: String) extends FileSystemFailure case class GenericFileSystemFailure(reason: String) extends FileSystemFailure {
override def hasData: Boolean = true
}

View File

@ -1,15 +1,7 @@
package org.enso.languageserver.filemanager package org.enso.languageserver.filemanager
import org.enso.languageserver.filemanager.FileManagerApi.{
ContentRootNotFoundError,
FileExistsError,
FileNotFoundError,
FileSystemError,
NotDirectoryError,
NotFileError,
OperationTimeoutError
}
import org.enso.jsonrpc.Error import org.enso.jsonrpc.Error
import org.enso.languageserver.filemanager.FileManagerApi._
import org.enso.languageserver.protocol.json.ErrorApi import org.enso.languageserver.protocol.json.ErrorApi
object FileSystemFailureMapper { object FileSystemFailureMapper {
@ -28,6 +20,8 @@ object FileSystemFailureMapper {
case OperationTimeout => OperationTimeoutError case OperationTimeout => OperationTimeoutError
case NotDirectory => NotDirectoryError case NotDirectory => NotDirectoryError
case NotFile => NotFileError case NotFile => NotFileError
case CannotOverwrite => CannotOverwriteError
case ReadOutOfBounds(l) => ReadOutOfBoundsError(l)
case GenericFileSystemFailure(reason) => FileSystemError(reason) case GenericFileSystemFailure(reason) => FileSystemError(reason)
} }

View File

@ -1,8 +1,5 @@
package org.enso.languageserver.protocol.binary package org.enso.languageserver.protocol.binary
import java.nio.ByteBuffer
import java.util.UUID
import akka.actor.{Actor, ActorRef, Props, Stash} import akka.actor.{Actor, ActorRef, Props, Stash}
import akka.http.scaladsl.model.RemoteAddress import akka.http.scaladsl.model.RemoteAddress
import com.google.flatbuffers.FlatBufferBuilder import com.google.flatbuffers.FlatBufferBuilder
@ -17,21 +14,14 @@ import org.enso.languageserver.http.server.BinaryWebSocketControlProtocol.{
OutboundStreamEstablished OutboundStreamEstablished
} }
import org.enso.languageserver.protocol.binary.BinaryConnectionController.InboundPayloadType import org.enso.languageserver.protocol.binary.BinaryConnectionController.InboundPayloadType
import org.enso.languageserver.protocol.binary.InboundPayload.{ import org.enso.languageserver.protocol.binary.InboundPayload._
INIT_SESSION_CMD,
READ_FILE_CMD,
WRITE_FILE_CMD
}
import org.enso.languageserver.protocol.binary.factory.{ import org.enso.languageserver.protocol.binary.factory.{
ErrorFactory, ErrorFactory,
OutboundMessageFactory, OutboundMessageFactory,
SuccessReplyFactory, SuccessReplyFactory,
VisualisationUpdateFactory VisualisationUpdateFactory
} }
import org.enso.languageserver.requesthandler.file.{ import org.enso.languageserver.requesthandler.file._
ReadBinaryFileHandler,
WriteBinaryFileHandler
}
import org.enso.languageserver.runtime.ContextRegistryProtocol.VisualisationUpdate import org.enso.languageserver.runtime.ContextRegistryProtocol.VisualisationUpdate
import org.enso.languageserver.session.BinarySession import org.enso.languageserver.session.BinarySession
import org.enso.languageserver.util.UnhandledLogging import org.enso.languageserver.util.UnhandledLogging
@ -42,6 +32,8 @@ import org.enso.languageserver.util.binary.DecodingFailure.{
GenericDecodingFailure GenericDecodingFailure
} }
import java.nio.ByteBuffer
import java.util.UUID
import scala.annotation.unused import scala.annotation.unused
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -204,6 +196,12 @@ class BinaryConnectionController(
WRITE_FILE_CMD -> WriteBinaryFileHandler WRITE_FILE_CMD -> WriteBinaryFileHandler
.props(requestTimeout, fileManager, outboundChannel), .props(requestTimeout, fileManager, outboundChannel),
READ_FILE_CMD -> ReadBinaryFileHandler READ_FILE_CMD -> ReadBinaryFileHandler
.props(requestTimeout, fileManager, outboundChannel),
CHECKSUM_BYTES_CMD -> ChecksumBytesHandler
.props(requestTimeout, fileManager, outboundChannel),
WRITE_BYTES_CMD -> WriteBytesHandler
.props(requestTimeout, fileManager, outboundChannel),
READ_BYTES_CMD -> ReadBytesHandler
.props(requestTimeout, fileManager, outboundChannel) .props(requestTimeout, fileManager, outboundChannel)
) )
} }

View File

@ -0,0 +1,39 @@
package org.enso.languageserver.protocol.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.{
ChecksumBytesReply,
EnsoUUID,
OutboundPayload
}
import java.nio.ByteBuffer
import java.util.UUID
object ChecksumBytesReplyFactory {
/** Creates a [[ChecksumBytesReply]] inside a [[FlatBufferBuilder]].
*
* @param checksum the checksum value for the reply
* @param correlationId an identifier used to correlate a response with a
* request
* @return a FlatBuffer representation of the reply
*/
def create(checksum: Array[Byte], correlationId: EnsoUUID): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val digestOffset = EnsoDigestFactory.create(checksum)
val replyOffset =
ChecksumBytesReply.createChecksumBytesReply(builder, digestOffset)
val outMsg = OutboundMessageFactory.create(
UUID.randomUUID(),
Some(correlationId),
OutboundPayload.CHECKSUM_BYTES_REPLY,
replyOffset
)
builder.finish(outMsg)
builder.dataBuffer()
}
}

View File

@ -0,0 +1,19 @@
package org.enso.languageserver.protocol.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.EnsoDigest
object EnsoDigestFactory {
/** Create a new EnsoDigest.
*
* @param bytes the bytes of the digest
* @param builder the flatbuffer builder in which the digest is created
* @return the offset of the digest in `builder`
*/
def create(bytes: Array[Byte])(implicit builder: FlatBufferBuilder): Int = {
val bytesOff = builder.createByteVector(bytes)
EnsoDigest.createEnsoDigest(builder, bytesOff)
}
}

View File

@ -7,7 +7,8 @@ import org.enso.languageserver.protocol.binary.{
EnsoUUID, EnsoUUID,
Error, Error,
ErrorPayload, ErrorPayload,
OutboundPayload OutboundPayload,
ReadOutOfBoundsError
} }
object ErrorFactory { object ErrorFactory {
@ -33,12 +34,41 @@ object ErrorFactory {
def createServiceError( def createServiceError(
maybeCorrelationId: Option[EnsoUUID] = None maybeCorrelationId: Option[EnsoUUID] = None
): ByteBuffer = ): ByteBuffer =
createGenericError(0, "Service error", maybeCorrelationId) createGenericError(
0,
"Service error",
maybeCorrelationId = maybeCorrelationId
)
/** Creates an error representing a read that is out of bounds in a file with
* length `actualLength`.
*
* @param actualLength the actual length of the file
* @param maybeCorrelationId an optional correlation ID for the error
* @return a FlatBuffer representation of the error
*/
def createReadOutOfBoundsError(
actualLength: Long,
maybeCorrelationId: Option[EnsoUUID] = None
): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val payloadData =
ReadOutOfBoundsError.createReadOutOfBoundsError(builder, actualLength)
createGenericErrorWithBuilder(
1009,
"Read is out of bounds for the file",
Some(ErrorData(ErrorPayload.READ_OOB, payloadData)),
maybeCorrelationId = maybeCorrelationId
)
}
/** Creates a generic error inside a [[FlatBufferBuilder]]. /** Creates a generic error inside a [[FlatBufferBuilder]].
* *
* @param code an error code * @param code an error code
* @param message an error textual message * @param message an error textual message
* @param data optional error payload
* @param maybeCorrelationId an optional correlation id used to correlate * @param maybeCorrelationId an optional correlation id used to correlate
* a response with a request * a response with a request
* @return an FlatBuffer representation of the created error * @return an FlatBuffer representation of the created error
@ -46,10 +76,40 @@ object ErrorFactory {
def createGenericError( def createGenericError(
code: Int, code: Int,
message: String, message: String,
data: Option[ErrorData] = None,
maybeCorrelationId: Option[EnsoUUID] = None maybeCorrelationId: Option[EnsoUUID] = None
): ByteBuffer = { ): ByteBuffer = {
implicit val builder = new FlatBufferBuilder(1024) implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val offset =
createGenericErrorWithBuilder(code, message, data, maybeCorrelationId)
}
/** Creates a generic error inside the provided [[FlatBufferBuilder]].
*
* @param code an error code
* @param message an error textual message
* @param data optional error payload
* @param maybeCorrelationId an optional correlation id used to correlate
* a response with a request
* @param builder the builder to use for creating the error
* @return a FlatBuffer representation of the created error
*/
def createGenericErrorWithBuilder(
code: Int,
message: String,
data: Option[ErrorData] = None,
maybeCorrelationId: Option[EnsoUUID] = None
)(implicit builder: FlatBufferBuilder): ByteBuffer = {
val offset = data match {
case Some(d) =>
Error.createError(
builder,
code,
builder.createString(message),
d.payloadVariant,
d.payloadData
)
case None =>
Error.createError( Error.createError(
builder, builder,
code, code,
@ -57,6 +117,8 @@ object ErrorFactory {
ErrorPayload.NONE, ErrorPayload.NONE,
0 0
) )
}
val outMsg = OutboundMessageFactory.create( val outMsg = OutboundMessageFactory.create(
UUID.randomUUID(), UUID.randomUUID(),
maybeCorrelationId, maybeCorrelationId,
@ -67,4 +129,10 @@ object ErrorFactory {
builder.dataBuffer() builder.dataBuffer()
} }
/** Stores additional data for the error.
*
* @param payloadVariant the variant set in the payload
* @param payloadData the data for that variant
*/
case class ErrorData(payloadVariant: Byte, payloadData: Int)
} }

View File

@ -15,7 +15,7 @@ object FileContentsReplyFactory {
* @param contents the binary contents of a file * @param contents the binary contents of a file
* @param correlationId correlation id used to correlate a response with a * @param correlationId correlation id used to correlate a response with a
* request * request
* @return an FlatBuffer representation of the created error * @return an FlatBuffer representation of the reply
*/ */
def createPacket( def createPacket(
contents: Array[Byte], contents: Array[Byte],

View File

@ -0,0 +1,45 @@
package org.enso.languageserver.protocol.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.{
EnsoUUID,
OutboundPayload,
ReadBytesReply
}
import java.nio.ByteBuffer
import java.util.UUID
object ReadBytesReplyFactory {
/** Creates a [[ReadBytesReply]] inside a [[FlatBufferBuilder]].
*
* @param checksum the checksum value of the read bytes
* @param bytes the bytes that were read
* @param correlationId an identifier used to correlate a response with a
* request
* @return a FlatBuffer representation of the reply
*/
def create(
checksum: Array[Byte],
bytes: Array[Byte],
correlationId: EnsoUUID
): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val digestOffset = EnsoDigestFactory.create(checksum)
val bytesOffset = builder.createByteVector(bytes)
val replyOffset =
ReadBytesReply.createReadBytesReply(builder, digestOffset, bytesOffset)
val outMsg = OutboundMessageFactory.create(
UUID.randomUUID(),
Some(correlationId),
OutboundPayload.READ_BYTES_REPLY,
replyOffset
)
builder.finish(outMsg)
builder.dataBuffer()
}
}

View File

@ -0,0 +1,39 @@
package org.enso.languageserver.protocol.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.{
EnsoUUID,
OutboundPayload,
WriteBytesReply
}
import java.nio.ByteBuffer
import java.util.UUID
object WriteBytesReplyFactory {
/** Creates a [[WriteBytesReply]] inside a [[FlatBufferBuilder]].
*
* @param checksum the checksum value of the written bytes
* @param correlationId an identifier used to correlate a response with a
* request
* @return a FlatBuffer representation of the reply
*/
def create(checksum: Array[Byte], correlationId: EnsoUUID): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val digestOffset = EnsoDigestFactory.create(checksum)
val replyOffset =
WriteBytesReply.createWriteBytesReply(builder, digestOffset)
val outMsg = OutboundMessageFactory.create(
UUID.randomUUID(),
Some(correlationId),
OutboundPayload.WRITE_BYTES_REPLY,
replyOffset
)
builder.finish(outMsg)
builder.dataBuffer()
}
}

View File

@ -0,0 +1,146 @@
package org.enso.languageserver.requesthandler.file
import akka.actor.{Actor, ActorRef, Cancellable, Props, Status}
import com.typesafe.scalalogging.LazyLogging
import org.enso.jsonrpc.Errors.RequestTimeout
import org.enso.languageserver.filemanager.{
FileManagerProtocol,
FileSystemFailureMapper,
ReadOutOfBounds
}
import org.enso.languageserver.protocol.binary.factory.{
ChecksumBytesReplyFactory,
ErrorFactory
}
import org.enso.languageserver.protocol.binary.{
ChecksumBytesCommand,
EnsoUUID,
FileSegment,
InboundMessage
}
import org.enso.languageserver.util.UnhandledLogging
import org.enso.languageserver.util.file.PathUtils
import org.enso.logger.masking.MaskedString
import scala.concurrent.duration.FiniteDuration
/** A handler for a checksum bytes request.
*
* @param requestTimeout a request timeout
* @param fileManager a reference to the file-manager actor
* @param replyTo the actor to reply to
*/
class ChecksumBytesHandler(
requestTimeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
) extends Actor
with LazyLogging
with UnhandledLogging {
import context.dispatcher
override def receive: Receive = requestStage
private def requestStage: Receive = { case msg: InboundMessage =>
val payload =
msg.payload(new ChecksumBytesCommand).asInstanceOf[ChecksumBytesCommand]
val segment = payload.segment
fileManager ! FileManagerProtocol.ChecksumBytesRequest(
ChecksumBytesHandler.convertFileSegment(segment)
)
val cancellable = context.system.scheduler.scheduleOnce(
requestTimeout,
self,
RequestTimeout
)
context.become(responseStage(msg.messageId(), cancellable))
}
private def responseStage(
requestId: EnsoUUID,
cancellable: Cancellable
): Receive = {
case Status.Failure(ex) =>
logger.error(
"Failure during the ChecksumBytes operation: {}",
MaskedString(ex.getMessage)
)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
cancellable.cancel()
context.stop(self)
case RequestTimeout =>
logger.error("Request ChecksumBytes [{}] timed out.", requestId)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
context.stop(self)
case FileManagerProtocol.ChecksumBytesResponse(Left(failure))
if failure.hasData =>
failure match {
case ReadOutOfBounds(fileLength) =>
val response =
ErrorFactory.createReadOutOfBoundsError(fileLength, Some(requestId))
replyTo ! response
cancellable.cancel()
context.stop(self)
case _ =>
logger.error("The impossible happened in request [{}].", requestId)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
context.stop(self)
}
case FileManagerProtocol.ChecksumBytesResponse(Left(failure))
if !failure.hasData =>
val error = FileSystemFailureMapper.mapFailure(failure)
val response = ErrorFactory.createGenericError(
error.code,
error.message,
maybeCorrelationId = Some(requestId)
)
replyTo ! response
cancellable.cancel()
context.stop(self)
case FileManagerProtocol.ChecksumBytesResponse(Right(checksum)) =>
val response = ChecksumBytesReplyFactory.create(checksum, requestId)
replyTo ! response
cancellable.cancel()
context.stop(self)
}
}
object ChecksumBytesHandler {
/** Creates a configuration object used to create a [[ChecksumBytesHandler]].
*
* @param timeout the request timeout
* @param fileManager the file system manager actor
* @param replyTo the outbound channel delivering replies to the client
* @return a configuration object
*/
def props(
timeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
): Props = {
Props(new ChecksumBytesHandler(timeout, fileManager, replyTo))
}
/** Converts from a binary file segment to a protocol one.
*
* @param segment the segment to convert
* @return `segment` using protocol types
*/
def convertFileSegment(
segment: FileSegment
): FileManagerProtocol.Data.FileSegment = {
FileManagerProtocol.Data.FileSegment(
PathUtils.convertBinaryPath(segment.path),
segment.byteOffset(),
segment.length()
)
}
}

View File

@ -31,7 +31,7 @@ class ChecksumFileHandler(
private def requestStage: Receive = { private def requestStage: Receive = {
case Request(ChecksumFile, id, params: ChecksumFile.Params) => case Request(ChecksumFile, id, params: ChecksumFile.Params) =>
fileManager ! FileManagerProtocol.ChecksumRequest(params.path) fileManager ! FileManagerProtocol.ChecksumFileRequest(params.path)
val cancellable = context.system.scheduler.scheduleOnce( val cancellable = context.system.scheduler.scheduleOnce(
requestTimeout, requestTimeout,
self, self,
@ -60,7 +60,7 @@ class ChecksumFileHandler(
replyTo ! ResponseError(Some(id), Errors.RequestTimeout) replyTo ! ResponseError(Some(id), Errors.RequestTimeout)
context.stop(self) context.stop(self)
case FileManagerProtocol.ChecksumResponse(Left(failure)) => case FileManagerProtocol.ChecksumFileResponse(Left(failure)) =>
replyTo ! ResponseError( replyTo ! ResponseError(
Some(id), Some(id),
FileSystemFailureMapper.mapFailure(failure) FileSystemFailureMapper.mapFailure(failure)
@ -68,7 +68,7 @@ class ChecksumFileHandler(
cancellable.cancel() cancellable.cancel()
context.stop(self) context.stop(self)
case FileManagerProtocol.ChecksumResponse(Right(result)) => case FileManagerProtocol.ChecksumFileResponse(Right(result)) =>
replyTo ! ResponseResult(ChecksumFile, id, ChecksumFile.Result(result)) replyTo ! ResponseResult(ChecksumFile, id, ChecksumFile.Result(result))
cancellable.cancel() cancellable.cancel()
context.stop(self) context.stop(self)

View File

@ -75,7 +75,7 @@ class ReadBinaryFileHandler(
val packet = ErrorFactory.createGenericError( val packet = ErrorFactory.createGenericError(
error.code, error.code,
error.message, error.message,
Some(requestId) maybeCorrelationId = Some(requestId)
) )
replyTo ! packet replyTo ! packet
cancellable.cancel() cancellable.cancel()

View File

@ -0,0 +1,133 @@
package org.enso.languageserver.requesthandler.file
import akka.actor.{Actor, ActorRef, Cancellable, Props, Status}
import com.typesafe.scalalogging.LazyLogging
import org.enso.jsonrpc.Errors.RequestTimeout
import org.enso.languageserver.filemanager.{
FileManagerProtocol,
FileSystemFailureMapper,
ReadOutOfBounds
}
import org.enso.languageserver.protocol.binary.factory.{
ErrorFactory,
ReadBytesReplyFactory
}
import org.enso.languageserver.protocol.binary.{
EnsoUUID,
InboundMessage,
ReadBytesCommand
}
import org.enso.languageserver.util.UnhandledLogging
import org.enso.logger.masking.MaskedString
import scala.concurrent.duration.FiniteDuration
/** A handler for a read bytes request
*
* @param requestTimeout a request timeout
* @param fileManager a reference to the file-manager actor
* @param replyTo the actor to reply to
*/
class ReadBytesHandler(
requestTimeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
) extends Actor
with LazyLogging
with UnhandledLogging {
import context.dispatcher
override def receive: Receive = requestStage
private def requestStage: Receive = { case msg: InboundMessage =>
val payload =
msg.payload(new ReadBytesCommand).asInstanceOf[ReadBytesCommand]
val segment = payload.segment
fileManager ! FileManagerProtocol.ReadBytesRequest(
ChecksumBytesHandler.convertFileSegment(segment)
)
val cancellable = context.system.scheduler.scheduleOnce(
requestTimeout,
self,
RequestTimeout
)
context.become(responseStage(msg.messageId(), cancellable))
}
private def responseStage(
requestId: EnsoUUID,
cancellable: Cancellable
): Receive = {
case Status.Failure(ex) =>
logger.error(
"Failure during the ChecksumBytes operation: {}",
MaskedString(ex.getMessage)
)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
cancellable.cancel()
context.stop(self)
case RequestTimeout =>
logger.error("Request ChecksumBytes [{}] timed out.", requestId)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
context.stop(self)
case FileManagerProtocol.ReadBytesResponse(Left(failure))
if failure.hasData =>
failure match {
case ReadOutOfBounds(fileLength) =>
val response =
ErrorFactory.createReadOutOfBoundsError(fileLength, Some(requestId))
replyTo ! response
cancellable.cancel()
context.stop(self)
case _ =>
logger.error("The impossible happened in request [{}].", requestId)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
context.stop(self)
}
case FileManagerProtocol.ReadBytesResponse(Left(failure))
if !failure.hasData =>
val error = FileSystemFailureMapper.mapFailure(failure)
val response = ErrorFactory.createGenericError(
error.code,
error.message,
maybeCorrelationId = Some(requestId)
)
replyTo ! response
cancellable.cancel()
context.stop(self)
case FileManagerProtocol.ReadBytesResponse(Right(readBytesResult)) =>
val response = ReadBytesReplyFactory.create(
readBytesResult.checksum.bytes,
readBytesResult.bytes,
requestId
)
replyTo ! response
cancellable.cancel()
context.stop(self)
}
}
object ReadBytesHandler {
/** Creates a configuration object used to create a [[ReadBytesHandler]].
*
* @param timeout the request timeout
* @param fileManager the file system manager actor
* @param replyTo the outbound channel delivering replies to the client
* @return a configuration object
*/
def props(
timeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
): Props = {
Props(new ReadBytesHandler(timeout, fileManager, replyTo))
}
}

View File

@ -78,7 +78,7 @@ class WriteBinaryFileHandler(
val packet = ErrorFactory.createGenericError( val packet = ErrorFactory.createGenericError(
error.code, error.code,
error.message, error.message,
Some(requestId) maybeCorrelationId = Some(requestId)
) )
replyTo ! packet replyTo ! packet
cancellable.cancel() cancellable.cancel()

View File

@ -0,0 +1,117 @@
package org.enso.languageserver.requesthandler.file
import akka.actor.{Actor, ActorRef, Cancellable, Props, Status}
import com.typesafe.scalalogging.LazyLogging
import org.enso.jsonrpc.Errors.RequestTimeout
import org.enso.languageserver.filemanager.{
FileManagerProtocol,
FileSystemFailureMapper
}
import org.enso.languageserver.protocol.binary.factory.{
ErrorFactory,
WriteBytesReplyFactory
}
import org.enso.languageserver.protocol.binary.{
EnsoUUID,
InboundMessage,
WriteBytesCommand
}
import org.enso.languageserver.util.UnhandledLogging
import org.enso.languageserver.util.file.PathUtils
import org.enso.logger.masking.MaskedString
import scala.concurrent.duration.FiniteDuration
/** A handler for a write bytes request
*
* @param requestTimeout a request timeout
* @param fileManager a reference to the file-manager actor
* @param replyTo the actor to reply to
*/
class WriteBytesHandler(
requestTimeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
) extends Actor
with LazyLogging
with UnhandledLogging {
import context.dispatcher
override def receive: Receive = requestStage
private def requestStage: Receive = { case msg: InboundMessage =>
val payload =
msg.payload(new WriteBytesCommand).asInstanceOf[WriteBytesCommand]
val byteBuf = payload.bytesAsByteBuffer()
val bytes = new Array[Byte](byteBuf.remaining())
byteBuf.get(bytes)
fileManager ! FileManagerProtocol.WriteBytesRequest(
PathUtils.convertBinaryPath(payload.path()),
payload.byteOffset(),
payload.overwriteExisting(),
bytes
)
val cancellable = context.system.scheduler.scheduleOnce(
requestTimeout,
self,
RequestTimeout
)
context.become(responseStage(msg.messageId(), cancellable))
}
private def responseStage(
requestId: EnsoUUID,
cancellable: Cancellable
): Receive = {
case Status.Failure(ex) =>
logger.error(
"Failure during the WriteBytes operation: {}",
MaskedString(ex.getMessage)
)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
cancellable.cancel()
context.stop(self)
case RequestTimeout =>
logger.error("Request WriteBytes [{}] timed out.", requestId)
val response = ErrorFactory.createServiceError(Some(requestId))
replyTo ! response
context.stop(self)
case FileManagerProtocol.WriteBytesResponse(Left(failure)) =>
val error = FileSystemFailureMapper.mapFailure(failure)
val response = ErrorFactory.createGenericError(
error.code,
error.message,
maybeCorrelationId = Some(requestId)
)
replyTo ! response
cancellable.cancel()
context.stop(self)
case FileManagerProtocol.WriteBytesResponse(Right(checksum)) =>
val response = WriteBytesReplyFactory.create(checksum, requestId)
replyTo ! response
cancellable.cancel()
context.stop(self)
}
}
object WriteBytesHandler {
/** Creates a configuration object used to create a [[WriteBytesHandler]].
*
* @param timeout the request timeout
* @param fileManager the file system manager actor
* @param replyTo the outbound channel delivering replies to the client
* @return a configuration object
*/
def props(
timeout: FiniteDuration,
fileManager: ActorRef,
replyTo: ActorRef
): Props = {
Props(new WriteBytesHandler(timeout, fileManager, replyTo))
}
}

View File

@ -1,23 +1,24 @@
package org.enso.languageserver.filemanager package org.enso.languageserver.filemanager
import java.nio.file.{Files, Path, Paths}
import java.nio.file.attribute.BasicFileAttributes
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import org.bouncycastle.util.encoders.Hex
import org.enso.languageserver.effect.Effects import org.enso.languageserver.effect.Effects
import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import java.nio.file.attribute.BasicFileAttributes
import java.nio.file.{Files, Path, Paths}
import java.security.MessageDigest import java.security.MessageDigest
import scala.io.Source
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.io.Source
import scala.jdk.CollectionConverters._
class FileSystemSpec extends AnyFlatSpec with Matchers with Effects { class FileSystemSpec extends AnyWordSpecLike with Matchers with Effects {
import FileSystemApi._ import FileSystemApi._
"A file system interpreter" should "write textual content to file" in new TestCtx { "Writing files" should {
"correctly write textual contents" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo.txt") val path = Paths.get(testDirPath.toString, "foo.txt")
val content = "123456789" val content = "123456789"
@ -29,7 +30,34 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
readTxtFile(path) shouldBe content readTxtFile(path) shouldBe content
} }
it should "write binary contents to a file" in new TestCtx { "overwrite existing files" in new TestCtx {
//given
val path = Paths.get(testDirPath.toString, "foo.txt")
val existingContent = "123456789"
val newContent = "abcdef"
//when
objectUnderTest.write(path.toFile, existingContent).unsafeRunSync()
objectUnderTest.write(path.toFile, newContent).unsafeRunSync()
//then
readTxtFile(path) shouldBe newContent
}
"create the parent directory if it doesn't exist" in new TestCtx {
//given
val path = Paths.get(testDirPath.toString, "foo.txt")
val content = "123456789"
testDirPath.toFile.delete()
//when
val result =
objectUnderTest.write(path.toFile, content).unsafeRunSync()
//then
result shouldBe Right(())
readTxtFile(path) shouldBe content
}
}
"Writing binary files" should {
"correctly write binary contents" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo.txt") val path = Paths.get(testDirPath.toString, "foo.txt")
val content = Array[Byte](1, 2, 3) val content = Array[Byte](1, 2, 3)
@ -43,19 +71,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
savedContent.toList shouldBe content.toList savedContent.toList shouldBe content.toList
} }
it should "overwrite existing files" in new TestCtx { "overwrite existing binary files" in new TestCtx {
//given
val path = Paths.get(testDirPath.toString, "foo.txt")
val existingContent = "123456789"
val newContent = "abcdef"
//when
objectUnderTest.write(path.toFile, existingContent).unsafeRunSync()
objectUnderTest.write(path.toFile, newContent).unsafeRunSync()
//then
readTxtFile(path) shouldBe newContent
}
it should "overwrite existing binary files" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo.txt") val path = Paths.get(testDirPath.toString, "foo.txt")
val existingContent = Array[Byte](1, 2, 3) val existingContent = Array[Byte](1, 2, 3)
@ -68,30 +84,10 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
//then //then
savedContent.toList shouldBe newContent.toList savedContent.toList shouldBe newContent.toList
} }
it should "create the parent directory if it doesn't exist" in new TestCtx {
//given
val path = Paths.get(testDirPath.toString, "foo.txt")
val content = "123456789"
testDirPath.toFile.delete()
//when
val result =
objectUnderTest.write(path.toFile, content).unsafeRunSync()
//then
result shouldBe Right(())
readTxtFile(path) shouldBe content
} }
it should "return FileNotFound failure if the file doesn't exist" in new TestCtx { "Reading files" should {
//given "read file contents" in new TestCtx {
val path = Paths.get(testDirPath.toString, "foo.txt")
//when
val result = objectUnderTest.read(path.toFile).unsafeRunSync()
//then
result shouldBe Left(FileNotFound)
}
it should "read a file content" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo.txt") val path = Paths.get(testDirPath.toString, "foo.txt")
val content = "123456789" val content = "123456789"
@ -102,7 +98,18 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result shouldBe Right(content) result shouldBe Right(content)
} }
it should "create a directory" in new TestCtx { "return FileNotFound failure if the file doesn't exist" in new TestCtx {
//given
val path = Paths.get(testDirPath.toString, "foo.txt")
//when
val result = objectUnderTest.read(path.toFile).unsafeRunSync()
//then
result shouldBe Left(FileNotFound)
}
}
"File management" should {
"create a directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar") val path = Paths.get(testDirPath.toString, "foo", "bar")
//when //when
@ -112,7 +119,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.isDirectory shouldBe true path.toFile.isDirectory shouldBe true
} }
it should "create an empty file" in new TestCtx { "create an empty file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar", "baz.txt") val path = Paths.get(testDirPath.toString, "foo", "bar", "baz.txt")
//when //when
@ -123,7 +130,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.isFile shouldBe true path.toFile.isFile shouldBe true
} }
it should "delete a file" in new TestCtx { "delete a file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar.txt") val path = Paths.get(testDirPath.toString, "foo", "bar.txt")
createEmptyFile(path) createEmptyFile(path)
@ -136,7 +143,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.getParentFile.isDirectory shouldBe true path.toFile.getParentFile.isDirectory shouldBe true
} }
it should "delete a directory recursively" in new TestCtx { "delete a directory recursively" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar.txt") val path = Paths.get(testDirPath.toString, "foo", "bar.txt")
createEmptyFile(path) createEmptyFile(path)
@ -150,7 +157,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.getParentFile.exists shouldBe false path.toFile.getParentFile.exists shouldBe false
} }
it should "return NotFoundError when deleting nonexistent file" in new TestCtx { "return NotFoundError when deleting nonexistent file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar.txt") val path = Paths.get(testDirPath.toString, "foo", "bar.txt")
//when //when
@ -161,7 +168,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.getParentFile.exists shouldBe false path.toFile.getParentFile.exists shouldBe false
} }
it should "return NotFoundError when deleting nonexistent directory" in new TestCtx { "return NotFoundError when deleting nonexistent directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar.txt") val path = Paths.get(testDirPath.toString, "foo", "bar.txt")
//when //when
@ -173,7 +180,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
path.toFile.getParentFile.exists shouldBe false path.toFile.getParentFile.exists shouldBe false
} }
it should "copy a file" in new TestCtx { "copy a file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "copy_file", "a.txt") val path = Paths.get(testDirPath.toString, "copy_file", "a.txt")
val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync() val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync()
@ -187,7 +194,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.isFile shouldBe true to.toFile.isFile shouldBe true
} }
it should "copy a directory" in new TestCtx { "copy a directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "copy_dir", "a.txt") val path = Paths.get(testDirPath.toString, "copy_dir", "a.txt")
val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync() val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync()
@ -203,7 +210,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.resolve(path.getFileName).toFile.isFile shouldBe true to.resolve(path.getFileName).toFile.isFile shouldBe true
} }
it should "copy a file to existing directory" in new TestCtx { "copy a file to existing directory" in new TestCtx {
//given //given
val from = Paths.get(testDirPath.toString, "copy_dir", "a.txt") val from = Paths.get(testDirPath.toString, "copy_dir", "a.txt")
val resultCreateFile = val resultCreateFile =
@ -222,7 +229,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.resolve(from.getFileName).toFile.isFile shouldBe true to.resolve(from.getFileName).toFile.isFile shouldBe true
} }
it should "return FileExists error when copying directory to existing file" in new TestCtx { "return FileExists error when copying directory to existing file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "copy_dir", "a.txt") val path = Paths.get(testDirPath.toString, "copy_dir", "a.txt")
val resultCreateFile = val resultCreateFile =
@ -241,7 +248,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.isFile shouldBe true to.toFile.isFile shouldBe true
} }
it should "return FileNotFound when copy nonexistent file" in new TestCtx { "return FileNotFound when copy nonexistent file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "copy_nonexistent", "a.txt") val path = Paths.get(testDirPath.toString, "copy_nonexistent", "a.txt")
val to = Paths.get(testDirPath.toString, "copy_file", "b.txt") val to = Paths.get(testDirPath.toString, "copy_file", "b.txt")
@ -254,7 +261,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.exists shouldBe false to.toFile.exists shouldBe false
} }
it should "reutrn FileNotFound when copy nonexistent directory" in new TestCtx { "reutrn FileNotFound when copy nonexistent directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "copy_nonexistent") val path = Paths.get(testDirPath.toString, "copy_nonexistent")
val to = Paths.get(testDirPath.toString, "copy_file") val to = Paths.get(testDirPath.toString, "copy_file")
@ -267,7 +274,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.exists shouldBe false to.toFile.exists shouldBe false
} }
it should "move a file" in new TestCtx { "move a file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "move_file", "a.txt") val path = Paths.get(testDirPath.toString, "move_file", "a.txt")
val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync() val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync()
@ -281,7 +288,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.isFile shouldBe true to.toFile.isFile shouldBe true
} }
it should "move a directory" in new TestCtx { "move a directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "move_dir", "a.txt") val path = Paths.get(testDirPath.toString, "move_dir", "a.txt")
val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync() val resultCreate = objectUnderTest.createFile(path.toFile).unsafeRunSync()
@ -297,7 +304,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.resolve(path.getFileName()).toFile.isFile shouldBe true to.resolve(path.getFileName()).toFile.isFile shouldBe true
} }
it should "move a file to existing directory" in new TestCtx { "move a file to existing directory" in new TestCtx {
//given //given
val from = Paths.get(testDirPath.toString, "move_dir", "a.txt") val from = Paths.get(testDirPath.toString, "move_dir", "a.txt")
val resultCreateFile = val resultCreateFile =
@ -315,7 +322,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.resolve(from.getFileName).toFile.isFile shouldBe true to.resolve(from.getFileName).toFile.isFile shouldBe true
} }
it should "move a directory to existing directory" in new TestCtx { "move a directory to existing directory" in new TestCtx {
//given //given
val from = Paths.get(testDirPath.toString, "move_dir", "a.txt") val from = Paths.get(testDirPath.toString, "move_dir", "a.txt")
val resultCreateFile = val resultCreateFile =
@ -337,7 +344,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
dest.resolve(from.getFileName).toFile.isFile shouldBe true dest.resolve(from.getFileName).toFile.isFile shouldBe true
} }
it should "return FileNotFound when moving nonexistent file" in new TestCtx { "return FileNotFound when moving nonexistent file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "nonexistent", "a.txt") val path = Paths.get(testDirPath.toString, "nonexistent", "a.txt")
val to = Paths.get(testDirPath.toString, "move_file", "b.txt") val to = Paths.get(testDirPath.toString, "move_file", "b.txt")
@ -349,7 +356,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.exists shouldBe false to.toFile.exists shouldBe false
} }
it should "return FileExists when moving to existing destination" in new TestCtx { "return FileExists when moving to existing destination" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "move_file", "a.txt") val path = Paths.get(testDirPath.toString, "move_file", "a.txt")
val resultCreateFrom = val resultCreateFrom =
@ -366,7 +373,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
to.toFile.isFile shouldBe true to.toFile.isFile shouldBe true
} }
it should "check file existence" in new TestCtx { "check file existence" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "foo", "bar.txt") val path = Paths.get(testDirPath.toString, "foo", "bar.txt")
createEmptyFile(path) createEmptyFile(path)
@ -377,7 +384,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result shouldBe Right(true) result shouldBe Right(true)
} }
it should "check file non-existence" in new TestCtx { "check file non-existence" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "nonexistent.txt") val path = Paths.get(testDirPath.toString, "nonexistent.txt")
path.toFile.exists shouldBe false path.toFile.exists shouldBe false
@ -386,8 +393,10 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
//then //then
result shouldBe Right(false) result shouldBe Right(false)
} }
}
it should "list directory contents" in new TestCtx { "Directory listing" should {
"list directory contents" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "list") val path = Paths.get(testDirPath.toString, "list")
val fileA = Paths.get(testDirPath.toString, "list", "a.txt") val fileA = Paths.get(testDirPath.toString, "list", "a.txt")
@ -409,7 +418,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
) )
} }
it should "return FileNotFound error when listing nonexistent path" in new TestCtx { "return FileNotFound error when listing nonexistent path" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "nonexistent") val path = Paths.get(testDirPath.toString, "nonexistent")
//when //when
@ -418,7 +427,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result shouldBe Left(FileNotFound) result shouldBe Left(FileNotFound)
} }
it should "return NotDirectory error when listing a file" in new TestCtx { "return NotDirectory error when listing a file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "a.txt") val path = Paths.get(testDirPath.toString, "a.txt")
createEmptyFile(path) createEmptyFile(path)
@ -427,8 +436,10 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
//then //then
result shouldBe Left(NotDirectory) result shouldBe Left(NotDirectory)
} }
}
it should "tree directory contents" in new TestCtx { "Directory treeing" should {
"tree directory contents" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir") val path = Paths.get(testDirPath.toString, "dir")
val subdir = Paths.get(testDirPath.toString, "dir", "subdir") val subdir = Paths.get(testDirPath.toString, "dir", "subdir")
@ -460,12 +471,13 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
) )
) )
//when //when
val result = objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync() val result =
objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync()
//then //then
result shouldBe Right(expectedEntry) result shouldBe Right(expectedEntry)
} }
it should "tree directory and limit depth" in new TestCtx { "tree directory and limit depth" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir") val path = Paths.get(testDirPath.toString, "dir")
val subdir = Paths.get(testDirPath.toString, "dir", "subdir") val subdir = Paths.get(testDirPath.toString, "dir", "subdir")
@ -489,7 +501,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result shouldBe Right(expectedEntry) result shouldBe Right(expectedEntry)
} }
it should "tree directory and detect symlink loops" in new TestCtx { "tree directory and detect symlink loops" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir") val path = Paths.get(testDirPath.toString, "dir")
val dirA = Paths.get(testDirPath.toString, "dir", "a") val dirA = Paths.get(testDirPath.toString, "dir", "a")
@ -515,7 +527,8 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
Paths.get(symlinkB.toString, "symlink_a"), Paths.get(symlinkB.toString, "symlink_a"),
ArrayBuffer( ArrayBuffer(
SymbolicLinkEntry( SymbolicLinkEntry(
Paths.get(symlinkB.toString, "symlink_a", "symlink_b"), Paths
.get(symlinkB.toString, "symlink_a", "symlink_b"),
symlinkB symlinkB
) )
) )
@ -534,7 +547,8 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
Paths.get(symlinkA.toString, "symlink_b"), Paths.get(symlinkA.toString, "symlink_b"),
ArrayBuffer( ArrayBuffer(
SymbolicLinkEntry( SymbolicLinkEntry(
Paths.get(symlinkA.toString, "symlink_b", "symlink_a"), Paths
.get(symlinkA.toString, "symlink_b", "symlink_a"),
symlinkA symlinkA
) )
) )
@ -546,12 +560,13 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
) )
) )
//when //when
val result = objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync() val result =
objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync()
//then //then
result shouldBe Right(expectedEntry) result shouldBe Right(expectedEntry)
} }
it should "tree directory with broken symlinks" in new TestCtx { "tree directory with broken symlinks" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir") val path = Paths.get(testDirPath.toString, "dir")
val fileA = Paths.get(testDirPath.toString, "dir", "a.txt") val fileA = Paths.get(testDirPath.toString, "dir", "a.txt")
@ -565,22 +580,24 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
) )
) )
//when //when
val result = objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync() val result =
objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync()
//then //then
result shouldBe Right(expectedEntry) result shouldBe Right(expectedEntry)
} }
it should "return NotDirectory when tree path is not a directory" in new TestCtx { "return NotDirectory when tree path is not a directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir", "a.txt") val path = Paths.get(testDirPath.toString, "dir", "a.txt")
createEmptyFile(path) createEmptyFile(path)
//when //when
val result = objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync() val result =
objectUnderTest.tree(path.toFile, depth = None).unsafeRunSync()
//then //then
result shouldBe Left(NotDirectory) result shouldBe Left(NotDirectory)
} }
it should "return FileNotFound when tree depth <= 0" in new TestCtx { "return FileNotFound when tree depth <= 0" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir", "a.txt") val path = Paths.get(testDirPath.toString, "dir", "a.txt")
createEmptyFile(path) createEmptyFile(path)
@ -591,8 +608,10 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
//then //then
result shouldBe Left(FileNotFound) result shouldBe Left(FileNotFound)
} }
}
it should "get attributes of a file" in new TestCtx { "File info" should {
"get attributes of a file" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "a.txt") val path = Paths.get(testDirPath.toString, "a.txt")
createEmptyFile(path) createEmptyFile(path)
@ -610,7 +629,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result shouldBe Right(expectedAttrs) result shouldBe Right(expectedAttrs)
} }
it should "get attributes of a directory" in new TestCtx { "get attributes of a directory" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "dir", "a.txt") val path = Paths.get(testDirPath.toString, "dir", "a.txt")
val dir = path.getParent() val dir = path.getParent()
@ -621,7 +640,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result.map(_.kind) shouldBe Right(DirectoryEntryTruncated(dir)) result.map(_.kind) shouldBe Right(DirectoryEntryTruncated(dir))
} }
it should "get attributes of a symlink" in new TestCtx { "get attributes of a symlink" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "a.txt") val path = Paths.get(testDirPath.toString, "a.txt")
val symlink = Paths.get(testDirPath.toString, "symlink.txt") val symlink = Paths.get(testDirPath.toString, "symlink.txt")
@ -633,7 +652,7 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
result.map(_.kind) shouldBe Right(FileEntry(symlink)) result.map(_.kind) shouldBe Right(FileEntry(symlink))
} }
it should "return FileNotFound getting attributes if file does not exist" in new TestCtx { "return FileNotFound getting attributes if file does not exist" in new TestCtx {
//given //given
val path = Paths.get(testDirPath.toString, "nonexistent.txt") val path = Paths.get(testDirPath.toString, "nonexistent.txt")
//when //when
@ -641,30 +660,332 @@ class FileSystemSpec extends AnyFlatSpec with Matchers with Effects {
//then //then
result shouldBe Left(FileNotFound) result shouldBe Left(FileNotFound)
} }
}
it should "return the correct checksum when the target is a file" in new TestCtx { "File checksum" should {
"return the correct checksum when the target is a file" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt") val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!" val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path) createFileContaining(fileContents, path)
val expectedDigest = val expectedDigest =
MessageDigest.getInstance("SHA3-224").digest(Files.readAllBytes(path)) MessageDigest.getInstance("SHA3-224").digest(Files.readAllBytes(path))
val expectedDigestString = Hex.toHexString(expectedDigest)
val result = objectUnderTest.digest(path.toFile).unsafeRunSync() val result = objectUnderTest
result shouldBe Right(expectedDigestString) .digest(path.toFile)
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes should contain theSameElementsAs expectedDigest
} }
it should "return an error if the provided path is not a file" in new TestCtx { "return an error if the provided path is not a file" in new TestCtx {
val result = objectUnderTest.digest(testDirPath.toFile).unsafeRunSync() val result = objectUnderTest.digest(testDirPath.toFile).unsafeRunSync()
result shouldBe Left(NotFile) result shouldBe Left(NotFile)
} }
it should "return a FileNotFound error when getting the checksum if the file does not exist" in new TestCtx { "return a FileNotFound error when getting the checksum if the file does not exist" in new TestCtx {
val path = Paths.get(testDirPath.toString, "nonexistent.txt") val path = Paths.get(testDirPath.toString, "nonexistent.txt")
val result = objectUnderTest.digest(path.toFile).unsafeRunSync() val result = objectUnderTest.digest(path.toFile).unsafeRunSync()
result shouldBe Left(FileNotFound) result shouldBe Left(FileNotFound)
} }
}
"Byte-range checksum" should {
"return the checksum for a provided byte range" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
val byteOffset = 1L
val byteCount = 3L
createFileContaining(fileContents, path)
// Digest of "ell"
val expectedDigest = MessageDigest
.getInstance("SHA3-224")
.digest(fileContents.getBytes(StandardCharsets.UTF_8).slice(1, 4))
val byteDigest = objectUnderTest
.digestBytes(
FileSystemApi.FileSegment(path.toFile, byteOffset, byteCount)
)
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
byteDigest.bytes should contain theSameElementsAs expectedDigest
}
"Return a `FileNotFound` error if the file does not exist" in new TestCtx {
val path = Paths.get(testDirPath.toString, "nonexistent.txt")
val byteOffset = 1L
val byteCount = 3L
val result = objectUnderTest
.digestBytes(
FileSystemApi.FileSegment(path.toFile, byteOffset, byteCount)
)
.unsafeRunSync()
result shouldBe Left(FileNotFound)
}
"Return a `ReadOutOfBounds` error if the byte range is out of bounds" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path)
val startOutOfBounds = objectUnderTest
.digestBytes(FileSystemApi.FileSegment(path.toFile, 13, 0))
.unsafeRunSync()
startOutOfBounds shouldBe Left(ReadOutOfBounds(12))
val endOutOfBounds = objectUnderTest
.digestBytes(FileSystemApi.FileSegment(path.toFile, 5, 10))
.unsafeRunSync()
endOutOfBounds shouldBe Left(ReadOutOfBounds(12))
}
"Return a `NotFile` error if the provided path is not a file" in new TestCtx {
val result = objectUnderTest
.digestBytes(FileSystemApi.FileSegment(testDirPath.toFile, 0L, 3L))
.unsafeRunSync()
result shouldBe Left(NotFile)
}
}
"Writing bytes" should {
"Write the provided bytes to the specified file" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, "
createFileContaining(fileContents, path)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 7L
val expectedChecksum = MessageDigest.getInstance("SHA3-224").digest(bytes)
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = false,
bytes
)
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes should contain theSameElementsAs expectedChecksum
val writtenContents = Files
.readAllLines(path, StandardCharsets.UTF_8)
.asScala
.toSeq
.mkString("\n")
writtenContents shouldEqual "Hello, Enso!"
}
"Create the file from scratch if it doesn't exist" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
objectUnderTest.exists(path.toFile).unsafeRunSync() shouldBe Right(false)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 0L
val expectedChecksum = MessageDigest.getInstance("SHA3-224").digest(bytes)
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = false,
bytes
)
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes should contain theSameElementsAs expectedChecksum
val writtenContents = Files
.readAllLines(path, StandardCharsets.UTF_8)
.asScala
.toSeq
.mkString("\n")
writtenContents shouldEqual "Enso!"
}
"Fill any intervening space with null bytes" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, "
createFileContaining(fileContents, path)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 20L
val writtenBytes = Array.fill(13)(0x0.toByte) ++ bytes
val expectedChecksum =
MessageDigest.getInstance("SHA3-224").digest(writtenBytes)
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = false,
bytes
)
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes should contain theSameElementsAs expectedChecksum
val writtenContents = Files.readAllBytes(path)
val expectedContents =
fileContents.getBytes(StandardCharsets.UTF_8) ++ writtenBytes
writtenContents shouldEqual expectedContents
}
"Truncate the file if `byteOffset < length` and `overwriteExisting` is set" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, World!"
createFileContaining(fileContents, path)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 7L
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = true,
bytes
)
.unsafeRunSync()
result.isRight shouldBe true
val writtenContents = Files
.readAllLines(path, StandardCharsets.UTF_8)
.asScala
.toSeq
.mkString("\n")
val expectedContents = "Hello, Enso!"
writtenContents shouldEqual expectedContents
}
"Return a `CannotOverwrite` error if `byteOffset < file.length`" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, World!"
createFileContaining(fileContents, path)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 7L
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = false,
bytes
)
.unsafeRunSync()
result shouldBe Left(CannotOverwrite)
}
"Return a `NotFile` error if the provided path is not a file" in new TestCtx {
val path = Paths.get(testDirPath.toString)
val bytes = "Enso!".getBytes(StandardCharsets.UTF_8)
val byteOffset = 7L
val result = objectUnderTest
.writeBytes(
path.toFile,
byteOffset,
overwriteExisting = false,
bytes
)
.unsafeRunSync()
result shouldBe Left(NotFile)
}
}
"Reading bytes" should {
"Read the specified bytes from the file" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path)
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 2, 3))
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes shouldEqual "llo".getBytes(StandardCharsets.UTF_8)
}
"Read less bytes if the specified segment would take it off the end of the file" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path)
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 9, 10))
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.bytes shouldEqual "so!".getBytes(StandardCharsets.UTF_8)
}
"Return a checksum for the read bytes" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path)
val expectedDigest = MessageDigest
.getInstance("SHA3-224")
.digest("llo".getBytes(StandardCharsets.UTF_8))
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 2, 3))
.unsafeRunSync()
.getOrElse(fail("Should be Right"))
result.checksum.bytes should contain theSameElementsAs expectedDigest
}
"Return a `FileNotFound` error if the file does not exist" in new TestCtx {
val path = Paths.get(testDirPath.toString, "nonexistent.txt")
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 2, 3))
.unsafeRunSync()
result shouldBe Left(FileNotFound)
}
"Return a `ReadOutOfBounds` error if the byte range is out of bounds" in new TestCtx {
val path = Paths.get(testDirPath.toString, "a.txt")
val fileContents = "Hello, Enso!"
createFileContaining(fileContents, path)
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 15, 10))
.unsafeRunSync()
result shouldBe Left(ReadOutOfBounds(12))
}
"Return a `NotFile` error if the provided path is not a file" in new TestCtx {
val path = Paths.get(testDirPath.toString)
val result = objectUnderTest
.readBytes(FileSystemApi.FileSegment(path.toFile, 2, 3))
.unsafeRunSync()
result shouldBe Left(NotFile)
}
}
def readTxtFile(path: Path): String = { def readTxtFile(path: Path): String = {
val buffer = Source.fromFile(path.toFile) val buffer = Source.fromFile(path.toFile)

View File

@ -1,25 +1,16 @@
package org.enso.languageserver.websocket.binary package org.enso.languageserver.websocket.binary
import java.io.File
import java.nio.ByteBuffer
import java.util.UUID
import com.google.flatbuffers.FlatBufferBuilder import com.google.flatbuffers.FlatBufferBuilder
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import org.enso.languageserver.protocol.binary.{ import org.enso.languageserver.protocol.binary._
InboundPayload, import org.enso.languageserver.websocket.binary.factory._
OutboundMessage,
OutboundPayload
}
import org.enso.languageserver.protocol.binary.FileContentsReply
import org.enso.languageserver.websocket.binary.factory.{
InboundMessageFactory,
PathFactory,
ReadFileCommandFactory,
WriteFileCommandFactory
}
import org.enso.testkit.FlakySpec import org.enso.testkit.FlakySpec
import java.io.File
import java.nio.ByteBuffer
import java.nio.file.Files
import java.security.MessageDigest
import java.util.UUID
import scala.io.Source import scala.io.Source
class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec { class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec {
@ -93,16 +84,522 @@ class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec {
} }
"A ChecksumBytesCommand" must {
"Return the checksum for the provided byte range" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val expectedChecksum = ByteBuffer.wrap(
MessageDigest.getInstance("SHA3-224").digest(contents.slice(0, 2))
)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val checksumBytesCmd = createChecksumBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
byteOffset = 0,
length = 2
)
client.send(checksumBytesCmd)
val checksumResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
checksumResponse
.payloadType() shouldEqual OutboundPayload.CHECKSUM_BYTES_REPLY
val payload = checksumResponse
.payload(new ChecksumBytesReply)
.asInstanceOf[ChecksumBytesReply]
val digest = payload.checksum().bytesAsByteBuffer()
digest shouldEqual expectedChecksum
Files.delete(barFile.toPath)
}
"Return a `FileNotFound` error if the file does not exist" in {
val requestId = UUID.randomUUID()
val filename = "does_not_exist.bin"
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val checksumBytesCmd = createChecksumBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
byteOffset = 0,
length = 2
)
client.send(checksumBytesCmd)
val checksumResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
checksumResponse.payloadType() shouldEqual OutboundPayload.ERROR
val payload = checksumResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code() shouldEqual 1003
payload.message() shouldEqual "File not found"
}
"Return a `ReadOutOfBounds` error if the byte range is out of bounds" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val checksumBytesCmd = createChecksumBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
byteOffset = 3,
length = 2
)
client.send(checksumBytesCmd)
val checksumResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
checksumResponse.payloadType() shouldEqual OutboundPayload.ERROR
val payload = checksumResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code() shouldEqual 1009
payload.message() shouldEqual "Read is out of bounds for the file"
payload.dataType() shouldEqual ErrorPayload.READ_OOB
val data = payload
.data(new ReadOutOfBoundsError)
.asInstanceOf[ReadOutOfBoundsError]
data.fileLength() shouldEqual 3
}
"Return a `NotFile` error if the provided path is not a file" in {
val requestId = UUID.randomUUID()
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val checksumBytesCmd = createChecksumBytesCommandPacket(
requestId,
Seq(),
testContentRootId,
byteOffset = 3,
length = 2
)
client.send(checksumBytesCmd)
val checksumResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
checksumResponse.payloadType() shouldEqual OutboundPayload.ERROR
val payload = checksumResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code() shouldEqual 1007
payload.message() shouldEqual "Path is not a file"
}
}
"A WriteBytesCommand" must {
"Write the provided bytes to the specified file" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val newBytes = Array[Byte](65, 66, 67)
val expectedChecksum =
ByteBuffer.wrap(MessageDigest.getInstance("SHA3-224").digest(newBytes))
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val writeBytesCommand = createWriteBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
3L,
newBytes,
overwriteExisting = false
)
client.send(writeBytesCommand)
val writeResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
writeResponse
.payloadType() shouldEqual OutboundPayload.WRITE_BYTES_REPLY
val payload = writeResponse
.payload(new WriteBytesReply)
.asInstanceOf[WriteBytesReply]
val digest = payload.checksum().bytesAsByteBuffer()
digest shouldEqual expectedChecksum
val expectedBytes = Array[Byte](65, 66, 67, 65, 66, 67)
val writtenBytes = Files.readAllBytes(barFile.toPath)
writtenBytes should contain theSameElementsAs expectedBytes
Files.delete(barFile.toPath)
}
"Create the file from scratch if it doesn't exist" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val newBytes = Array[Byte](65, 66, 67)
val expectedChecksum =
MessageDigest.getInstance("SHA3-224").digest(newBytes)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val writeBytesCommand = createWriteBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
0L,
newBytes,
overwriteExisting = false
)
client.send(writeBytesCommand)
val writeResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
writeResponse
.payloadType() shouldEqual OutboundPayload.WRITE_BYTES_REPLY
val payload = writeResponse
.payload(new WriteBytesReply)
.asInstanceOf[WriteBytesReply]
val digest = payload.checksum().bytesAsByteBuffer()
val digestBytes = new Array[Byte](digest.remaining())
digest.get(digestBytes)
digestBytes shouldEqual expectedChecksum
val expectedBytes = Array[Byte](65, 66, 67)
val writtenBytes = Files.readAllBytes(barFile.toPath)
writtenBytes should contain theSameElementsAs expectedBytes
Files.delete(barFile.toPath)
}
"Return a `CannotOverwrite` error if `byteOffset < file.length`" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val newBytes = Array[Byte](65, 66, 67)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val writeBytesCommand = createWriteBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
1L,
newBytes,
overwriteExisting = false
)
client.send(writeBytesCommand)
val writeResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
writeResponse
.payloadType() shouldEqual OutboundPayload.ERROR
val payload = writeResponse
.payload(new Error)
.asInstanceOf[Error]
payload.dataType() shouldEqual ErrorPayload.NONE
payload.code() shouldEqual 1008
payload.message() shouldEqual "Cannot overwrite the file without `overwriteExisting` set"
Files.delete(barFile.toPath)
}
"Return a `NotFile` error if the provided path is not a file" in {
val requestId = UUID.randomUUID()
val newBytes = Array[Byte](65, 66, 67)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val writeBytesCommand = createWriteBytesCommandPacket(
requestId,
Seq(),
testContentRootId,
1L,
newBytes,
overwriteExisting = false
)
client.send(writeBytesCommand)
val writeResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
writeResponse
.payloadType() shouldEqual OutboundPayload.ERROR
val payload = writeResponse
.payload(new Error)
.asInstanceOf[Error]
payload.dataType() shouldEqual ErrorPayload.NONE
payload.code() shouldEqual 1007
payload.message() shouldEqual "Path is not a file"
}
}
"A ReadBytesCommand" must {
"Read the specified bytes from the file" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val expectedBytes = Array[Byte](65, 66)
val expectedChecksum =
MessageDigest.getInstance("SHA3-224").digest(expectedBytes)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val readBytesCommand = createReadBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
0L,
2L
)
client.send(readBytesCommand)
val readResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
readResponse
.payloadType() shouldEqual OutboundPayload.READ_BYTES_REPLY
val payload = readResponse
.payload(new ReadBytesReply)
.asInstanceOf[ReadBytesReply]
val digestBuffer = payload.checksum().bytesAsByteBuffer()
val digest = new Array[Byte](payload.checksum().bytesLength())
digestBuffer.get(digest)
val bytesBuffer = payload.bytesAsByteBuffer()
val bytes = new Array[Byte](payload.bytesLength())
bytesBuffer.get(bytes)
bytes should contain theSameElementsAs expectedBytes
digest should contain theSameElementsAs expectedChecksum
Files.delete(barFile.toPath)
}
"Return a `FileNotFound` error if the file does not exist" in {
val requestId = UUID.randomUUID()
val filename = "does_not_exist.bin"
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val readBytesCommand = createReadBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
0L,
2L
)
client.send(readBytesCommand)
val readResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
readResponse
.payloadType() shouldEqual OutboundPayload.ERROR
val payload = readResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code shouldEqual 1003
payload.message shouldEqual "File not found"
payload.dataType shouldEqual ErrorPayload.NONE
}
"Return a `ReadOutOfBounds` error if the byte range is out of bounds" in {
val requestId = UUID.randomUUID()
val filename = "bar.bin"
val barFile = new File(testContentRoot.toFile, filename)
val contents = Array[Byte](65, 66, 67) //ABC
FileUtils.writeByteArrayToFile(barFile, contents)
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val readBytesCommand = createReadBytesCommandPacket(
requestId,
Seq(filename),
testContentRootId,
3L,
2L
)
client.send(readBytesCommand)
val readResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
readResponse
.payloadType() shouldEqual OutboundPayload.ERROR
val payload = readResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code shouldEqual 1009
payload.message shouldEqual "Read is out of bounds for the file"
payload.dataType() shouldEqual ErrorPayload.READ_OOB
val data = payload
.data(new ReadOutOfBoundsError)
.asInstanceOf[ReadOutOfBoundsError]
data.fileLength() shouldEqual 3
Files.delete(barFile.toPath)
}
"Return a `NotFile` error if the provided path is not a file" in {
val requestId = UUID.randomUUID()
val client = newWsClient()
client.send(createSessionInitCmd())
client.expectFrame()
val readBytesCommand = createReadBytesCommandPacket(
requestId,
Seq(),
testContentRootId,
0L,
2L
)
client.send(readBytesCommand)
val readResponse = client
.receiveMessage[OutboundMessage]()
.getOrElse(fail("Should be right"))
readResponse
.payloadType() shouldEqual OutboundPayload.ERROR
val payload = readResponse
.payload(new Error)
.asInstanceOf[Error]
payload.code shouldEqual 1007
payload.message shouldEqual "Path is not a file"
payload.dataType shouldEqual ErrorPayload.NONE
}
}
def createChecksumBytesCommandPacket(
requestId: UUID,
pathSegments: Seq[String],
rootId: UUID,
byteOffset: Long,
length: Long
): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val path = PathFactory.create(rootId, pathSegments)
val fileSegment = FileSegmentFactory.create(path, byteOffset, length)
val command = ChecksumBytesCommandFactory.create(fileSegment)
val incomingMessage = InboundMessageFactory.create(
requestId,
None,
InboundPayload.CHECKSUM_BYTES_CMD,
command
)
builder.finish(incomingMessage)
builder.dataBuffer()
}
def createWriteBytesCommandPacket(
requestId: UUID,
pathSegments: Seq[String],
rootId: UUID,
byteOffset: Long,
bytes: Array[Byte],
overwriteExisting: Boolean
): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val path = PathFactory.create(rootId, pathSegments)
val command = WriteBytesCommandFactory.create(
path,
byteOffset,
overwriteExisting,
bytes
)
val incomingMessage = InboundMessageFactory.create(
requestId,
None,
InboundPayload.WRITE_BYTES_CMD,
command
)
builder.finish(incomingMessage)
builder.dataBuffer()
}
def createReadBytesCommandPacket(
requestId: UUID,
pathSegments: Seq[String],
rootId: UUID,
byteOffset: Long,
length: Long
): ByteBuffer = {
implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val path = PathFactory.create(rootId, pathSegments)
val fileSegment = FileSegmentFactory.create(path, byteOffset, length)
val command = ReadBytesCommandFactory.create(fileSegment)
val incomingMessage = InboundMessageFactory.create(
requestId,
None,
InboundPayload.READ_BYTES_CMD,
command
)
builder.finish(incomingMessage)
builder.dataBuffer()
}
def createWriteFileCmdPacket( def createWriteFileCmdPacket(
requestId: UUID, requestId: UUID,
pathSegment: String, pathSegment: String,
rootId: UUID, rootId: UUID,
contents: Array[Byte] contents: Array[Byte]
): ByteBuffer = { ): ByteBuffer = {
implicit val builder = new FlatBufferBuilder(1024) implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val path = PathFactory.create(rootId, Seq(pathSegment)) val path = PathFactory.create(rootId, Seq(pathSegment))
val cmd = WriteFileCommandFactory.create(path, contents) val cmd = WriteFileCommandFactory.create(path, contents)
val inMsg = InboundMessageFactory.create( val inMsg = InboundMessageFactory.create(
@ -111,6 +608,7 @@ class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec {
InboundPayload.WRITE_FILE_CMD, InboundPayload.WRITE_FILE_CMD,
cmd cmd
) )
builder.finish(inMsg) builder.finish(inMsg)
builder.dataBuffer() builder.dataBuffer()
} }
@ -120,10 +618,9 @@ class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec {
pathSegment: String, pathSegment: String,
rootId: UUID rootId: UUID
): ByteBuffer = { ): ByteBuffer = {
implicit val builder = new FlatBufferBuilder(1024) implicit val builder: FlatBufferBuilder = new FlatBufferBuilder(1024)
val path = PathFactory.create(rootId, Seq(pathSegment)) val path = PathFactory.create(rootId, Seq(pathSegment))
val cmd = ReadFileCommandFactory.create(path) val cmd = ReadFileCommandFactory.create(path)
val inMsg = InboundMessageFactory.create( val inMsg = InboundMessageFactory.create(
@ -132,6 +629,7 @@ class BinaryFileManipulationTest extends BaseBinaryServerTest with FlakySpec {
InboundPayload.READ_FILE_CMD, InboundPayload.READ_FILE_CMD,
cmd cmd
) )
builder.finish(inMsg) builder.finish(inMsg)
builder.dataBuffer() builder.dataBuffer()
} }

View File

@ -0,0 +1,17 @@
package org.enso.languageserver.websocket.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.ChecksumBytesCommand
object ChecksumBytesCommandFactory {
/** Creates a new ChecksumBytesCommand.
*
* @param fileSegment the file segment to get the checksum of
* @param builder the flat buffers builder
* @return a new binary representation of a ChecksumBytesCommand.
*/
def create(fileSegment: Int)(implicit builder: FlatBufferBuilder): Int = {
ChecksumBytesCommand.createChecksumBytesCommand(builder, fileSegment)
}
}

View File

@ -0,0 +1,21 @@
package org.enso.languageserver.websocket.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.FileSegment
object FileSegmentFactory {
/** Create a new binary representation of a file segment.
*
* @param path the path to the file in which the segment exists
* @param byteOffset the start byte in the file (inclusive)
* @param segmentLength the number of bytes to read from `byteOffset`
* @param builder the flat buffers builder
* @return a new binary representation of a file segment
*/
def create(path: Int, byteOffset: Long, segmentLength: Long)(implicit
builder: FlatBufferBuilder
): Int = {
FileSegment.createFileSegment(builder, path, byteOffset, segmentLength)
}
}

View File

@ -0,0 +1,18 @@
package org.enso.languageserver.websocket.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.ReadBytesCommand
object ReadBytesCommandFactory {
/** Creates a ReadBytes command.
*
* @param fileSegment the file segment to read bytes from
* @param builder the flatbuffers builder
* @return the offset of the ReadBytesCommand
*/
def create(fileSegment: Int)(implicit builder: FlatBufferBuilder): Int = {
ReadBytesCommand.createReadBytesCommand(builder, fileSegment)
}
}

View File

@ -0,0 +1,33 @@
package org.enso.languageserver.websocket.binary.factory
import com.google.flatbuffers.FlatBufferBuilder
import org.enso.languageserver.protocol.binary.WriteBytesCommand
object WriteBytesCommandFactory {
/** Creates a WriteBytes command.
*
* @param path the path to the file into which bytes should be written
* @param byteOffset the byte offset at which to start writing
* @param overwriteExisting whether or not existing bytes should be
* overwritten
* @param bytes the bytes to be written
* @param builder the FlatBuffers builder
* @return the offset of the WriteBytesCommand.
*/
def create(
path: Int,
byteOffset: Long,
overwriteExisting: Boolean,
bytes: Array[Byte]
)(implicit builder: FlatBufferBuilder): Int = {
val bytesOff = builder.createByteVector(bytes)
WriteBytesCommand.createWriteBytesCommand(
builder,
path,
byteOffset,
overwriteExisting,
bytesOff
)
}
}

View File

@ -26,8 +26,7 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
) )
} }
"File Server" must { "Writing files" must {
"write textual content to a file" taggedAs Retry in { "write textual content to a file" taggedAs Retry in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(json"""
@ -79,10 +78,13 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"Reading files" must {
"read a file content" in { "read a file content" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/write", "method": "file/write",
"id": 4, "id": 4,
@ -95,13 +97,15 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 4, "id": 4,
"result": null "result": null
} }
""") """)
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/read", "method": "file/read",
"id": 5, "id": 5,
@ -113,7 +117,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 5, "id": 5,
"result": { "contents": "123456789" } "result": { "contents": "123456789" }
@ -123,7 +128,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"return FileNotFoundError if a file doesn't exist" in { "return FileNotFoundError if a file doesn't exist" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/read", "method": "file/read",
"id": 6, "id": 6,
@ -135,7 +141,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 6, "id": 6,
"error" : { "error" : {
@ -145,10 +152,13 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"Creating file-system entities" must {
"create a file" in { "create a file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 7, "id": 7,
@ -164,7 +174,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 7, "id": 7,
"result": null "result": null
@ -177,7 +188,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"create a directory" in { "create a directory" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 7, "id": 7,
@ -193,7 +205,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 7, "id": 7,
"result": null "result": null
@ -203,11 +216,14 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val file = Paths.get(testContentRoot.toString, "foo1", "baz").toFile val file = Paths.get(testContentRoot.toString, "foo1", "baz").toFile
file.isDirectory shouldBe true file.isDirectory shouldBe true
} }
}
"File management" must {
"delete a file" in { "delete a file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 8, "id": 8,
@ -223,7 +239,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 8, "id": 8,
"result": null "result": null
@ -234,7 +251,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
file.isFile shouldBe true file.isFile shouldBe true
// delete a file // delete a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/delete", "method": "file/delete",
"id": 9, "id": 9,
@ -246,7 +264,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 9, "id": 9,
"result": null "result": null
@ -260,7 +279,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"delete a directory" in { "delete a directory" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a directory // create a directory
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 10, "id": 10,
@ -276,7 +296,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 10, "id": 10,
"result": null "result": null
@ -287,7 +308,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
file.isDirectory shouldBe true file.isDirectory shouldBe true
// delete a directory // delete a directory
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/delete", "method": "file/delete",
"id": 11, "id": 11,
@ -299,7 +321,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 11, "id": 11,
"result": null "result": null
@ -314,7 +337,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
val file = Paths.get(testContentRoot.toString, "foo1", "bar.txt").toFile val file = Paths.get(testContentRoot.toString, "foo1", "bar.txt").toFile
file.isFile shouldBe false file.isFile shouldBe false
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/delete", "method": "file/delete",
"id": 12, "id": 12,
@ -326,7 +350,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 12, "id": 12,
"error": { "error": {
@ -344,7 +369,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
val file = Paths.get(testContentRoot.toString, "foo1", "baz").toFile val file = Paths.get(testContentRoot.toString, "foo1", "baz").toFile
file.isDirectory shouldBe false file.isDirectory shouldBe false
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/delete", "method": "file/delete",
"id": 13, "id": 13,
@ -356,7 +382,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 13, "id": 13,
"error": { "error": {
@ -373,7 +400,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"copy a file" in { "copy a file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 14, "id": 14,
@ -389,7 +417,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 14, "id": 14,
"result": null "result": null
@ -399,7 +428,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
from.toFile.isFile shouldBe true from.toFile.isFile shouldBe true
// copy a file // copy a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/copy", "method": "file/copy",
"id": 15, "id": 15,
@ -415,7 +445,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 15, "id": 15,
"result": null "result": null
@ -429,7 +460,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"copy a directory" in { "copy a directory" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 16, "id": 16,
@ -445,7 +477,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 16, "id": 16,
"result": null "result": null
@ -455,7 +488,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
from.toFile.isFile shouldBe true from.toFile.isFile shouldBe true
// copy a directory // copy a directory
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/copy", "method": "file/copy",
"id": 17, "id": 17,
@ -471,7 +505,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 17, "id": 17,
"result": null "result": null
@ -484,7 +519,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"return failure when copying nonexistent file" in { "return failure when copying nonexistent file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/copy", "method": "file/copy",
"id": 18, "id": 18,
@ -500,7 +536,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 18, "id": 18,
"error": { "error": {
@ -517,7 +554,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"move a file" in { "move a file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 19, "id": 19,
@ -533,7 +571,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 19, "id": 19,
"result": null "result": null
@ -543,7 +582,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
from.toFile.isFile shouldBe true from.toFile.isFile shouldBe true
// move a file // move a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/move", "method": "file/move",
"id": 20, "id": 20,
@ -559,7 +599,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 20, "id": 20,
"result": null "result": null
@ -574,7 +615,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"move a directory" in { "move a directory" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 21, "id": 21,
@ -590,7 +632,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 21, "id": 21,
"result": null "result": null
@ -603,7 +646,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
from.toFile.isDirectory shouldBe true from.toFile.isDirectory shouldBe true
// move a directory // move a directory
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/move", "method": "file/move",
"id": 22, "id": 22,
@ -619,7 +663,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 22, "id": 22,
"result": null "result": null
@ -633,7 +678,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"return failure when moving nonexistent file" in { "return failure when moving nonexistent file" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/move", "method": "file/move",
"id": 23, "id": 23,
@ -649,7 +695,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 23, "id": 23,
"error": { "error": {
@ -666,7 +713,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"return failure when target file exists" in { "return failure when target file exists" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a source file // create a source file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 24, "id": 24,
@ -682,7 +730,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 24, "id": 24,
"result": null "result": null
@ -692,7 +741,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
from.toFile.isFile shouldBe true from.toFile.isFile shouldBe true
// create a destination file // create a destination file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 25, "id": 25,
@ -708,7 +758,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 25, "id": 25,
"result": null "result": null
@ -718,7 +769,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
to.toFile.isFile shouldBe true to.toFile.isFile shouldBe true
// move to existing file // move to existing file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/move", "method": "file/move",
"id": 26, "id": 26,
@ -734,7 +786,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 26, "id": 26,
"error": { "error": {
@ -753,7 +806,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val path = Paths.get(testContentRoot.toString, "nonexistent.txt") val path = Paths.get(testContentRoot.toString, "nonexistent.txt")
path.toFile.exists shouldBe false path.toFile.exists shouldBe false
// check file exists // check file exists
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/exists", "method": "file/exists",
"id": 27, "id": 27,
@ -765,7 +819,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 27, "id": 27,
"result": { "result": {
@ -774,7 +829,9 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"Treeing files" must {
"get a root tree" in withCleanRoot { "get a root tree" in withCleanRoot {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
@ -786,7 +843,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// create base/a.txt // create base/a.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 28, "id": 28,
@ -802,7 +860,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 28, "id": 28,
"result": null "result": null
@ -810,7 +869,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// create base/subdir/b.txt // create base/subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 29, "id": 29,
@ -826,7 +886,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 29, "id": 29,
"result": null "result": null
@ -834,7 +895,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// get a tree of a root // get a tree of a root
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 30, "id": 30,
@ -853,7 +915,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// ├── a.txt // ├── a.txt
// └── subdir // └── subdir
// └── b.txt // └── b.txt
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 30, "id": 30,
"result": { "result": {
@ -931,7 +994,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// create base/a.txt // create base/a.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 31, "id": 31,
@ -947,7 +1011,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 31, "id": 31,
"result": null "result": null
@ -955,7 +1020,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// create base/subdir/b.txt // create base/subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 32, "id": 32,
@ -971,7 +1037,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 32, "id": 32,
"result": null "result": null
@ -979,7 +1046,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// get a tree of 'base' // get a tree of 'base'
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 33, "id": 33,
@ -997,7 +1065,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// ├── a.txt // ├── a.txt
// └── subdir // └── subdir
// └── b.txt // └── b.txt
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 33, "id": 33,
"result": { "result": {
@ -1062,7 +1131,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// create base/a.txt // create base/a.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 34, "id": 34,
@ -1078,7 +1148,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 34, "id": 34,
"result": null "result": null
@ -1086,7 +1157,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// create base/subdir/b.txt // create base/subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 35, "id": 35,
@ -1102,7 +1174,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 35, "id": 35,
"result": null "result": null
@ -1110,7 +1183,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// get a tree of 'base' // get a tree of 'base'
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 36, "id": 36,
@ -1128,7 +1202,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// base // base
// ├── a.txt // ├── a.txt
// └── subdir // └── subdir
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 36, "id": 36,
"result": { "result": {
@ -1172,7 +1247,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"get a subdirectory tree" in { "get a subdirectory tree" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create base/a.txt // create base/a.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 37, "id": 37,
@ -1188,7 +1264,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 37, "id": 37,
"result": null "result": null
@ -1196,7 +1273,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// create base/subdir/b.txt // create base/subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 38, "id": 38,
@ -1212,7 +1290,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 38, "id": 38,
"result": null "result": null
@ -1220,7 +1299,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// get a tree of 'base/subdir' // get a tree of 'base/subdir'
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 39, "id": 39,
@ -1236,7 +1316,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// //
// subdir // subdir
// └── b.txt // └── b.txt
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 39, "id": 39,
"result": { "result": {
@ -1279,7 +1360,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// create base2/subdir/b.txt // create base2/subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 40, "id": 40,
@ -1295,7 +1377,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 40, "id": 40,
"result": null "result": null
@ -1309,7 +1392,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
Files.isSymbolicLink(symlink) shouldBe true Files.isSymbolicLink(symlink) shouldBe true
// get a tree of 'base' // get a tree of 'base'
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 41, "id": 41,
@ -1328,7 +1412,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// └── subdir // └── subdir
// └── b.txt // └── b.txt
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 41, "id": 41,
"result": { "result": {
@ -1400,7 +1485,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
"get a directory tree with symlink outside of root" in { "get a directory tree with symlink outside of root" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create base3 // create base3
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 42, "id": 42,
@ -1416,7 +1502,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 42, "id": 42,
"result": null "result": null
@ -1431,7 +1518,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
Files.isSymbolicLink(symlink) shouldBe true Files.isSymbolicLink(symlink) shouldBe true
// get a tree of 'base3' // get a tree of 'base3'
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/tree", "method": "file/tree",
"id": 43, "id": 43,
@ -1447,7 +1535,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// //
// base3 // base3
// └── link // └── link
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 43, "id": 43,
"result": { "result": {
@ -1480,7 +1569,9 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"Listing directories" must {
"list a subdirectory" in { "list a subdirectory" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create: // create:
@ -1489,7 +1580,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
// └── b.txt // └── b.txt
// create subdir/b.txt // create subdir/b.txt
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 44, "id": 44,
@ -1505,7 +1597,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 44, "id": 44,
"result": null "result": null
@ -1513,7 +1606,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
""") """)
// get a tree of subdir // get a tree of subdir
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/list", "method": "file/list",
"id": 45, "id": 45,
@ -1526,7 +1620,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
// expect: b.txt // expect: b.txt
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 45, "id": 45,
"result" : { "result" : {
@ -1546,11 +1641,14 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"Getting file information" must {
"get file info" in { "get file info" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/create", "method": "file/create",
"id": 46, "id": 46,
@ -1566,7 +1664,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 46, "id": 46,
"result": null "result": null
@ -1577,7 +1676,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val attrs = Files.readAttributes(path, classOf[BasicFileAttributes]) val attrs = Files.readAttributes(path, classOf[BasicFileAttributes])
// get file info // get file info
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/info", "method": "file/info",
"id": 47, "id": 47,
@ -1589,7 +1689,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 47, "id": 47,
"result" : { "result" : {
@ -1618,7 +1719,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
val file = Paths.get(testContentRoot.toString, "nonexistent.txt").toFile val file = Paths.get(testContentRoot.toString, "nonexistent.txt").toFile
file.exists shouldBe false file.exists shouldBe false
client.send(json""" client.send(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"method": "file/info", "method": "file/info",
"id": 48, "id": 48,
@ -1630,7 +1732,8 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
} }
""") """)
client.expectJson(json""" client.expectJson(
json"""
{ "jsonrpc": "2.0", { "jsonrpc": "2.0",
"id": 48, "id": 48,
"error": { "error": {
@ -1640,7 +1743,9 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
} }
""") """)
} }
}
"file/checksum" must {
"get file checksum" in { "get file checksum" in {
val client = getInitialisedWsClient() val client = getInitialisedWsClient()
// create a file // create a file
@ -1749,5 +1854,4 @@ class FileManagerTest extends BaseServerTest with RetrySpec {
FileUtils.cleanDirectory(testContentRoot.toFile) FileUtils.cleanDirectory(testContentRoot.toFile)
test test
} }
} }