Sync file system and language server after restore (#4020)

VCS restore operation was correctly restoring the state of projects to the requested commit. Unfortunately, after the operation file system was becoming out-of-sync with language server's buffers (and IDE's content versions).

A few important changes are introduced here that complicate the interaction between components:
1) `vcs restore` returns an actual diff between the current state and the
requested commit
2) the response is forwarded to buffer registry first rather than to the client
3) the diff is used to identify appropriate collaborative editors and
notify them about the need to reload buffers from file system
4) all clients of affected open buffers are notified of the change via
`text/didChange` notification. If a file was removed and there were open buffers for it, clients will be notified via `file/event` and editor will be stopped
5) only then the client is notified about a successful restore operation

This PR addresses one of the two problems reported in https://www.pivotaltracker.com/story/show/184097084.

# Important Notes
We need to make sure that IDE correctly responds to `text/didChange` notifications.
This commit is contained in:
Hubert Plociniczak 2023-01-05 15:00:00 +01:00 committed by GitHub
parent 8c661fdb74
commit 3980c48d61
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 1048 additions and 50 deletions

View File

@ -496,6 +496,7 @@
- [Simplify exception handling for polyglot exceptions][3981]
- [Simplify compilation of nested patterns][4005]
- [IGV can jump to JMH sources & more][4008]
- [Sync language server with file system after VCS restore][4020]
[3227]: https://github.com/enso-org/enso/pull/3227
[3248]: https://github.com/enso-org/enso/pull/3248
@ -575,6 +576,7 @@
[3981]: https://github.com/enso-org/enso/pull/3981
[4005]: https://github.com/enso-org/enso/pull/4005
[4008]: https://github.com/enso-org/enso/pull/4008
[4020]: https://github.com/enso-org/enso/pull/4020
# Enso 2.0.0-alpha.18 (2021-10-12)

View File

