mirror of
https://github.com/enso-org/enso.git
synced 2024-12-23 18:34:03 +03:00
Refactor FileManager file commands (#609)
* refactor: create Config.scala * WIP FileSystemHandler * doc: update FileSystem list * add: zio FileSystem * update: FileSystemHandler runAsync * add: config timeouts * rename FileSystemHandler to FileSystemManager * add: ZioExec * add: FileManager router * fix: FileManager return FileWiteResult * update: FileSystemApi interface * refactor: FileSystem with Zio * impl: FileManager * impl: cleanup LanguageServer * impl: ReadFileHandler * impl: CreateFileHandler * impl: DeleteFileHandler * impl: CopyFileHandler * impl: MoveFileHandler * impl: ExistsFileHandler * impl: TreeFileHandler * fix: filemanager tests * misc: cleanup * fix: BufferRegistry tests * doc: add misc * doc: misc * feat: add ZioExec parameter to FileManager * feat: FileManager uses FileSystemApi interface * feat: FileSystem has blocking semantics * feat: FileManager props
This commit is contained in:
parent
2863498da3
commit
4864d2623f
17
build.sbt
17
build.sbt
@ -438,17 +438,16 @@ lazy val polyglot_api = project
|
|||||||
lazy val language_server = (project in file("engine/language-server"))
|
lazy val language_server = (project in file("engine/language-server"))
|
||||||
.settings(
|
.settings(
|
||||||
libraryDependencies ++= akka ++ circe ++ Seq(
|
libraryDependencies ++= akka ++ circe ++ Seq(
|
||||||
"ch.qos.logback" % "logback-classic" % "1.2.3",
|
"ch.qos.logback" % "logback-classic" % "1.2.3",
|
||||||
"io.circe" %% "circe-generic-extras" % "0.12.2",
|
"io.circe" %% "circe-generic-extras" % "0.12.2",
|
||||||
"io.circe" %% "circe-literal" % circeVersion,
|
"io.circe" %% "circe-literal" % circeVersion,
|
||||||
"org.typelevel" %% "cats-core" % "2.0.0",
|
"org.bouncycastle" % "bcpkix-jdk15on" % "1.64",
|
||||||
"org.typelevel" %% "cats-effect" % "2.0.0",
|
"dev.zio" %% "zio" % "1.0.0-RC18-2",
|
||||||
"org.bouncycastle" % "bcpkix-jdk15on" % "1.64",
|
|
||||||
"commons-io" % "commons-io" % "2.6",
|
|
||||||
akkaTestkit % Test,
|
akkaTestkit % Test,
|
||||||
"org.scalatest" %% "scalatest" % "3.2.0-M2" % Test,
|
"commons-io" % "commons-io" % "2.6",
|
||||||
"org.scalacheck" %% "scalacheck" % "1.14.0" % Test,
|
"org.scalatest" %% "scalatest" % "3.2.0-M2" % Test,
|
||||||
"org.graalvm.sdk" % "polyglot-tck" % graalVersion % "provided"
|
"org.scalacheck" %% "scalacheck" % "1.14.0" % Test,
|
||||||
|
"org.graalvm.sdk" % "polyglot-tck" % graalVersion % "provided"
|
||||||
),
|
),
|
||||||
testOptions in Test += Tests
|
testOptions in Test += Tests
|
||||||
.Argument(TestFrameworks.ScalaCheck, "-minSuccessfulTests", "1000")
|
.Argument(TestFrameworks.ScalaCheck, "-minSuccessfulTests", "1000")
|
||||||
|
@ -1,19 +1,12 @@
|
|||||||
package org.enso.languageserver
|
package org.enso.languageserver
|
||||||
|
|
||||||
import akka.actor.{Actor, ActorLogging, Stash}
|
import akka.actor.{Actor, ActorLogging, Stash}
|
||||||
import cats.effect.IO
|
|
||||||
import org.enso.languageserver.data._
|
import org.enso.languageserver.data._
|
||||||
import org.enso.languageserver.event.{
|
import org.enso.languageserver.event.{
|
||||||
ClientConnected,
|
ClientConnected,
|
||||||
ClientDisconnected,
|
ClientDisconnected,
|
||||||
ClientEvent
|
ClientEvent
|
||||||
}
|
}
|
||||||
import org.enso.languageserver.filemanager.FileManagerProtocol._
|
|
||||||
import org.enso.languageserver.filemanager.{
|
|
||||||
DirectoryTree,
|
|
||||||
FileSystemApi,
|
|
||||||
FileSystemObject
|
|
||||||
}
|
|
||||||
|
|
||||||
object LanguageProtocol {
|
object LanguageProtocol {
|
||||||
|
|
||||||
@ -27,14 +20,14 @@ object LanguageProtocol {
|
|||||||
*
|
*
|
||||||
* @param config the configuration used by this Language Server.
|
* @param config the configuration used by this Language Server.
|
||||||
*/
|
*/
|
||||||
class LanguageServer(config: Config, fs: FileSystemApi[IO])
|
class LanguageServer(config: Config)
|
||||||
extends Actor
|
extends Actor
|
||||||
with Stash
|
with Stash
|
||||||
with ActorLogging {
|
with ActorLogging {
|
||||||
import LanguageProtocol._
|
import LanguageProtocol._
|
||||||
|
|
||||||
override def preStart(): Unit = {
|
override def preStart(): Unit = {
|
||||||
context.system.eventStream.subscribe(self, classOf[ClientEvent])
|
context.system.eventStream.subscribe(self, classOf[ClientEvent]): Unit
|
||||||
}
|
}
|
||||||
|
|
||||||
override def receive: Receive = {
|
override def receive: Receive = {
|
||||||
@ -58,97 +51,5 @@ class LanguageServer(config: Config, fs: FileSystemApi[IO])
|
|||||||
case ClientDisconnected(clientId) =>
|
case ClientDisconnected(clientId) =>
|
||||||
log.info("Client disconnected [{}].", clientId)
|
log.info("Client disconnected [{}].", clientId)
|
||||||
context.become(initialized(config, env.removeClient(clientId)))
|
context.become(initialized(config, env.removeClient(clientId)))
|
||||||
|
|
||||||
case WriteFile(path, content) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
_ <- fs.write(path.toFile(rootPath), content).unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! WriteFileResult(result)
|
|
||||||
|
|
||||||
case ReadFile(path) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
content <- fs.read(path.toFile(rootPath)).unsafeRunSync()
|
|
||||||
} yield content
|
|
||||||
|
|
||||||
sender ! ReadFileResult(result)
|
|
||||||
|
|
||||||
case CreateFile(FileSystemObject.File(name, path)) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
_ <- fs.createFile(path.toFile(rootPath, name)).unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! CreateFileResult(result)
|
|
||||||
|
|
||||||
case CreateFile(FileSystemObject.Directory(name, path)) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
_ <- fs.createDirectory(path.toFile(rootPath, name)).unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! CreateFileResult(result)
|
|
||||||
|
|
||||||
case DeleteFile(path) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
_ <- fs.delete(path.toFile(rootPath)).unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! DeleteFileResult(result)
|
|
||||||
|
|
||||||
case CopyFile(from, to) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPathFrom <- config.findContentRoot(from.rootId)
|
|
||||||
rootPathTo <- config.findContentRoot(to.rootId)
|
|
||||||
_ <- fs
|
|
||||||
.copy(from.toFile(rootPathFrom), to.toFile(rootPathTo))
|
|
||||||
.unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! CopyFileResult(result)
|
|
||||||
|
|
||||||
case MoveFile(from, to) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPathFrom <- config.findContentRoot(from.rootId)
|
|
||||||
rootPathTo <- config.findContentRoot(to.rootId)
|
|
||||||
_ <- fs
|
|
||||||
.move(from.toFile(rootPathFrom), to.toFile(rootPathTo))
|
|
||||||
.unsafeRunSync()
|
|
||||||
} yield ()
|
|
||||||
|
|
||||||
sender ! MoveFileResult(result)
|
|
||||||
|
|
||||||
case ExistsFile(path) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
exists <- fs.exists(path.toFile(rootPath)).unsafeRunSync()
|
|
||||||
} yield exists
|
|
||||||
|
|
||||||
sender ! ExistsFileResult(result)
|
|
||||||
|
|
||||||
case TreeFile(path, depth) =>
|
|
||||||
val result =
|
|
||||||
for {
|
|
||||||
rootPath <- config.findContentRoot(path.rootId)
|
|
||||||
directory <- fs.tree(path.toFile(rootPath), depth).unsafeRunSync()
|
|
||||||
} yield DirectoryTree.fromDirectoryEntry(rootPath, path, directory)
|
|
||||||
|
|
||||||
sender ! TreeFileResult(result)
|
|
||||||
}
|
}
|
||||||
/* Note [Usage of unsafe methods]
|
|
||||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
|
||||||
It invokes side-effecting function, all exceptions are caught and
|
|
||||||
explicitly returned as left side of disjunction.
|
|
||||||
*/
|
|
||||||
}
|
}
|
||||||
|
@ -5,15 +5,16 @@ import java.net.URI
|
|||||||
|
|
||||||
import akka.actor.{ActorSystem, Props}
|
import akka.actor.{ActorSystem, Props}
|
||||||
import akka.stream.SystemMaterializer
|
import akka.stream.SystemMaterializer
|
||||||
import cats.effect.IO
|
|
||||||
import org.enso.jsonrpc.JsonRpcServer
|
import org.enso.jsonrpc.JsonRpcServer
|
||||||
import org.enso.languageserver.capability.CapabilityRouter
|
import org.enso.languageserver.capability.CapabilityRouter
|
||||||
import org.enso.languageserver.data.{
|
import org.enso.languageserver.data.{
|
||||||
Config,
|
Config,
|
||||||
ContentBasedVersioning,
|
ContentBasedVersioning,
|
||||||
|
FileManagerConfig,
|
||||||
Sha3_224VersionCalculator
|
Sha3_224VersionCalculator
|
||||||
}
|
}
|
||||||
import org.enso.languageserver.filemanager.{FileSystem, FileSystemApi}
|
import org.enso.languageserver.effect.ZioExec
|
||||||
|
import org.enso.languageserver.filemanager.{FileManager, FileSystem}
|
||||||
import org.enso.languageserver.protocol.{JsonRpc, ServerClientControllerFactory}
|
import org.enso.languageserver.protocol.{JsonRpc, ServerClientControllerFactory}
|
||||||
import org.enso.languageserver.runtime.RuntimeConnector
|
import org.enso.languageserver.runtime.RuntimeConnector
|
||||||
import org.enso.languageserver.text.BufferRegistry
|
import org.enso.languageserver.text.BufferRegistry
|
||||||
@ -21,6 +22,8 @@ import org.enso.polyglot.{LanguageInfo, RuntimeServerInfo}
|
|||||||
import org.graalvm.polyglot.Context
|
import org.graalvm.polyglot.Context
|
||||||
import org.graalvm.polyglot.io.MessageEndpoint
|
import org.graalvm.polyglot.io.MessageEndpoint
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A main module containing all components of th server.
|
* A main module containing all components of th server.
|
||||||
*
|
*
|
||||||
@ -29,10 +32,13 @@ import org.graalvm.polyglot.io.MessageEndpoint
|
|||||||
class MainModule(serverConfig: LanguageServerConfig) {
|
class MainModule(serverConfig: LanguageServerConfig) {
|
||||||
|
|
||||||
lazy val languageServerConfig = Config(
|
lazy val languageServerConfig = Config(
|
||||||
Map(serverConfig.contentRootUuid -> new File(serverConfig.contentRootPath))
|
Map(serverConfig.contentRootUuid -> new File(serverConfig.contentRootPath)),
|
||||||
|
FileManagerConfig(timeout = 3.seconds)
|
||||||
)
|
)
|
||||||
|
|
||||||
lazy val fileSystem: FileSystemApi[IO] = new FileSystem[IO]
|
val zioExec = ZioExec(zio.Runtime.default)
|
||||||
|
|
||||||
|
lazy val fileSystem: FileSystem = new FileSystem
|
||||||
|
|
||||||
implicit val versionCalculator: ContentBasedVersioning =
|
implicit val versionCalculator: ContentBasedVersioning =
|
||||||
Sha3_224VersionCalculator
|
Sha3_224VersionCalculator
|
||||||
@ -43,12 +49,17 @@ class MainModule(serverConfig: LanguageServerConfig) {
|
|||||||
|
|
||||||
lazy val languageServer =
|
lazy val languageServer =
|
||||||
system.actorOf(
|
system.actorOf(
|
||||||
Props(new LanguageServer(languageServerConfig, fileSystem)),
|
Props(new LanguageServer(languageServerConfig)),
|
||||||
"server"
|
"server"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
lazy val fileManager = system.actorOf(
|
||||||
|
FileManager.pool(languageServerConfig, fileSystem, zioExec),
|
||||||
|
"file-manager"
|
||||||
|
)
|
||||||
|
|
||||||
lazy val bufferRegistry =
|
lazy val bufferRegistry =
|
||||||
system.actorOf(BufferRegistry.props(languageServer), "buffer-registry")
|
system.actorOf(BufferRegistry.props(fileManager), "buffer-registry")
|
||||||
|
|
||||||
lazy val capabilityRouter =
|
lazy val capabilityRouter =
|
||||||
system.actorOf(CapabilityRouter.props(bufferRegistry), "capability-router")
|
system.actorOf(CapabilityRouter.props(bufferRegistry), "capability-router")
|
||||||
@ -76,7 +87,8 @@ class MainModule(serverConfig: LanguageServerConfig) {
|
|||||||
lazy val clientControllerFactory = new ServerClientControllerFactory(
|
lazy val clientControllerFactory = new ServerClientControllerFactory(
|
||||||
languageServer,
|
languageServer,
|
||||||
bufferRegistry,
|
bufferRegistry,
|
||||||
capabilityRouter
|
capabilityRouter,
|
||||||
|
fileManager
|
||||||
)
|
)
|
||||||
|
|
||||||
lazy val server =
|
lazy val server =
|
||||||
|
@ -0,0 +1,40 @@
|
|||||||
|
package org.enso.languageserver.data
|
||||||
|
|
||||||
|
import java.io.File
|
||||||
|
import java.util.UUID
|
||||||
|
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
ContentRootNotFound,
|
||||||
|
FileSystemFailure
|
||||||
|
}
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
case class FileManagerConfig(timeout: FiniteDuration, parallelism: Int)
|
||||||
|
|
||||||
|
object FileManagerConfig {
|
||||||
|
|
||||||
|
def apply(timeout: FiniteDuration): FileManagerConfig =
|
||||||
|
FileManagerConfig(
|
||||||
|
timeout = timeout,
|
||||||
|
parallelism = Runtime.getRuntime().availableProcessors()
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The config of the running Language Server instance.
|
||||||
|
*
|
||||||
|
* @param contentRoots a mapping between content root id and absolute path to
|
||||||
|
* the content root
|
||||||
|
*/
|
||||||
|
case class Config(
|
||||||
|
contentRoots: Map[UUID, File],
|
||||||
|
fileManager: FileManagerConfig
|
||||||
|
) {
|
||||||
|
|
||||||
|
def findContentRoot(rootId: UUID): Either[FileSystemFailure, File] =
|
||||||
|
contentRoots
|
||||||
|
.get(rootId)
|
||||||
|
.toRight(ContentRootNotFound)
|
||||||
|
|
||||||
|
}
|
@ -1,28 +1,5 @@
|
|||||||
package org.enso.languageserver.data
|
package org.enso.languageserver.data
|
||||||
|
|
||||||
import java.io.File
|
|
||||||
import java.util.UUID
|
|
||||||
|
|
||||||
import org.enso.languageserver.filemanager.{
|
|
||||||
ContentRootNotFound,
|
|
||||||
FileSystemFailure
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The config of the running Language Server instance.
|
|
||||||
*
|
|
||||||
* @param contentRoots a mapping between content root id and absolute path to
|
|
||||||
* the content root
|
|
||||||
*/
|
|
||||||
case class Config(contentRoots: Map[UUID, File] = Map.empty) {
|
|
||||||
|
|
||||||
def findContentRoot(rootId: UUID): Either[FileSystemFailure, File] =
|
|
||||||
contentRoots
|
|
||||||
.get(rootId)
|
|
||||||
.toRight(ContentRootNotFound)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The state of the running Language Server instance.
|
* The state of the running Language Server instance.
|
||||||
*
|
*
|
||||||
|
@ -0,0 +1,116 @@
|
|||||||
|
package org.enso.languageserver.effect
|
||||||
|
|
||||||
|
import java.util.concurrent.{ExecutionException, TimeoutException}
|
||||||
|
|
||||||
|
import zio._
|
||||||
|
|
||||||
|
import scala.concurrent.{Future, Promise}
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract entity that executes effects `F`.
|
||||||
|
*/
|
||||||
|
trait Exec[-F[_, _]] {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute Zio effect.
|
||||||
|
*
|
||||||
|
* @param op effect to execute
|
||||||
|
* @return a future containing either a failure or a result
|
||||||
|
*/
|
||||||
|
def exec[E, A](op: F[E, A]): Future[Either[E, A]]
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute Zio effect with timeout.
|
||||||
|
*
|
||||||
|
* @param timeout execution timeout
|
||||||
|
* @param op effect to execute
|
||||||
|
* @return a future
|
||||||
|
*/
|
||||||
|
def execTimed[E, A](
|
||||||
|
timeout: FiniteDuration,
|
||||||
|
op: ZIO[ZEnv, E, A]
|
||||||
|
): Future[Either[E, A]]
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Executor of [[ZIO]] effects.
|
||||||
|
*
|
||||||
|
* @param runtime zio runtime
|
||||||
|
*/
|
||||||
|
case class ZioExec(runtime: Runtime[ZEnv]) extends Exec[ZioExec.IO] {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute Zio effect.
|
||||||
|
*
|
||||||
|
* @param op effect to execute
|
||||||
|
* @return a future containing either a failure or a result
|
||||||
|
*/
|
||||||
|
override def exec[E, A](op: ZIO[ZEnv, E, A]): Future[Either[E, A]] = {
|
||||||
|
val promise = Promise[Either[E, A]]
|
||||||
|
runtime.unsafeRunAsync(op) {
|
||||||
|
_.fold(
|
||||||
|
ZioExec.completeFailure(promise, _),
|
||||||
|
ZioExec.completeSuccess(promise, _)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
promise.future
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Execute Zio effect with timeout.
|
||||||
|
*
|
||||||
|
* @param timeout execution timeout
|
||||||
|
* @param op effect to execute
|
||||||
|
* @return a future. On timeout future is failed with [[TimeoutException]].
|
||||||
|
* Otherwise future contains either a failure or a result.
|
||||||
|
*/
|
||||||
|
override def execTimed[E, A](
|
||||||
|
timeout: FiniteDuration,
|
||||||
|
op: ZIO[ZEnv, E, A]
|
||||||
|
): Future[Either[E, A]] = {
|
||||||
|
val promise = Promise[Either[E, A]]
|
||||||
|
runtime.unsafeRunAsync(
|
||||||
|
op.disconnect.timeout(zio.duration.Duration.fromScala(timeout))
|
||||||
|
) {
|
||||||
|
_.fold(
|
||||||
|
ZioExec.completeFailure(promise, _),
|
||||||
|
_.fold(promise.failure(ZioExec.timeoutFailure))(
|
||||||
|
a => promise.success(Right(a))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
}
|
||||||
|
promise.future
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object ZioExec {
|
||||||
|
|
||||||
|
type IO[+E, +A] = ZIO[ZEnv, E, A]
|
||||||
|
|
||||||
|
object ZioExecutionException extends ExecutionException
|
||||||
|
|
||||||
|
private def completeSuccess[E, A](
|
||||||
|
promise: Promise[Either[E, A]],
|
||||||
|
result: A
|
||||||
|
): Unit =
|
||||||
|
promise.success(Right(result))
|
||||||
|
|
||||||
|
private def completeFailure[E, A](
|
||||||
|
promise: Promise[Either[E, A]],
|
||||||
|
cause: Cause[E]
|
||||||
|
): Unit =
|
||||||
|
cause.failureOption match {
|
||||||
|
case Some(e) =>
|
||||||
|
promise.success(Left(e))
|
||||||
|
case None =>
|
||||||
|
val error = cause.defects.headOption.getOrElse(executionFailure)
|
||||||
|
promise.failure(error)
|
||||||
|
}
|
||||||
|
|
||||||
|
private val executionFailure: Throwable =
|
||||||
|
new ExecutionException("ZIO execution failed", ZioExecutionException)
|
||||||
|
|
||||||
|
private val timeoutFailure: Throwable =
|
||||||
|
new TimeoutException()
|
||||||
|
}
|
@ -0,0 +1,9 @@
|
|||||||
|
package org.enso.languageserver
|
||||||
|
|
||||||
|
import zio._
|
||||||
|
import zio.blocking.Blocking
|
||||||
|
|
||||||
|
package object effect {
|
||||||
|
|
||||||
|
type BlockingIO[+E, +A] = ZIO[Blocking, E, A]
|
||||||
|
}
|
@ -0,0 +1,149 @@
|
|||||||
|
package org.enso.languageserver.filemanager
|
||||||
|
|
||||||
|
import akka.actor.{Actor, Props}
|
||||||
|
import akka.routing.SmallestMailboxPool
|
||||||
|
import akka.pattern.pipe
|
||||||
|
import org.enso.languageserver.effect._
|
||||||
|
import org.enso.languageserver.data.Config
|
||||||
|
import zio._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Handles the [[FileManagerProtocol]] messages, executes the [[FileSystem]]
|
||||||
|
* effects and forms the responses.
|
||||||
|
*
|
||||||
|
* @param config configuration
|
||||||
|
* @param fs an instance of a [[FileSyste]] that creates the effects
|
||||||
|
* @param exec effects executor
|
||||||
|
*/
|
||||||
|
class FileManager(
|
||||||
|
config: Config,
|
||||||
|
fs: FileSystemApi[BlockingIO],
|
||||||
|
exec: Exec[BlockingIO]
|
||||||
|
) extends Actor {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = {
|
||||||
|
case FileManagerProtocol.WriteFile(path, content) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
_ <- fs.write(path.toFile(rootPath), content)
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.WriteFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.ReadFile(path) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
content <- fs.read(path.toFile(rootPath))
|
||||||
|
} yield content
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.ReadFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.CreateFile(FileSystemObject.File(name, path)) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
_ <- fs.createFile(path.toFile(rootPath, name))
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.CreateFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.CreateFile(
|
||||||
|
FileSystemObject.Directory(name, path)
|
||||||
|
) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
_ <- fs.createDirectory(path.toFile(rootPath, name))
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.CreateFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.DeleteFile(path) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
_ <- fs.delete(path.toFile(rootPath))
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.DeleteFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.CopyFile(from, to) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPathFrom <- IO.fromEither(config.findContentRoot(from.rootId))
|
||||||
|
rootPathTo <- IO.fromEither(config.findContentRoot(to.rootId))
|
||||||
|
_ <- fs.copy(from.toFile(rootPathFrom), to.toFile(rootPathTo))
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.CopyFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.MoveFile(from, to) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPathFrom <- IO.fromEither(config.findContentRoot(from.rootId))
|
||||||
|
rootPathTo <- IO.fromEither(config.findContentRoot(to.rootId))
|
||||||
|
_ <- fs.move(from.toFile(rootPathFrom), to.toFile(rootPathTo))
|
||||||
|
} yield ()
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.MoveFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.ExistsFile(path) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
exists <- fs.exists(path.toFile(rootPath))
|
||||||
|
} yield exists
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.ExistsFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
|
||||||
|
case FileManagerProtocol.TreeFile(path, depth) =>
|
||||||
|
val result =
|
||||||
|
for {
|
||||||
|
rootPath <- IO.fromEither(config.findContentRoot(path.rootId))
|
||||||
|
directory <- fs.tree(path.toFile(rootPath), depth)
|
||||||
|
} yield DirectoryTree.fromDirectoryEntry(rootPath, path, directory)
|
||||||
|
exec
|
||||||
|
.execTimed(config.fileManager.timeout, result)
|
||||||
|
.map(FileManagerProtocol.TreeFileResult)
|
||||||
|
.pipeTo(sender())
|
||||||
|
()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object FileManager {
|
||||||
|
|
||||||
|
def props(config: Config, fs: FileSystem, exec: Exec[BlockingIO]): Props =
|
||||||
|
Props(new FileManager(config, fs, exec))
|
||||||
|
|
||||||
|
def pool(config: Config, fs: FileSystem, exec: Exec[BlockingIO]): Props =
|
||||||
|
SmallestMailboxPool(config.fileManager.parallelism)
|
||||||
|
.props(props(config, fs, exec))
|
||||||
|
}
|
@ -1,12 +1,12 @@
|
|||||||
package org.enso.languageserver.filemanager
|
package org.enso.languageserver.filemanager
|
||||||
|
|
||||||
import java.io.{File, FileNotFoundException, IOException}
|
import java.io.{File, FileNotFoundException}
|
||||||
import java.nio.file._
|
import java.nio.file._
|
||||||
|
|
||||||
import cats.data.EitherT
|
|
||||||
import cats.effect.Sync
|
|
||||||
import cats.implicits._
|
|
||||||
import org.apache.commons.io.{FileExistsException, FileUtils}
|
import org.apache.commons.io.{FileExistsException, FileUtils}
|
||||||
|
import org.enso.languageserver.effect.BlockingIO
|
||||||
|
import zio._
|
||||||
|
import zio.blocking.effectBlocking
|
||||||
|
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
|
|
||||||
@ -15,7 +15,7 @@ import scala.collection.mutable
|
|||||||
*
|
*
|
||||||
* @tparam F represents target monad
|
* @tparam F represents target monad
|
||||||
*/
|
*/
|
||||||
class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
class FileSystem extends FileSystemApi[BlockingIO] {
|
||||||
|
|
||||||
import FileSystemApi._
|
import FileSystemApi._
|
||||||
|
|
||||||
@ -23,20 +23,15 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* Writes textual content to a file.
|
* Writes textual content to a file.
|
||||||
*
|
*
|
||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @param content a textual content of the file
|
* @param content a textual content of the file
|
||||||
* @return either FileSystemFailure or Unit
|
* @return either FileSystemFailure or Unit
|
||||||
*/
|
*/
|
||||||
override def write(
|
override def write(
|
||||||
file: File,
|
file: File,
|
||||||
content: String
|
content: String
|
||||||
): F[Either[FileSystemFailure, Unit]] =
|
): BlockingIO[FileSystemFailure, Unit] =
|
||||||
Sync[F].delay {
|
effectBlocking(FileUtils.write(file, content, "UTF-8"))
|
||||||
Either
|
.mapError(errorHandling)
|
||||||
.catchOnly[IOException] {
|
|
||||||
FileUtils.write(file, content, "UTF-8")
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads the contents of a textual file.
|
* Reads the contents of a textual file.
|
||||||
@ -44,14 +39,9 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @return either [[FileSystemFailure]] or the content of a file as a String
|
* @return either [[FileSystemFailure]] or the content of a file as a String
|
||||||
*/
|
*/
|
||||||
override def read(file: File): F[Either[FileSystemFailure, String]] =
|
override def read(file: File): BlockingIO[FileSystemFailure, String] =
|
||||||
Sync[F].delay {
|
effectBlocking(FileUtils.readFileToString(file, "UTF-8"))
|
||||||
Either
|
.mapError(errorHandling)
|
||||||
.catchOnly[IOException] {
|
|
||||||
FileUtils.readFileToString(file, "UTF-8")
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes the specified file or directory recursively.
|
* Deletes the specified file or directory recursively.
|
||||||
@ -59,18 +49,14 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* @param file path to the file or directory
|
* @param file path to the file or directory
|
||||||
* @return either [[FileSystemFailure]] or Unit
|
* @return either [[FileSystemFailure]] or Unit
|
||||||
*/
|
*/
|
||||||
def delete(file: File): F[Either[FileSystemFailure, Unit]] =
|
def delete(file: File): BlockingIO[FileSystemFailure, Unit] =
|
||||||
Sync[F].delay {
|
effectBlocking({
|
||||||
Either
|
if (file.isDirectory) {
|
||||||
.catchOnly[IOException] {
|
FileUtils.deleteDirectory(file)
|
||||||
if (file.isDirectory) {
|
} else {
|
||||||
FileUtils.deleteDirectory(file)
|
Files.delete(file.toPath)
|
||||||
} else {
|
}
|
||||||
Files.delete(file.toPath)
|
}).mapError(errorHandling)
|
||||||
}
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an empty file with parent directory.
|
* Creates an empty file with parent directory.
|
||||||
@ -78,25 +64,15 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
override def createFile(file: File): F[Either[FileSystemFailure, Unit]] = {
|
override def createFile(file: File): BlockingIO[FileSystemFailure, Unit] =
|
||||||
val op =
|
for {
|
||||||
for {
|
_ <- createDirectory(file.getParentFile)
|
||||||
_ <- EitherT { createDirectory(file.getParentFile) }
|
_ <- createEmptyFile(file)
|
||||||
_ <- EitherT { createEmptyFile(file) }
|
} yield ()
|
||||||
} yield ()
|
|
||||||
|
|
||||||
op.value
|
private def createEmptyFile(file: File): BlockingIO[FileSystemFailure, Unit] =
|
||||||
}
|
effectBlocking(file.createNewFile(): Unit)
|
||||||
|
.mapError(errorHandling)
|
||||||
private def createEmptyFile(file: File): F[Either[FileSystemFailure, Unit]] =
|
|
||||||
Sync[F].delay {
|
|
||||||
Either
|
|
||||||
.catchOnly[IOException] {
|
|
||||||
file.createNewFile()
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
.map(_ => ())
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a directory, including any necessary but nonexistent parent
|
* Creates a directory, including any necessary but nonexistent parent
|
||||||
@ -107,14 +83,9 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
*/
|
*/
|
||||||
override def createDirectory(
|
override def createDirectory(
|
||||||
file: File
|
file: File
|
||||||
): F[Either[FileSystemFailure, Unit]] =
|
): BlockingIO[FileSystemFailure, Unit] =
|
||||||
Sync[F].delay {
|
effectBlocking(FileUtils.forceMkdir(file))
|
||||||
Either
|
.mapError(errorHandling)
|
||||||
.catchOnly[IOException] {
|
|
||||||
FileUtils.forceMkdir(file)
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Copy a file or directory recursively.
|
* Copy a file or directory recursively.
|
||||||
@ -125,25 +96,19 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* be a directory.
|
* be a directory.
|
||||||
* @return either [[FileSystemFailure]] or Unit
|
* @return either [[FileSystemFailure]] or Unit
|
||||||
*/
|
*/
|
||||||
override def copy(
|
override def copy(from: File, to: File): BlockingIO[FileSystemFailure, Unit] =
|
||||||
from: File,
|
if (from.isDirectory && to.isFile) {
|
||||||
to: File
|
IO.fail(FileExists)
|
||||||
): F[Either[FileSystemFailure, Unit]] =
|
} else {
|
||||||
Sync[F].delay {
|
effectBlocking({
|
||||||
if (from.isDirectory && to.isFile) {
|
if (from.isFile && to.isDirectory) {
|
||||||
Left(FileExists)
|
FileUtils.copyFileToDirectory(from, to)
|
||||||
} else {
|
} else if (from.isDirectory) {
|
||||||
Either
|
FileUtils.copyDirectory(from, to)
|
||||||
.catchOnly[IOException] {
|
} else {
|
||||||
if (from.isFile && to.isDirectory) {
|
FileUtils.copyFile(from, to)
|
||||||
FileUtils.copyFileToDirectory(from, to)
|
}
|
||||||
} else if (from.isDirectory) {
|
}).mapError(errorHandling)
|
||||||
FileUtils.copyDirectory(from, to)
|
|
||||||
} else {
|
|
||||||
FileUtils.copyFile(from, to)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}.leftMap(errorHandling)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -153,24 +118,17 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* @param to a path to the destination
|
* @param to a path to the destination
|
||||||
* @return either [[FileSystemFailure]] or Unit
|
* @return either [[FileSystemFailure]] or Unit
|
||||||
*/
|
*/
|
||||||
override def move(
|
override def move(from: File, to: File): BlockingIO[FileSystemFailure, Unit] =
|
||||||
from: File,
|
effectBlocking({
|
||||||
to: File
|
if (to.isDirectory) {
|
||||||
): F[Either[FileSystemFailure, Unit]] =
|
val createDestDir = false
|
||||||
Sync[F].delay {
|
FileUtils.moveToDirectory(from, to, createDestDir)
|
||||||
Either
|
} else if (from.isDirectory) {
|
||||||
.catchOnly[IOException] {
|
FileUtils.moveDirectory(from, to)
|
||||||
if (to.isDirectory) {
|
} else {
|
||||||
val createDestDir = false
|
FileUtils.moveFile(from, to)
|
||||||
FileUtils.moveToDirectory(from, to, createDestDir)
|
}
|
||||||
} else if (from.isDirectory) {
|
}).mapError(errorHandling)
|
||||||
FileUtils.moveDirectory(from, to)
|
|
||||||
} else {
|
|
||||||
FileUtils.moveFile(from, to)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the specified file exists.
|
* Checks if the specified file exists.
|
||||||
@ -178,40 +136,31 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
* @param file path to the file or directory
|
* @param file path to the file or directory
|
||||||
* @return either [[FileSystemFailure]] or file existence flag
|
* @return either [[FileSystemFailure]] or file existence flag
|
||||||
*/
|
*/
|
||||||
override def exists(file: File): F[Either[FileSystemFailure, Boolean]] =
|
override def exists(file: File): BlockingIO[FileSystemFailure, Boolean] =
|
||||||
Sync[F].delay {
|
effectBlocking(Files.exists(file.toPath))
|
||||||
Either
|
.mapError(errorHandling)
|
||||||
.catchOnly[IOException] {
|
|
||||||
Files.exists(file.toPath)
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def list(path: File): F[Either[FileSystemFailure, Vector[Entry]]] =
|
override def list(path: File): BlockingIO[FileSystemFailure, Vector[Entry]] =
|
||||||
Sync[F].delay {
|
if (path.exists) {
|
||||||
if (path.exists) {
|
if (path.isDirectory) {
|
||||||
if (path.isDirectory) {
|
effectBlocking({
|
||||||
Either
|
FileSystem
|
||||||
.catchOnly[IOException] {
|
.list(path.toPath)
|
||||||
FileSystem
|
.map {
|
||||||
.list(path.toPath)
|
case SymbolicLinkEntry(path, _) =>
|
||||||
.map {
|
FileSystem.readSymbolicLink(path)
|
||||||
case SymbolicLinkEntry(path, _) =>
|
case entry => entry
|
||||||
FileSystem.readSymbolicLink(path)
|
|
||||||
case entry => entry
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
.leftMap(errorHandling)
|
}).mapError(errorHandling)
|
||||||
} else {
|
|
||||||
Left(NotDirectory)
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
Left(FileNotFound)
|
IO.fail(NotDirectory)
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
IO.fail(FileNotFound)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns contents of a given path.
|
* Returns tree of a given path.
|
||||||
*
|
*
|
||||||
* @param path to the file system object
|
* @param path to the file system object
|
||||||
* @param depth maximum depth of a directory tree
|
* @param depth maximum depth of a directory tree
|
||||||
@ -220,34 +169,30 @@ class FileSystem[F[_]: Sync] extends FileSystemApi[F] {
|
|||||||
override def tree(
|
override def tree(
|
||||||
path: File,
|
path: File,
|
||||||
depth: Option[Int]
|
depth: Option[Int]
|
||||||
): F[Either[FileSystemFailure, DirectoryEntry]] = {
|
): BlockingIO[FileSystemFailure, DirectoryEntry] = {
|
||||||
Sync[F].delay {
|
val limit = FileSystem.Depth(depth)
|
||||||
val limit = FileSystem.Depth(depth)
|
if (path.exists && limit.canGoDeeper) {
|
||||||
if (path.exists && limit.canGoDeeper) {
|
if (path.isDirectory) {
|
||||||
if (path.isDirectory) {
|
effectBlocking({
|
||||||
Either
|
val directory = DirectoryEntry.empty(path.toPath)
|
||||||
.catchOnly[IOException] {
|
FileSystem.readDirectoryEntry(
|
||||||
val directory = DirectoryEntry.empty(path.toPath)
|
directory,
|
||||||
FileSystem.readDirectoryEntry(
|
limit.goDeeper,
|
||||||
directory,
|
Vector(),
|
||||||
limit.goDeeper,
|
mutable.Queue().appendAll(FileSystem.list(path.toPath)),
|
||||||
Vector(),
|
mutable.Queue()
|
||||||
mutable.Queue().appendAll(FileSystem.list(path.toPath)),
|
)
|
||||||
mutable.Queue()
|
directory
|
||||||
)
|
}).mapError(errorHandling)
|
||||||
directory
|
|
||||||
}
|
|
||||||
.leftMap(errorHandling)
|
|
||||||
} else {
|
|
||||||
Left(NotDirectory)
|
|
||||||
}
|
|
||||||
} else {
|
} else {
|
||||||
Left(FileNotFound)
|
IO.fail(NotDirectory)
|
||||||
}
|
}
|
||||||
|
} else {
|
||||||
|
IO.fail(FileNotFound)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private val errorHandling: IOException => FileSystemFailure = {
|
private val errorHandling: Throwable => FileSystemFailure = {
|
||||||
case _: FileNotFoundException => FileNotFound
|
case _: FileNotFoundException => FileNotFound
|
||||||
case _: NoSuchFileException => FileNotFound
|
case _: NoSuchFileException => FileNotFound
|
||||||
case _: FileExistsException => FileExists
|
case _: FileExistsException => FileExists
|
||||||
@ -308,6 +253,9 @@ object FileSystem {
|
|||||||
visited: Vector[SymbolicLinkEntry]
|
visited: Vector[SymbolicLinkEntry]
|
||||||
)
|
)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read an entry without following the symlinks.
|
||||||
|
*/
|
||||||
private def readEntry(path: Path): Entry = {
|
private def readEntry(path: Path): Entry = {
|
||||||
if (Files.isRegularFile(path, LinkOption.NOFOLLOW_LINKS)) {
|
if (Files.isRegularFile(path, LinkOption.NOFOLLOW_LINKS)) {
|
||||||
FileEntry(path)
|
FileEntry(path)
|
||||||
@ -325,6 +273,9 @@ object FileSystem {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read the target of a symlink.
|
||||||
|
*/
|
||||||
private def readSymbolicLink(path: Path): Entry = {
|
private def readSymbolicLink(path: Path): Entry = {
|
||||||
if (Files.isRegularFile(path)) {
|
if (Files.isRegularFile(path)) {
|
||||||
FileEntry(path)
|
FileEntry(path)
|
||||||
@ -339,6 +290,7 @@ object FileSystem {
|
|||||||
* Returns the entries of the provided path. Symlinks are not resolved.
|
* Returns the entries of the provided path. Symlinks are not resolved.
|
||||||
*
|
*
|
||||||
* @param path to the directory
|
* @param path to the directory
|
||||||
|
* @return list of entries
|
||||||
*/
|
*/
|
||||||
private def list(path: Path): Vector[Entry] = {
|
private def list(path: Path): Vector[Entry] = {
|
||||||
def accumulator(acc: Vector[Entry], path: Path): Vector[Entry] =
|
def accumulator(acc: Vector[Entry], path: Path): Vector[Entry] =
|
||||||
|
@ -10,7 +10,7 @@ import scala.collection.mutable.ArrayBuffer
|
|||||||
*
|
*
|
||||||
* @tparam F represents target monad
|
* @tparam F represents target monad
|
||||||
*/
|
*/
|
||||||
trait FileSystemApi[F[_]] {
|
trait FileSystemApi[F[_, _]] {
|
||||||
|
|
||||||
import FileSystemApi._
|
import FileSystemApi._
|
||||||
|
|
||||||
@ -24,7 +24,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
def write(
|
def write(
|
||||||
file: File,
|
file: File,
|
||||||
content: String
|
content: String
|
||||||
): F[Either[FileSystemFailure, Unit]]
|
): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads the contents of a textual file.
|
* Reads the contents of a textual file.
|
||||||
@ -32,7 +32,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @return either [[FileSystemFailure]] or the content of a file as a String
|
* @return either [[FileSystemFailure]] or the content of a file as a String
|
||||||
*/
|
*/
|
||||||
def read(file: File): F[Either[FileSystemFailure, String]]
|
def read(file: File): F[FileSystemFailure, String]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deletes the specified file or directory recursively.
|
* Deletes the specified file or directory recursively.
|
||||||
@ -40,7 +40,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param file path to the file or directory
|
* @param file path to the file or directory
|
||||||
* @return either [[FileSystemFailure]] or Unit
|
* @return either [[FileSystemFailure]] or Unit
|
||||||
*/
|
*/
|
||||||
def delete(file: File): F[Either[FileSystemFailure, Unit]]
|
def delete(file: File): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates an empty file with parent directory.
|
* Creates an empty file with parent directory.
|
||||||
@ -48,7 +48,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
def createFile(file: File): F[Either[FileSystemFailure, Unit]]
|
def createFile(file: File): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a directory, including any necessary but nonexistent parent
|
* Creates a directory, including any necessary but nonexistent parent
|
||||||
@ -57,7 +57,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param file path to the file
|
* @param file path to the file
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
def createDirectory(file: File): F[Either[FileSystemFailure, Unit]]
|
def createDirectory(file: File): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Copy a file or directory recursively
|
* Copy a file or directory recursively
|
||||||
@ -69,7 +69,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
def copy(
|
def copy(
|
||||||
from: File,
|
from: File,
|
||||||
to: File
|
to: File
|
||||||
): F[Either[FileSystemFailure, Unit]]
|
): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Move a file or directory recursively
|
* Move a file or directory recursively
|
||||||
@ -81,7 +81,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
def move(
|
def move(
|
||||||
from: File,
|
from: File,
|
||||||
to: File
|
to: File
|
||||||
): F[Either[FileSystemFailure, Unit]]
|
): F[FileSystemFailure, Unit]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if the specified file exists.
|
* Checks if the specified file exists.
|
||||||
@ -89,7 +89,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param file path to the file or directory
|
* @param file path to the file or directory
|
||||||
* @return either [[FileSystemFailure]] or file existence flag
|
* @return either [[FileSystemFailure]] or file existence flag
|
||||||
*/
|
*/
|
||||||
def exists(file: File): F[Either[FileSystemFailure, Boolean]]
|
def exists(file: File): F[FileSystemFailure, Boolean]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* List contents of a given path.
|
* List contents of a given path.
|
||||||
@ -97,7 +97,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
* @param path to the file system object
|
* @param path to the file system object
|
||||||
* @return either [[FileSystemFailure]] or list of entries
|
* @return either [[FileSystemFailure]] or list of entries
|
||||||
*/
|
*/
|
||||||
def list(path: File): F[Either[FileSystemFailure, Vector[Entry]]]
|
def list(path: File): F[FileSystemFailure, Vector[Entry]]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns contents of a given path.
|
* Returns contents of a given path.
|
||||||
@ -109,7 +109,7 @@ trait FileSystemApi[F[_]] {
|
|||||||
def tree(
|
def tree(
|
||||||
path: File,
|
path: File,
|
||||||
depth: Option[Int]
|
depth: Option[Int]
|
||||||
): F[Either[FileSystemFailure, DirectoryEntry]]
|
): F[FileSystemFailure, DirectoryEntry]
|
||||||
}
|
}
|
||||||
|
|
||||||
object FileSystemApi {
|
object FileSystemApi {
|
||||||
|
@ -3,9 +3,7 @@ package org.enso.languageserver.protocol
|
|||||||
import java.util.UUID
|
import java.util.UUID
|
||||||
|
|
||||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Stash}
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Stash}
|
||||||
import akka.pattern.ask
|
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
import org.enso.jsonrpc.Errors.ServiceError
|
|
||||||
import org.enso.jsonrpc._
|
import org.enso.jsonrpc._
|
||||||
import org.enso.languageserver.capability.CapabilityApi.{
|
import org.enso.languageserver.capability.CapabilityApi.{
|
||||||
AcquireCapability,
|
AcquireCapability,
|
||||||
@ -17,20 +15,11 @@ import org.enso.languageserver.capability.CapabilityProtocol
|
|||||||
import org.enso.languageserver.data.Client
|
import org.enso.languageserver.data.Client
|
||||||
import org.enso.languageserver.event.{ClientConnected, ClientDisconnected}
|
import org.enso.languageserver.event.{ClientConnected, ClientDisconnected}
|
||||||
import org.enso.languageserver.filemanager.FileManagerApi._
|
import org.enso.languageserver.filemanager.FileManagerApi._
|
||||||
import org.enso.languageserver.filemanager.FileManagerProtocol.{
|
|
||||||
CreateFileResult,
|
|
||||||
WriteFileResult
|
|
||||||
}
|
|
||||||
import org.enso.languageserver.filemanager.{
|
|
||||||
FileManagerProtocol,
|
|
||||||
FileSystemFailureMapper
|
|
||||||
}
|
|
||||||
import org.enso.languageserver.requesthandler._
|
import org.enso.languageserver.requesthandler._
|
||||||
import org.enso.languageserver.text.TextApi._
|
import org.enso.languageserver.text.TextApi._
|
||||||
import org.enso.languageserver.text.TextProtocol
|
import org.enso.languageserver.text.TextProtocol
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.util.{Failure, Success}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An actor handling communications between a single client and the language
|
* An actor handling communications between a single client and the language
|
||||||
@ -47,13 +36,12 @@ class ClientController(
|
|||||||
val server: ActorRef,
|
val server: ActorRef,
|
||||||
val bufferRegistry: ActorRef,
|
val bufferRegistry: ActorRef,
|
||||||
val capabilityRouter: ActorRef,
|
val capabilityRouter: ActorRef,
|
||||||
|
val fileManager: ActorRef,
|
||||||
requestTimeout: FiniteDuration = 10.seconds
|
requestTimeout: FiniteDuration = 10.seconds
|
||||||
) extends Actor
|
) extends Actor
|
||||||
with Stash
|
with Stash
|
||||||
with ActorLogging {
|
with ActorLogging {
|
||||||
|
|
||||||
import context.dispatcher
|
|
||||||
|
|
||||||
implicit val timeout = Timeout(requestTimeout)
|
implicit val timeout = Timeout(requestTimeout)
|
||||||
|
|
||||||
private val client = Client(clientId, self)
|
private val client = Client(clientId, self)
|
||||||
@ -69,7 +57,15 @@ class ClientController(
|
|||||||
.props(bufferRegistry, requestTimeout, client),
|
.props(bufferRegistry, requestTimeout, client),
|
||||||
ApplyEdit -> ApplyEditHandler
|
ApplyEdit -> ApplyEditHandler
|
||||||
.props(bufferRegistry, requestTimeout, client),
|
.props(bufferRegistry, requestTimeout, client),
|
||||||
SaveFile -> SaveFileHandler.props(bufferRegistry, requestTimeout, client)
|
SaveFile -> SaveFileHandler.props(bufferRegistry, requestTimeout, client),
|
||||||
|
WriteFile -> file.WriteFileHandler.props(requestTimeout, fileManager),
|
||||||
|
ReadFile -> file.ReadFileHandler.props(requestTimeout, fileManager),
|
||||||
|
CreateFile -> file.CreateFileHandler.props(requestTimeout, fileManager),
|
||||||
|
DeleteFile -> file.DeleteFileHandler.props(requestTimeout, fileManager),
|
||||||
|
CopyFile -> file.CopyFileHandler.props(requestTimeout, fileManager),
|
||||||
|
MoveFile -> file.MoveFileHandler.props(requestTimeout, fileManager),
|
||||||
|
ExistsFile -> file.ExistsFileHandler.props(requestTimeout, fileManager),
|
||||||
|
TreeFile -> file.TreeFileHandler.props(requestTimeout, fileManager)
|
||||||
)
|
)
|
||||||
|
|
||||||
override def unhandled(message: Any): Unit =
|
override def unhandled(message: Any): Unit =
|
||||||
@ -101,209 +97,7 @@ class ClientController(
|
|||||||
case r @ Request(method, _, _) if (requestHandlers.contains(method)) =>
|
case r @ Request(method, _, _) if (requestHandlers.contains(method)) =>
|
||||||
val handler = context.actorOf(requestHandlers(method))
|
val handler = context.actorOf(requestHandlers(method))
|
||||||
handler.forward(r)
|
handler.forward(r)
|
||||||
|
|
||||||
case Request(WriteFile, id, params: WriteFile.Params) =>
|
|
||||||
writeFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(ReadFile, id, params: ReadFile.Params) =>
|
|
||||||
readFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(CreateFile, id, params: CreateFile.Params) =>
|
|
||||||
createFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(DeleteFile, id, params: DeleteFile.Params) =>
|
|
||||||
deleteFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(CopyFile, id, params: CopyFile.Params) =>
|
|
||||||
copyFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(MoveFile, id, params: MoveFile.Params) =>
|
|
||||||
moveFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(ExistsFile, id, params: ExistsFile.Params) =>
|
|
||||||
existsFile(webActor, id, params)
|
|
||||||
|
|
||||||
case Request(TreeFile, id, params: TreeFile.Params) =>
|
|
||||||
treeFile(webActor, id, params)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def readFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: ReadFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.ReadFile(params.path)).onComplete {
|
|
||||||
case Success(
|
|
||||||
FileManagerProtocol.ReadFileResult(Right(content: String))
|
|
||||||
) =>
|
|
||||||
webActor ! ResponseResult(ReadFile, id, ReadFile.Result(content))
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.ReadFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occurred during reading a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def writeFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: WriteFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.WriteFile(params.path, params.contents))
|
|
||||||
.onComplete {
|
|
||||||
case Success(WriteFileResult(Right(()))) =>
|
|
||||||
webActor ! ResponseResult(WriteFile, id, Unused)
|
|
||||||
|
|
||||||
case Success(WriteFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occurred during writing to a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def createFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: CreateFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.CreateFile(params.`object`))
|
|
||||||
.onComplete {
|
|
||||||
case Success(CreateFileResult(Right(()))) =>
|
|
||||||
webActor ! ResponseResult(CreateFile, id, Unused)
|
|
||||||
|
|
||||||
case Success(CreateFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occurred during creating a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def deleteFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: DeleteFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.DeleteFile(params.path))
|
|
||||||
.onComplete {
|
|
||||||
case Success(FileManagerProtocol.DeleteFileResult(Right(()))) =>
|
|
||||||
webActor ! ResponseResult(DeleteFile, id, Unused)
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.DeleteFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occurred during deleting a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def copyFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: CopyFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.CopyFile(params.from, params.to))
|
|
||||||
.onComplete {
|
|
||||||
case Success(FileManagerProtocol.CopyFileResult(Right(()))) =>
|
|
||||||
webActor ! ResponseResult(CopyFile, id, Unused)
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.CopyFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occured during copying a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def moveFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: MoveFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.MoveFile(params.from, params.to))
|
|
||||||
.onComplete {
|
|
||||||
case Success(FileManagerProtocol.MoveFileResult(Right(()))) =>
|
|
||||||
webActor ! ResponseResult(MoveFile, id, Unused)
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.MoveFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occured during moving a file", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def existsFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: ExistsFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.ExistsFile(params.path))
|
|
||||||
.onComplete {
|
|
||||||
case Success(FileManagerProtocol.ExistsFileResult(Right(exists))) =>
|
|
||||||
webActor ! ResponseResult(ExistsFile, id, ExistsFile.Result(exists))
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.ExistsFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occurred during exists file command", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def treeFile(
|
|
||||||
webActor: ActorRef,
|
|
||||||
id: Id,
|
|
||||||
params: TreeFile.Params
|
|
||||||
): Unit = {
|
|
||||||
(server ? FileManagerProtocol.TreeFile(params.path, params.depth))
|
|
||||||
.onComplete {
|
|
||||||
case Success(FileManagerProtocol.TreeFileResult(Right(tree))) =>
|
|
||||||
webActor ! ResponseResult(TreeFile, id, TreeFile.Result(tree))
|
|
||||||
|
|
||||||
case Success(FileManagerProtocol.TreeFileResult(Left(failure))) =>
|
|
||||||
webActor ! ResponseError(
|
|
||||||
Some(id),
|
|
||||||
FileSystemFailureMapper.mapFailure(failure)
|
|
||||||
)
|
|
||||||
|
|
||||||
case Failure(th) =>
|
|
||||||
log.error("An exception occured during a tree operation", th)
|
|
||||||
webActor ! ResponseError(Some(id), ServiceError)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object ClientController {
|
object ClientController {
|
||||||
@ -323,6 +117,7 @@ object ClientController {
|
|||||||
server: ActorRef,
|
server: ActorRef,
|
||||||
bufferRegistry: ActorRef,
|
bufferRegistry: ActorRef,
|
||||||
capabilityRouter: ActorRef,
|
capabilityRouter: ActorRef,
|
||||||
|
fileManager: ActorRef,
|
||||||
requestTimeout: FiniteDuration = 10.seconds
|
requestTimeout: FiniteDuration = 10.seconds
|
||||||
): Props =
|
): Props =
|
||||||
Props(
|
Props(
|
||||||
@ -331,6 +126,7 @@ object ClientController {
|
|||||||
server,
|
server,
|
||||||
bufferRegistry,
|
bufferRegistry,
|
||||||
capabilityRouter,
|
capabilityRouter,
|
||||||
|
fileManager,
|
||||||
requestTimeout
|
requestTimeout
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
|
@ -16,7 +16,8 @@ import org.enso.jsonrpc.ClientControllerFactory
|
|||||||
class ServerClientControllerFactory(
|
class ServerClientControllerFactory(
|
||||||
server: ActorRef,
|
server: ActorRef,
|
||||||
bufferRegistry: ActorRef,
|
bufferRegistry: ActorRef,
|
||||||
capabilityRouter: ActorRef
|
capabilityRouter: ActorRef,
|
||||||
|
fileManager: ActorRef
|
||||||
)(implicit system: ActorSystem)
|
)(implicit system: ActorSystem)
|
||||||
extends ClientControllerFactory {
|
extends ClientControllerFactory {
|
||||||
|
|
||||||
@ -28,6 +29,7 @@ class ServerClientControllerFactory(
|
|||||||
*/
|
*/
|
||||||
override def createClientController(clientId: UUID): ActorRef =
|
override def createClientController(clientId: UUID): ActorRef =
|
||||||
system.actorOf(
|
system.actorOf(
|
||||||
ClientController.props(clientId, server, bufferRegistry, capabilityRouter)
|
ClientController
|
||||||
|
.props(clientId, server, bufferRegistry, capabilityRouter, fileManager)
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.CopyFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class CopyFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(CopyFile, id, params: CopyFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.CopyFile(params.from, params.to)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $CopyFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.CopyFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.CopyFileResult(Right(())) =>
|
||||||
|
replyTo ! ResponseResult(CopyFile, id, Unused)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object CopyFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new CopyFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.CreateFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class CreateFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(CreateFile, id, params: CreateFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.CreateFile(params.`object`)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $CreateFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.CreateFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.CreateFileResult(Right(())) =>
|
||||||
|
replyTo ! ResponseResult(CreateFile, id, Unused)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object CreateFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new CreateFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.DeleteFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class DeleteFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(DeleteFile, id, params: DeleteFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.DeleteFile(params.path)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $DeleteFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.DeleteFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.DeleteFileResult(Right(())) =>
|
||||||
|
replyTo ! ResponseResult(DeleteFile, id, Unused)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object DeleteFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new DeleteFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.ExistsFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class ExistsFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(ExistsFile, id, params: ExistsFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.ExistsFile(params.path)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $ExistsFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.ExistsFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.ExistsFileResult(Right(result)) =>
|
||||||
|
replyTo ! ResponseResult(ExistsFile, id, ExistsFile.Result(result))
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object ExistsFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new ExistsFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.MoveFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class MoveFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(MoveFile, id, params: MoveFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.MoveFile(params.from, params.to)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $MoveFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.MoveFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.MoveFileResult(Right(())) =>
|
||||||
|
replyTo ! ResponseResult(MoveFile, id, Unused)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object MoveFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new MoveFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.ReadFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class ReadFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(ReadFile, id, params: ReadFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.ReadFile(params.path)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $ReadFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.ReadFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.ReadFileResult(Right(content)) =>
|
||||||
|
replyTo ! ResponseResult(ReadFile, id, ReadFile.Result(content))
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object ReadFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new ReadFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.TreeFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class TreeFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(TreeFile, id, params: TreeFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.TreeFile(params.path, params.depth)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $TreeFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.TreeFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.TreeFileResult(Right(result)) =>
|
||||||
|
replyTo ! ResponseResult(TreeFile, id, TreeFile.Result(result))
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object TreeFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new TreeFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -0,0 +1,60 @@
|
|||||||
|
package org.enso.languageserver.requesthandler.file
|
||||||
|
|
||||||
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Status}
|
||||||
|
import org.enso.jsonrpc.Errors.ServiceError
|
||||||
|
import org.enso.jsonrpc._
|
||||||
|
import org.enso.languageserver.filemanager.{
|
||||||
|
FileManagerProtocol,
|
||||||
|
FileSystemFailureMapper
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.FileManagerApi.WriteFile
|
||||||
|
import org.enso.languageserver.requesthandler.RequestTimeout
|
||||||
|
|
||||||
|
import scala.concurrent.duration.FiniteDuration
|
||||||
|
|
||||||
|
class WriteFileHandler(requestTimeout: FiniteDuration, fileManager: ActorRef)
|
||||||
|
extends Actor
|
||||||
|
with ActorLogging {
|
||||||
|
|
||||||
|
import context.dispatcher
|
||||||
|
|
||||||
|
override def receive: Receive = requestStage
|
||||||
|
|
||||||
|
private def requestStage: Receive = {
|
||||||
|
case Request(WriteFile, id, params: WriteFile.Params) =>
|
||||||
|
fileManager ! FileManagerProtocol.WriteFile(params.path, params.contents)
|
||||||
|
context.system.scheduler
|
||||||
|
.scheduleOnce(requestTimeout, self, RequestTimeout)
|
||||||
|
context.become(responseStage(id, sender()))
|
||||||
|
}
|
||||||
|
|
||||||
|
private def responseStage(id: Id, replyTo: ActorRef): Receive = {
|
||||||
|
case Status.Failure(ex) =>
|
||||||
|
log.error(s"Failure during $WriteFile operation:", ex)
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case RequestTimeout =>
|
||||||
|
log.error(s"Request $id timed out")
|
||||||
|
replyTo ! ResponseError(Some(id), ServiceError)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.WriteFileResult(Left(failure)) =>
|
||||||
|
replyTo ! ResponseError(
|
||||||
|
Some(id),
|
||||||
|
FileSystemFailureMapper.mapFailure(failure)
|
||||||
|
)
|
||||||
|
context.stop(self)
|
||||||
|
|
||||||
|
case FileManagerProtocol.WriteFileResult(Right(())) =>
|
||||||
|
replyTo ! ResponseResult(WriteFile, id, Unused)
|
||||||
|
context.stop(self)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
object WriteFileHandler {
|
||||||
|
|
||||||
|
def props(timeout: FiniteDuration, fileManager: ActorRef): Props =
|
||||||
|
Props(new WriteFileHandler(timeout, fileManager))
|
||||||
|
|
||||||
|
}
|
@ -1,7 +1,6 @@
|
|||||||
package org.enso.languageserver.runtime
|
package org.enso.languageserver.runtime
|
||||||
|
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util.UUID
|
|
||||||
|
|
||||||
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Stash}
|
import akka.actor.{Actor, ActorLogging, ActorRef, Props, Stash}
|
||||||
import org.enso.languageserver.runtime.RuntimeConnector.Destroy
|
import org.enso.languageserver.runtime.RuntimeConnector.Destroy
|
||||||
|
@ -2,12 +2,14 @@ package org.enso.languageserver.filemanager
|
|||||||
|
|
||||||
import java.nio.file.{Files, Path, Paths}
|
import java.nio.file.{Files, Path, Paths}
|
||||||
|
|
||||||
import cats.effect.IO
|
import org.enso.languageserver.effect.ZioExec
|
||||||
import org.scalatest.flatspec.AnyFlatSpec
|
import org.scalatest.flatspec.AnyFlatSpec
|
||||||
import org.scalatest.matchers.should.Matchers
|
import org.scalatest.matchers.should.Matchers
|
||||||
|
|
||||||
import scala.io.Source
|
import scala.io.Source
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
import scala.concurrent.Await
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
class FileSystemSpec extends AnyFlatSpec with Matchers {
|
class FileSystemSpec extends AnyFlatSpec with Matchers {
|
||||||
|
|
||||||
@ -538,8 +540,12 @@ class FileSystemSpec extends AnyFlatSpec with Matchers {
|
|||||||
val testDir = testDirPath.toFile
|
val testDir = testDirPath.toFile
|
||||||
testDir.deleteOnExit()
|
testDir.deleteOnExit()
|
||||||
|
|
||||||
val objectUnderTest = new FileSystem[IO]
|
val objectUnderTest = new FileSystem
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
implicit final class UnsafeRunZio[E, A](io: zio.ZIO[zio.ZEnv, E, A]) {
|
||||||
|
def unsafeRunSync(): Either[E, A] =
|
||||||
|
Await.result(ZioExec(zio.Runtime.default).exec(io), 3.seconds)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -4,44 +4,53 @@ import java.nio.file.Files
|
|||||||
import java.util.UUID
|
import java.util.UUID
|
||||||
|
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import cats.effect.IO
|
|
||||||
import org.enso.jsonrpc.{ClientControllerFactory, Protocol}
|
import org.enso.jsonrpc.{ClientControllerFactory, Protocol}
|
||||||
import org.enso.jsonrpc.test.JsonRpcServerTestKit
|
import org.enso.jsonrpc.test.JsonRpcServerTestKit
|
||||||
import org.enso.languageserver.{LanguageProtocol, LanguageServer}
|
import org.enso.languageserver.{LanguageProtocol, LanguageServer}
|
||||||
|
import org.enso.languageserver.effect.ZioExec
|
||||||
import org.enso.languageserver.capability.CapabilityRouter
|
import org.enso.languageserver.capability.CapabilityRouter
|
||||||
import org.enso.languageserver.data.{Config, Sha3_224VersionCalculator}
|
import org.enso.languageserver.data.{
|
||||||
import org.enso.languageserver.filemanager.FileSystem
|
Config,
|
||||||
|
FileManagerConfig,
|
||||||
|
Sha3_224VersionCalculator
|
||||||
|
}
|
||||||
|
import org.enso.languageserver.filemanager.{FileManager, FileSystem}
|
||||||
import org.enso.languageserver.protocol.{JsonRpc, ServerClientControllerFactory}
|
import org.enso.languageserver.protocol.{JsonRpc, ServerClientControllerFactory}
|
||||||
import org.enso.languageserver.text.BufferRegistry
|
import org.enso.languageserver.text.BufferRegistry
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
class BaseServerTest extends JsonRpcServerTestKit {
|
class BaseServerTest extends JsonRpcServerTestKit {
|
||||||
|
|
||||||
val testContentRoot = Files.createTempDirectory(null)
|
val testContentRoot = Files.createTempDirectory(null)
|
||||||
val testContentRootId = UUID.randomUUID()
|
val testContentRootId = UUID.randomUUID()
|
||||||
val config = Config(Map(testContentRootId -> testContentRoot.toFile))
|
val config = Config(
|
||||||
|
Map(testContentRootId -> testContentRoot.toFile),
|
||||||
|
FileManagerConfig(timeout = 3.seconds)
|
||||||
|
)
|
||||||
|
|
||||||
testContentRoot.toFile.deleteOnExit()
|
testContentRoot.toFile.deleteOnExit()
|
||||||
|
|
||||||
override def protocol: Protocol = JsonRpc.protocol
|
override def protocol: Protocol = JsonRpc.protocol
|
||||||
|
|
||||||
override def clientControllerFactory: ClientControllerFactory = {
|
override def clientControllerFactory: ClientControllerFactory = {
|
||||||
val languageServer =
|
val zioExec = ZioExec(zio.Runtime.default)
|
||||||
system.actorOf(
|
val languageServer = system.actorOf(Props(new LanguageServer(config)))
|
||||||
Props(new LanguageServer(config, new FileSystem[IO]))
|
|
||||||
)
|
|
||||||
languageServer ! LanguageProtocol.Initialize
|
languageServer ! LanguageProtocol.Initialize
|
||||||
|
val fileManager =
|
||||||
|
system.actorOf(FileManager.props(config, new FileSystem, zioExec))
|
||||||
val bufferRegistry =
|
val bufferRegistry =
|
||||||
system.actorOf(
|
system.actorOf(
|
||||||
BufferRegistry.props(languageServer)(Sha3_224VersionCalculator)
|
BufferRegistry.props(fileManager)(Sha3_224VersionCalculator)
|
||||||
)
|
)
|
||||||
|
|
||||||
lazy val capabilityRouter =
|
lazy val capabilityRouter =
|
||||||
system.actorOf(CapabilityRouter.props(bufferRegistry))
|
system.actorOf(CapabilityRouter.props(bufferRegistry))
|
||||||
|
|
||||||
new ServerClientControllerFactory(
|
new ServerClientControllerFactory(
|
||||||
languageServer,
|
languageServer,
|
||||||
bufferRegistry,
|
bufferRegistry,
|
||||||
capabilityRouter
|
capabilityRouter,
|
||||||
|
fileManager
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user