@ -2804,6 +2804,17 @@ checkpoint recorded with `vcs/save`. If no save exists with a provided
restore the project to the last saved state, will all current modifications
forgotten.
If the contents of any open buffer has changed as a result of this operation,
all subscribed clients will be notified about the new version of the file via
`text/didChange` push notification.
A file might have been removed during the operation while there were still open
buffers for that file. Any such clients will be modified of a file removal via
the `file/event` notification.
The result of the call returns a list of files that have been modified during
the operation.
#### Parameters
```typescript
@ -2826,7 +2837,9 @@ forgotten.
#### Result
```typescript
null;
{
changed: [Path];
}
```
### `vcs/list`
@ -3485,7 +3498,7 @@ on the stack. In general, all consequent stack items should be `LocalCall`s.
}
```
Returns successful reponse.
Returns successful response.
```json
{

View File

@ -303,6 +303,9 @@ class JsonConnectionController(
case TextProtocol.FileAutoSaved(path) =>
webActor ! Notification(FileAutoSaved, FileAutoSaved.Params(path))
case TextProtocol.FileEvent(path, event) =>
webActor ! Notification(EventFile, EventFile.Params(path, event))
case PathWatcherProtocol.FileEventResult(event) =>
webActor ! Notification(
EventFile,

View File

@ -51,8 +51,8 @@ class RestoreVcsHandler(
replyTo ! ResponseError(Some(id), Errors.RequestTimeout)
context.stop(self)
case VcsProtocol.RestoreRepoResponse(Right(_)) =>
replyTo ! ResponseResult(RestoreVcs, id, Unused)
case VcsProtocol.RestoreRepoResponse(Right(paths)) =>
replyTo ! ResponseResult(RestoreVcs, id, RestoreVcs.Result(paths))
cancellable.cancel()
context.stop(self)

View File

@ -1,9 +1,10 @@
package org.enso.languageserver.text
import java.io.File
import org.enso.text.{ContentBasedVersioning, ContentVersion}
import org.enso.text.buffer.Rope
import org.enso.text.editing.model.Position
import org.enso.text.editing.model.Range
/** A buffer state representation.
*
@ -17,7 +18,18 @@ case class Buffer(
contents: Rope,
inMemory: Boolean,
version: ContentVersion
)
) {
/** Returns a range covering the whole buffer.
*/
lazy val fullRange: Range = {
val lines = contents.lines.length
Range(
Position(0, 0),
Position(lines - 1, contents.lines.drop(lines - 1).characters.length)
)
}
}
object Buffer {

View File

@ -13,8 +13,18 @@ import org.enso.languageserver.data.{CanEdit, CapabilityRegistration, ClientId}
import org.enso.languageserver.event.InitializedEvent
import org.enso.languageserver.filemanager.Path
import org.enso.languageserver.monitoring.MonitoringProtocol.{Ping, Pong}
import org.enso.languageserver.text.BufferRegistry.SaveTimeout
import org.enso.languageserver.text.CollaborativeBuffer.ForceSave
import org.enso.languageserver.session.JsonSession
import org.enso.languageserver.text.BufferRegistry.{
ReloadBufferTimeout,
SaveTimeout,
VcsTimeout
}
import org.enso.languageserver.text.CollaborativeBuffer.{
ForceSave,
ReloadBuffer,
ReloadBufferFailed,
ReloadedBuffer
}
import org.enso.languageserver.util.UnhandledLogging
import org.enso.languageserver.text.TextProtocol.{
ApplyEdit,
@ -27,9 +37,11 @@ import org.enso.languageserver.text.TextProtocol.{
SaveFailed,
SaveFile
}
import org.enso.languageserver.vcsmanager.GenericVcsFailure
import org.enso.languageserver.vcsmanager.VcsProtocol.{
InitRepo,
RestoreRepo,
RestoreRepoResponse,
SaveRepo
}
import org.enso.text.ContentBasedVersioning
@ -192,20 +204,25 @@ class BufferRegistry(
}
case msg @ InitRepo(clientId, path) =>
waitOnVCSActionToComplete((msg, sender()), clientId, registry, path)
forwardMessageToVCS((msg, sender()), clientId, path, registry)
case msg @ SaveRepo(clientId, path, _) =>
waitOnVCSActionToComplete((msg, sender()), clientId, registry, path)
forwardMessageToVCS((msg, sender()), clientId, path, registry)
case msg @ RestoreRepo(clientId, path, _) =>
waitOnVCSActionToComplete((msg, sender()), clientId, registry, path)
forwardMessageToVCSAndReloadBuffers(
(msg, sender()),
clientId,
path,
registry
)
}
private def waitOnVCSActionToComplete(
private def forwardMessageToVCS(
msgWithSender: (Any, ActorRef),
clientId: ClientId,
registry: Map[Path, ActorRef],
root: Path
root: Path,
registry: Map[Path, ActorRef]
): Unit = {
val openBuffers = registry.filter(_._1.startsWith(root))
val timeouts = openBuffers.map { case (_, actorRef) =>
@ -232,6 +249,7 @@ class BufferRegistry(
timeouts: Map[ActorRef, Cancellable]
): Receive = {
case SaveTimeout(from) =>
// TODO: log failure
val timeouts1 = timeouts.removed(from)
if (timeouts1.isEmpty) {
vcsManager.tell(msg._1, msg._2)
@ -243,6 +261,7 @@ class BufferRegistry(
)
}
case SaveFailed | FileSaved =>
// TODO: log failure
timeouts.get(sender()).foreach(_.cancel())
val timeouts1 = timeouts.removed(sender())
if (timeouts1.isEmpty) {
@ -258,12 +277,203 @@ class BufferRegistry(
stash()
}
private def forwardMessageToVCSAndReloadBuffers(
msgWithSender: (Any, ActorRef),
clientId: ClientId,
root: Path,
registry: Map[Path, ActorRef]
): Unit = {
val openBuffers = registry.filter(_._1.startsWith(root))
val timeouts = openBuffers.map { case (_, actorRef) =>
actorRef ! ForceSave(clientId)
(
actorRef,
context.system.scheduler
.scheduleOnce(timingsConfig.requestTimeout, self, SaveTimeout)
)
}
if (timeouts.isEmpty) {
vcsManager.tell(msgWithSender._1, self)
val timeout = context.system.scheduler
.scheduleOnce(timingsConfig.requestTimeout, self, VcsTimeout)
context.become(
waitOnVcsRestoreResponse(clientId, msgWithSender._2, timeout, registry)
)
} else {
context.become(
waitOnSaveConfirmationForwardToVCSAndReload(
clientId,
msgWithSender,
registry,
timeouts
)
)
}
}
private def waitOnSaveConfirmationForwardToVCSAndReload(
clientId: ClientId,
msg: (Any, ActorRef),
registry: Map[Path, ActorRef],
timeouts: Map[ActorRef, Cancellable]
): Receive = {
case SaveTimeout(from) =>
val timeouts1 = timeouts.removed(from)
if (timeouts1.isEmpty) {
vcsManager ! msg._1
val vcsTimeout = context.system.scheduler
.scheduleOnce(timingsConfig.requestTimeout, self, SaveTimeout)
unstashAll()
context.become(
waitOnVcsRestoreResponse(clientId, msg._2, vcsTimeout, registry)
)
} else {
context.become(
waitOnSaveConfirmationForwardToVCSAndReload(
clientId,
msg,
registry,
timeouts1
)
)
}
case SaveFailed | FileSaved =>
timeouts.get(sender()).foreach(_.cancel())
val timeouts1 = timeouts.removed(sender())
if (timeouts1.isEmpty) {
vcsManager ! msg._1
val vcsTimeout = context.system.scheduler
.scheduleOnce(timingsConfig.requestTimeout, self, VcsTimeout)
unstashAll()
context.become(
waitOnVcsRestoreResponse(clientId, msg._2, vcsTimeout, registry)
)
} else {
context.become(
waitOnSaveConfirmationForwardToVCSAndReload(
clientId,
msg,
registry,
timeouts1
)
)
}
case _ =>
stash()
}
private def waitOnVcsRestoreResponse(
clientId: ClientId,
sender: ActorRef,
timeout: Cancellable,
registry: Map[Path, ActorRef]
): Receive = {
case response @ RestoreRepoResponse(Right(_)) =>
if (timeout != null) timeout.cancel()
reloadBuffers(clientId, sender, response, registry)
case response @ RestoreRepoResponse(Left(_)) =>
if (timeout != null) timeout.cancel()
sender ! response
unstashAll()
context.become(running(registry))
case VcsTimeout =>
sender ! RestoreRepoResponse(Left(GenericVcsFailure("operation timeout")))
unstashAll()
context.become(running(registry))
case _ =>
stash()
}
private def reloadBuffers(
clientId: ClientId,
from: ActorRef,
response: RestoreRepoResponse,
registry: Map[Path, ActorRef]
): Unit = {
val filesDiff = response.result.getOrElse(Nil)
val timeouts = registry.filter(r => filesDiff.contains(r._1)).map {
case (path, collaborativeEditor) =>
collaborativeEditor ! ReloadBuffer(JsonSession(clientId, from), path)
(
path,
context.system.scheduler
.scheduleOnce(
timingsConfig.requestTimeout,
self,
ReloadBufferTimeout(path)
)
)
}
if (timeouts.isEmpty) {
from ! response
context.become(running(registry))
} else {
context.become(
waitingOnBuffersToReload(from, timeouts, registry, response)
)
}
}
private def waitingOnBuffersToReload(
from: ActorRef,
timeouts: Map[Path, Cancellable],
registry: Map[Path, ActorRef],
response: RestoreRepoResponse
): Receive = {
case ReloadedBuffer(path) =>
timeouts.get(path).foreach(_.cancel())
val timeouts1 = timeouts.removed(path)
if (timeouts1.isEmpty) {
from ! response
context.become(running(registry))
} else {
context.become(
waitingOnBuffersToReload(from, timeouts1, registry, response)
)
}
case ReloadBufferFailed(path, _) =>
timeouts.get(path).foreach(_.cancel())
val timeouts1 = timeouts.removed(path)
if (timeouts1.isEmpty) {
// TODO: log failure
from ! response
context.become(running(registry))
} else {
context.become(
waitingOnBuffersToReload(from, timeouts1, registry, response)
)
}
case ReloadBufferTimeout(path) =>
val timeouts1 = timeouts.removed(path)
if (timeouts1.isEmpty) {
// TODO: log failure
from ! response
context.become(running(registry))
} else {
context.become(
waitingOnBuffersToReload(from, timeouts1, registry, response)
)
}
}
}
object BufferRegistry {
case class SaveTimeout(ref: ActorRef)
case class ReloadBufferTimeout(path: Path)
case object VcsTimeout
/** Creates a configuration object used to create a [[BufferRegistry]]
*
* @param fileManager a file manager actor

View File

@ -17,7 +17,9 @@ import org.enso.languageserver.filemanager.FileManagerProtocol.{
WriteFileResult
}
import org.enso.languageserver.filemanager.{
FileEventKind,
FileManagerProtocol,
FileNotFound,
OperationTimeout,
Path
}
@ -25,7 +27,10 @@ import org.enso.languageserver.session.JsonSession
import org.enso.languageserver.text.CollaborativeBuffer.{
AutoSave,
ForceSave,
IOTimeout
IOTimeout,
ReloadBuffer,
ReloadBufferFailed,
ReloadedBuffer
}
import org.enso.languageserver.text.TextProtocol._
import org.enso.languageserver.util.UnhandledLogging
@ -185,6 +190,7 @@ class CollaborativeBuffer(
isAutoSave = false,
onClose = None
)
case AutoSave(clientId, clientVersion) =>
saveFile(
buffer,
@ -196,6 +202,7 @@ class CollaborativeBuffer(
isAutoSave = true,
onClose = None
)
case ForceSave(clientId) =>
autoSave.get(clientId) match {
case Some((contentVersion, cancellable)) =>
@ -214,6 +221,95 @@ class CollaborativeBuffer(
case None =>
sender() ! FileSaved
}
case ReloadBuffer(rpcSession, path) =>
if (buffer.inMemory) {
fileManager ! FileManagerProtocol.OpenBuffer(path)
} else {
fileManager ! FileManagerProtocol.ReadFile(path)
}
val timeoutCancellable = context.system.scheduler
.scheduleOnce(timingsConfig.requestTimeout, self, IOTimeout)
context.become(
waitingOnReloadedContent(
sender(),
rpcSession,
path,
buffer,
timeoutCancellable,
clients,
buffer.inMemory
)
)
}
private def waitingOnReloadedContent(
replyTo: ActorRef,
rpcSession: JsonSession,
path: Path,
oldBuffer: Buffer,
timeoutCancellable: Cancellable,
clients: Map[ClientId, JsonSession],
inMemoryBuffer: Boolean
): Receive = {
case ReadTextualFileResult(Right(file)) =>
val buffer = Buffer(file.path, file.content, inMemoryBuffer)
// Notify *all* clients about the new buffer
// This also ensures that the client that requested the restore operation
// also gets a notification.
val change = FileEdit(
path,
List(TextEdit(buffer.fullRange, file.content)),
oldBuffer.version.toHexString,
buffer.version.toHexString
)
clients.values.foreach { _.rpcController ! TextDidChange(List(change)) }
timeoutCancellable.cancel()
unstashAll()
replyTo ! ReloadedBuffer(path)
context.become(
collaborativeEditing(
buffer,
clients,
lockHolder = Some(rpcSession),
Map.empty
)
)
case ReadTextualFileResult(Left(FileNotFound)) =>
clients.values.foreach {
_.rpcController ! TextProtocol.FileEvent(path, FileEventKind.Removed)
}
replyTo ! ReloadedBuffer(path)
timeoutCancellable.cancel()
stop(Map.empty)
case ReadTextualFileResult(Left(err)) =>
replyTo ! ReloadBufferFailed(path, "io failure: " + err.toString)
timeoutCancellable.cancel()
context.become(
collaborativeEditing(
oldBuffer,
clients,
lockHolder = Some(rpcSession),
Map.empty
)
)
case IOTimeout =>
replyTo ! ReloadBufferFailed(path, "io timeout")
context.become(
collaborativeEditing(
oldBuffer,
clients,
lockHolder = Some(rpcSession),
Map.empty
)
)
case _ => stash()
}
private def saving(
@ -422,7 +518,7 @@ class CollaborativeBuffer(
lockHolder: Option[JsonSession],
clientId: ClientId,
change: FileEdit
): Either[ApplyEditFailure, Buffer] =
): Either[ApplyEditFailure, Buffer] = {
for {
_ <- validateAccess(lockHolder, clientId)
_ <- validateVersions(ContentVersion(change.oldVersion), buffer.version)
@ -432,6 +528,7 @@ class CollaborativeBuffer(
modifiedBuffer.version
)
} yield modifiedBuffer
}
private def validateVersions(
clientVersion: ContentVersion,
@ -675,6 +772,12 @@ object CollaborativeBuffer {
case class ForceSave(clientId: ClientId)
case class ReloadBuffer(rpcSession: JsonSession, path: Path)
case class ReloadBufferFailed(path: Path, reason: String)
case class ReloadedBuffer(path: Path)
/** Creates a configuration object used to create a [[CollaborativeBuffer]]
*
* @param bufferPath a path to a file

View File

@ -1,7 +1,11 @@
package org.enso.languageserver.text
import org.enso.languageserver.data.{CapabilityRegistration, ClientId}
import org.enso.languageserver.filemanager.{FileSystemFailure, Path}
import org.enso.languageserver.filemanager.{
FileEventKind,
FileSystemFailure,
Path
}
import org.enso.languageserver.session.JsonSession
import org.enso.polyglot.runtime.Runtime.Api.ExpressionId
import org.enso.text.editing.model.TextEdit
@ -132,6 +136,13 @@ object TextProtocol {
*/
case class FileAutoSaved(path: Path)
/** A notification sent by the Language Server, notifying a client about
* a file event after reloading the buffer to sync with file system
*
* @param path path to the file
*/
case class FileEvent(path: Path, event: FileEventKind)
/** Requests the language server to save a file on behalf of a given user.
*
* @param clientId the client closing the file.

View File

@ -1,7 +1,7 @@
package org.enso.languageserver.vcsmanager
import org.apache.commons.io.FileUtils
import org.eclipse.jgit.lib.Config
import org.eclipse.jgit.lib.{Config, Constants}
import org.eclipse.jgit.storage.file.FileBasedConfig
import org.eclipse.jgit.util.{FS, SystemReader}
@ -21,8 +21,13 @@ final class EmptyUserConfigReader extends SystemReader {
proxy.getHostname
/** @inheritdoc */
override def getenv(variable: String): String =
proxy.getenv(variable)
override def getenv(variable: String): String = {
if (Constants.GIT_CONFIG_NOSYSTEM_KEY.equals(variable)) {
"1"
} else {
proxy.getenv(variable)
}
}
/** @inheritdoc */
override def getProperty(key: String): String =

View File

@ -15,6 +15,8 @@ import org.eclipse.jgit.errors.{
import org.eclipse.jgit.lib.{ObjectId, Repository}
import org.eclipse.jgit.storage.file.FileRepositoryBuilder
import org.eclipse.jgit.revwalk.{RevCommit, RevWalk}
import org.eclipse.jgit.treewalk.filter.PathFilter
import org.eclipse.jgit.treewalk.{CanonicalTreeParser, FileTreeIterator}
import org.eclipse.jgit.util.SystemReader
import org.enso.languageserver.vcsmanager.Git.{
AuthorEmail,
@ -27,7 +29,6 @@ import scala.jdk.CollectionConverters._
import zio.blocking.effectBlocking
import java.time.Instant
import scala.jdk.CollectionConverters.CollectionHasAsScala
private class Git(ensoDataDirectory: Option[Path]) extends VcsApi[BlockingIO] {
@ -162,7 +163,7 @@ private class Git(ensoDataDirectory: Option[Path]) extends VcsApi[BlockingIO] {
override def restore(
root: Path,
commitId: Option[String]
): BlockingIO[VcsFailure, Unit] = {
): BlockingIO[VcsFailure, List[Path]] = {
effectBlocking {
val repo = repository(root)
@ -176,19 +177,49 @@ private class Git(ensoDataDirectory: Option[Path]) extends VcsApi[BlockingIO] {
val foundRev = findRevision(repo, name).getOrElse(
throw new RefNotFoundException(name)
)
val diff = inferDiff(jgit, foundRev, repo)
// Reset first to avoid checkout conflicts
resetCmd.call()
jgit
.checkout()
.setName(foundRev.getName)
.call()
diff
case None =>
val latest = jgit.log.setMaxCount(1).call().iterator().next()
val diff = inferDiff(jgit, latest, repo)
resetCmd.call()
diff
}
()
}.mapError(errorHandling)
}
private def inferDiff(
jgit: JGit,
targetRevision: RevCommit,
repo: Repository
): List[Path] = {
val oldTree = new FileTreeIterator(repo)
val newTree = {
val reader = repo.newObjectReader()
val treeId = targetRevision.getTree.getId
try new CanonicalTreeParser(null, reader, treeId)
finally if (reader != null) reader.close()
}
val diffResult = jgit
.diff()
.setOldTree(oldTree)
.setNewTree(newTree)
.setPathFilter(
PathFilter.create(ensureUnixPathSeparator(gitDir.toString)).negate()
)
.call()
diffResult.asScala.map { diff =>
Path.of(diff.getOldPath)
}.toList
}
private def findRevision(
repo: Repository,
sha: String

View File

@ -43,7 +43,7 @@ abstract class VcsApi[F[_, _]] {
* @param commitId optional commit to which the project should be reverted to
* @return any failures during the commit
*/
def restore(root: Path, commitId: Option[String]): F[VcsFailure, Unit]
def restore(root: Path, commitId: Option[String]): F[VcsFailure, List[Path]]
/** Report the current status of the project, reporting all modified, new or deleted projects.
*

View File

@ -71,9 +71,9 @@ class VcsManager(
case VcsProtocol.RestoreRepo(_, repoRoot, optRevName) =>
val result =
for {
root <- resolvePath(repoRoot)
_ <- vcs.restore(root.toPath, optRevName)
} yield ()
root <- resolvePath(repoRoot)
paths <- vcs.restore(root.toPath, optRevName)
} yield paths.map(p => Path.apply(repoRoot.rootId, p))
exec
.execTimed(config.timeout, result)
.map(VcsProtocol.RestoreRepoResponse)

View File

@ -50,12 +50,13 @@ object VcsManagerApi {
case object RestoreVcs extends Method("vcs/restore") {
case class Params(root: Path, commitId: Option[String])
case class Result(changed: List[Path])
implicit val hasParams = new HasParams[this.type] {
type Params = RestoreVcs.Params
}
implicit val hasResult = new HasResult[this.type] {
type Result = Unused.type
type Result = RestoreVcs.Result
}
}

View File

@ -25,8 +25,8 @@ object VcsProtocol {
revName: Option[String]
)
case class RestoreRepoResponse(result: Either[VcsFailure, Unit])
extends VCSResponse[Unit]
case class RestoreRepoResponse(result: Either[VcsFailure, List[Path]])
extends VCSResponse[List[Path]]
case class StatusRepo(clientId: ClientId, root: Path)

View File

@ -211,11 +211,17 @@ class GitSpec extends AnyWordSpecLike with Matchers with Effects {
"reset to last saved state" in new TestCtx with InitialRepoSetup {
val fooFile = repoPath.resolve("Foo.enso")
val barFile = repoPath.resolve("Bar.enso")
val bazFile = repoPath.resolve("Baz.enso")
createStubFile(fooFile) should equal(true)
Files.write(
fooFile,
"file contents".getBytes(StandardCharsets.UTF_8)
)
createStubFile(bazFile) should equal(true)
Files.write(
bazFile,
"baz file contents".getBytes(StandardCharsets.UTF_8)
)
val commitResult = vcs.commit(repoPath, "New files").unsafeRunSync()
commitResult.isRight shouldBe true
@ -235,17 +241,23 @@ class GitSpec extends AnyWordSpecLike with Matchers with Effects {
val restoreResult = vcs.restore(repoPath, commitId = None).unsafeRunSync()
restoreResult.isRight shouldBe true
restoreResult.getOrElse(Nil) shouldEqual List(
Path.of("Bar.enso"),
Path.of("Foo.enso")
)
val text2 = Files.readAllLines(fooFile)
text2.get(0) should equal("file contents")
barFile.toFile should exist // TODO: verify this is the expected logic
bazFile.toFile should exist
}
"reset to a named saved state" in new TestCtx with InitialRepoSetup {
"reset to a named saved state while preserving original line endings" in new TestCtx
with InitialRepoSetup {
val fooFile = repoPath.resolve("Foo.enso")
createStubFile(fooFile) should equal(true)
val text1 = "file contents"
val text1 = "file contents\r\nand more\u0000"
Files.write(
fooFile,
text1.getBytes(StandardCharsets.UTF_8)
@ -254,7 +266,7 @@ class GitSpec extends AnyWordSpecLike with Matchers with Effects {
commitResult.isRight shouldBe true
val commitId = commitResult.getOrElse(null).commitId
val text2 = "different contents"
val text2 = "different contents\r\nanother line"
Files.write(
fooFile,
text2.getBytes(StandardCharsets.UTF_8)
@ -263,15 +275,16 @@ class GitSpec extends AnyWordSpecLike with Matchers with Effects {
val commitResult2 = vcs.commit(repoPath, "More changes").unsafeRunSync()
commitResult2.isRight shouldBe true
val fileText1 = Files.readAllLines(fooFile)
fileText1.get(0) should equal("different contents")
val fileText1 = Files.readString(fooFile)
fileText1 should equal(text2)
val restoreResult =
vcs.restore(repoPath, Some(commitId)).unsafeRunSync()
restoreResult.isRight shouldBe true
restoreResult.getOrElse(Nil) shouldEqual List(Path.of("Foo.enso"))
val fileText2 = Files.readAllLines(fooFile)
fileText2.get(0) should equal("file contents")
val fileText2 = Files.readString(fooFile)
fileText2 should equal(text1)
}
"report problem when named save does not exist" in new TestCtx

View File

@ -1,6 +1,8 @@
package org.enso.languageserver.websocket.json
import io.circe.literal._
import io.circe.parser.parse
import org.apache.commons.io.FileUtils
import org.eclipse.jgit.api.{Git => JGit}
import org.eclipse.jgit.lib.Repository
@ -574,7 +576,14 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 3,
"result": null
"result": {
"changed": [
{
"rootId": $testContentRootId,
"segments": [ "src", "Foo.enso" ]
}
]
}
}
""")
@ -609,6 +618,10 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
}
"reset to a named save" taggedAs Retry in withCleanRoot { client =>
timingsConfig = timingsConfig.withAutoSave(0.5.seconds)
val sleepDuration: Long = 2 * 1000 // 2 seconds
val client2 = getInitialisedWsClient()
val testFileName = "Foo2.enso"
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/status",
@ -637,12 +650,14 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
val srcDir = testContentRoot.file.toPath.resolve("src")
Files.createDirectory(srcDir)
val fooPath = srcDir.resolve("Foo.enso")
val fooPath = srcDir.resolve(testFileName)
fooPath.toFile.createNewFile()
Files.write(
fooPath,
"file contents".getBytes(StandardCharsets.UTF_8)
)
// "file contents" version: 4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389
add(testContentRoot.file, srcDir)
commit(testContentRoot.file, "Add missing files")
val barPath = srcDir.resolve("Bar.enso")
@ -657,13 +672,176 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
fooPath,
"different contents".getBytes(StandardCharsets.UTF_8)
)
// "different contents" version: e2bf8493b00a13749e643e2f970b6025c227cc91340c2acb7d67e1da
add(testContentRoot.file, srcDir)
commit(testContentRoot.file, "More changes")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/status",
"method": "text/openFile",
"id": 2,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 2,
"result": {
"writeCapability": null,
"content": "different contents",
"currentVersion": "e2bf8493b00a13749e643e2f970b6025c227cc91340c2acb7d67e1da"
}
}
""")
client2.send(json"""
{ "jsonrpc": "2.0",
"method": "text/openFile",
"id": 2,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
client2.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 2,
"result": {
"writeCapability": null,
"content": "different contents",
"currentVersion": "e2bf8493b00a13749e643e2f970b6025c227cc91340c2acb7d67e1da"
}
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "capability/acquire",
"id": 3,
"params": {
"method": "text/canEdit",
"registerOptions": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 3,
"result": null
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/applyEdit",
"id": 4,
"params": {
"edit": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
},
"oldVersion": "e2bf8493b00a13749e643e2f970b6025c227cc91340c2acb7d67e1da",
"newVersion": "e4bb87ced8ddafa060f08f2a79cc2861355eb9f596e462d7df462ef4",
"edits": [
{
"range": {
"start": { "line": 0, "character": 0 },
"end": { "line": 0, "character": 9 }
},
"text": "bar"
}
]
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 4,
"result": null
}
""")
client2.expectJson(json"""
{ "jsonrpc" : "2.0",
"method" : "text/didChange",
"params" : {
"edits" : [
{
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testFileName
]
},
"edits" : [
{
"range" : {
"start" : {
"line" : 0,
"character" : 0
},
"end" : {
"line" : 0,
"character" : 9
}
},
"text" : "bar"
}
],
"oldVersion" : "e2bf8493b00a13749e643e2f970b6025c227cc91340c2acb7d67e1da",
"newVersion" : "e4bb87ced8ddafa060f08f2a79cc2861355eb9f596e462d7df462ef4"
}
]
}
}
""")
// Ensure auto-save kicks in
Thread.sleep(sleepDuration)
client.expectJson(json"""
{ "jsonrpc": "2.0",
"method":"text/autoSave",
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
client2.expectJson(json"""
{ "jsonrpc": "2.0",
"method":"text/autoSave",
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/status",
"id": 5,
"params": {
"root": {
"rootId": $testContentRootId,
@ -674,10 +852,18 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
""")
client.fuzzyExpectJson(json"""
{ "jsonrpc": "2.0",
"id": 2,
"id": 5,
"result": {
"dirty": false,
"changed": [],
"dirty": true,
"changed": [
{
"rootId" : $testContentRootId,
"segments" : [
"src",
$testFileName
]
}
],
"lastSave": {
"commitId": "*",
"message": "More changes"
@ -685,12 +871,16 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
}
}
""")
val sndToLast = commits(testContentRoot.file).tail.head
val allCommits = commits(testContentRoot.file)
val sndToLast = allCommits.tail.head
val text0 = Files.readAllLines(fooPath)
text0.get(0) should equal("bar contents")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/restore",
"id": 3,
"id": 6,
"params": {
"root": {
"rootId": $testContentRootId,
@ -700,20 +890,189 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
}
}
""")
client.expectJson(json"""
{ "jsonrpc" : "2.0",
"method" : "text/didChange",
"params" : {
"edits" : [
{
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testFileName
]
},
"edits" : [
{
"range" : {
"start" : {
"line" : 0,
"character" : 0
},
"end" : {
"line" : 0,
"character" : 13
}
},
"text" : "file contents"
}
],
"oldVersion" : "e4bb87ced8ddafa060f08f2a79cc2861355eb9f596e462d7df462ef4",
"newVersion" : "4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389"
}
]
}
}""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 3,
"result": null
"id": 6,
"result": {
"changed": [
{
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
]
}
}
""")
client2.expectJson(json"""
{ "jsonrpc" : "2.0",
"method" : "text/didChange",
"params" : {
"edits" : [
{
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testFileName
]
},
"edits" : [
{
"range" : {
"start" : {
"line" : 0,
"character" : 0
},
"end" : {
"line" : 0,
"character" : 13
}
},
"text" : "file contents"
}
],
"oldVersion" : "e4bb87ced8ddafa060f08f2a79cc2861355eb9f596e462d7df462ef4",
"newVersion" : "4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389"
}
]
}
}""")
val text1 = Files.readAllLines(fooPath)
text1.get(0) should equal("file contents")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/applyEdit",
"id": 7,
"params": {
"edit": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
},
"oldVersion": "4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389",
"newVersion": "1141745721c08c1c1c26ca32b95f103c0721f70eedaa6db765dfc43e",
"edits": [
{
"range": {
"start": { "line": 0, "character": 0 },
"end": { "line": 0, "character": 4 }
},
"text": "foo"
}
]
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 7,
"id": 7,
"result": null
}
""")
client2.expectJson(json"""
{ "jsonrpc" : "2.0",
"method" : "text/didChange",
"params" : {
"edits" : [
{
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testFileName
]
},
"edits" : [
{
"range" : {
"start" : {
"line" : 0,
"character" : 0
},
"end" : {
"line" : 0,
"character" : 4
}
},
"text" : "foo"
}
],
"oldVersion" : "4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389",
"newVersion" : "1141745721c08c1c1c26ca32b95f103c0721f70eedaa6db765dfc43e"
}
]
}
}""")
// Ensure auto-save kicks in
Thread.sleep(sleepDuration)
client.expectJson(json"""
{ "jsonrpc": "2.0",
"method":"text/autoSave",
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
client2.expectJson(json"""
{ "jsonrpc": "2.0",
"method":"text/autoSave",
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFileName ]
}
}
}
""")
val text2 = Files.readAllLines(fooPath)
text2.get(0) should equal("foo contents")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/restore",
"id": 4,
"id": 8,
"params": {
"root": {
"rootId": $testContentRootId,
@ -725,7 +1084,7 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 4,
"id": 8,
"error": {
"code": 1004,
"message": "Requested save not found"
@ -733,6 +1092,241 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
}
""")
}
"reset to a named save and notify about removed files" taggedAs Retry in withCleanRoot {
client =>
timingsConfig = timingsConfig.withAutoSave(0.5.seconds)
val client2 = getInitialisedWsClient()
val testFooFileName = "Foo.enso"
val testBarFileName = "Bar.enso"
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/status",
"id": 1,
"params": {
"root": {
"rootId": $testContentRootId,
"segments": []
}
}
}
""")
client.fuzzyExpectJson(json"""
{ "jsonrpc": "2.0",
"id": 1,
"result": {
"dirty": false,
"changed": [],
"lastSave": {
"commitId": "*",
"message": "Initial commit"
}
}
}
""")
val srcDir = testContentRoot.file.toPath.resolve("src")
Files.createDirectory(srcDir)
val fooPath = srcDir.resolve(testFooFileName)
fooPath.toFile.createNewFile()
Files.write(
fooPath,
"file contents".getBytes(StandardCharsets.UTF_8)
)
// "file contents" version: 4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389
add(testContentRoot.file, srcDir)
commit(testContentRoot.file, "Add first file")
val barPath = srcDir.resolve(testBarFileName)
barPath.toFile.createNewFile()
Files.write(
barPath,
"file contents b".getBytes(StandardCharsets.UTF_8)
)
// "file contents b" version: 4b6a8df62627ea7fbd1f4d9296d16c166b17b037c01d7298454cee99
add(testContentRoot.file, srcDir)
commit(testContentRoot.file, "Add second file")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/openFile",
"id": 2,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testFooFileName ]
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 2,
"result": {
"writeCapability": null,
"content": "file contents",
"currentVersion": "4d23065da489de360890285072c209b2b39d45d12283dbb5d1fa4389"
}
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/openFile",
"id": 3,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testBarFileName ]
}
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 3,
"result": {
"writeCapability": {
"method" : "text/canEdit",
"registerOptions" : {
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testBarFileName
]
}
}
},
"content": "file contents b",
"currentVersion": "4b6a8df62627ea7fbd1f4d9296d16c166b17b037c01d7298454cee99"
}
}
""")
client2.send(json"""
{ "jsonrpc": "2.0",
"method": "text/openFile",
"id": 4,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "src", $testBarFileName ]
}
}
}
""")
client2.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 4,
"result": {
"writeCapability": null,
"content": "file contents b",
"currentVersion": "4b6a8df62627ea7fbd1f4d9296d16c166b17b037c01d7298454cee99"
}
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/status",
"id": 5,
"params": {
"root": {
"rootId": $testContentRootId,
"segments": []
}
}
}
""")
client.fuzzyExpectJson(json"""
{ "jsonrpc": "2.0",
"id": 5,
"result": {
"dirty": false,
"changed": [],
"lastSave": {
"commitId": "*",
"message": "Add second file"
}
}
}
""")
val allCommits = commits(testContentRoot.file)
val sndToLast = allCommits.tail.head
client.send(json"""
{ "jsonrpc": "2.0",
"method": "vcs/restore",
"id": 6,
"params": {
"root": {
"rootId": $testContentRootId,
"segments": []
},
"commitId": ${sndToLast.getName}
}
}
""")
// Additional logic to deal with out-of-order messages due
// to multiple actors being involved in forwarding messages.
val msg1 = parse(client.expectMessage()).getOrElse(fail())
val msg2 = parse(client.expectMessage()).getOrElse(fail())
val isFileEvent = msg1.hcursor.get[String]("method").toOption.isEmpty
val (methodsEvent, response) = if (isFileEvent) {
(msg2, msg1)
} else {
(msg1, msg2)
}
methodsEvent shouldEqual json"""
{ "jsonrpc" : "2.0",
"method" : "file/event",
"params" : {
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testBarFileName
]
},
"kind" : "Removed"
}
}
"""
response shouldEqual json"""
{ "jsonrpc": "2.0",
"id": 6,
"result": {
"changed": [
{
"rootId": $testContentRootId,
"segments": [ "src", $testBarFileName ]
}
]
}
}
"""
client2.expectJson(json"""
{ "jsonrpc" : "2.0",
"method" : "file/event",
"params" : {
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"src",
$testBarFileName
]
},
"kind" : "Removed"
}
}
""")
}
}
"List project saves" must {
@ -894,7 +1488,7 @@ class VcsManagerTest extends BaseServerTest with RetrySpec {
jgit.log().call().asScala.toList
}
def commit(root: File, msg: String): Unit = {
def commit(root: File, msg: String): RevCommit = {
val jgit = new JGit(repository(root.toPath))
jgit.commit.setMessage(msg).setAuthor("Enso VCS", "vcs@enso.io").call()
}