TextEdit with custom IdMap (#10283)

close #10182

Changelog:
- add: IdMap parameter to the `text/applyEdit` request
- add: IdMap to the runtime module
- update: set IdMap during the interactive compilation
- update: set the IR identifiers in the `TreeToIR` parsing step
This commit is contained in:
Dmitry Bushev 2024-06-15 19:03:18 +01:00 committed by GitHub
parent dee9e079d4
commit 04a92ef765
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
30 changed files with 582 additions and 118 deletions

View File

@ -2834,10 +2834,20 @@ that some edits are applied and others are not.
interface TextApplyEditParameters {
/** The file edit. */
edit: FileEdit;
/** A flag indicating whether we should re-execute the program after applying
/**
* A flag indicating whether we should re-execute the program after applying
* the edit. Default value is `true`, indicating the program should be
* re-executed. */
* re-executed.
*/
execute?: boolean;
/**
* An identifiers map associated with this file as an array of
* index, length, uuid triples. The old id map format that was used in the
* source file is also supported.
*/
idMap?: [number, number, UUID][];
}
```

View File

@ -35,7 +35,8 @@ class ApplyEditHandler(
bufferRegistry ! TextProtocol.ApplyEdit(
Some(rpcSession.clientId),
params.edit,
params.execute.getOrElse(true)
params.execute.getOrElse(true),
params.idMap
)
val cancellable =
context.system.scheduler.scheduleOnce(timeout, self, RequestTimeout)

View File

@ -189,7 +189,7 @@ class BufferRegistry(
sender() ! CapabilityReleaseBadRequest
}
case msg @ ApplyEdit(_, FileEdit(path, _, _, _), _) =>
case msg @ ApplyEdit(_, FileEdit(path, _, _, _), _, _) =>
if (registry.contains(path)) {
registry(path).forward(msg)
} else {
@ -245,7 +245,8 @@ class BufferRegistry(
msg.oldVersion,
msg.newVersion
),
execute = true
execute = true,
None
)
}
case None =>

View File

@ -35,7 +35,7 @@ import org.enso.languageserver.util.UnhandledLogging
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.text.{ContentBasedVersioning, ContentVersion}
import org.enso.text.editing._
import org.enso.text.editing.model.TextEdit
import org.enso.text.editing.model.{IdMap, TextEdit}
import java.util.UUID
@ -259,8 +259,17 @@ class CollaborativeBuffer(
sender() ! FileNotOpened
}
case ApplyEdit(clientId, change, execute) =>
edit(buffer, clients, lockHolder, clientId, change, execute, autoSave)
case ApplyEdit(clientId, change, execute, idMap) =>
edit(
buffer,
clients,
lockHolder,
clientId,
change,
execute,
idMap,
autoSave
)
case ApplyExpressionValue(
clientId,
@ -412,7 +421,7 @@ class CollaborativeBuffer(
buffer.version.toHexString
)
runtimeConnector ! Api.Request(
Api.EditFileNotification(file.path, edits, execute = true)
Api.EditFileNotification(file.path, edits, execute = true, None)
)
clients.values.foreach { _.rpcController ! TextDidChange(List(change)) }
unstashAll()
@ -727,6 +736,7 @@ class CollaborativeBuffer(
clientId: Option[ClientId],
change: FileEdit,
execute: Boolean,
idMap: Option[IdMap],
autoSave: Map[ClientId, (ContentVersion, Cancellable)]
): Unit = {
applyEdits(buffer, lockHolder, clientId, change) match {
@ -743,7 +753,8 @@ class CollaborativeBuffer(
Api.EditFileNotification(
buffer.fileWithMetadata.file,
change.edits,
execute
execute,
idMap
)
)
}

View File

@ -4,13 +4,13 @@ import org.enso.languageserver.data.CapabilityRegistration
import org.enso.languageserver.filemanager.Path
import org.enso.jsonrpc.{Error, HasParams, HasResult, Method, Unused}
import org.enso.polyglot.runtime.Runtime.Api.ExpressionId
import org.enso.text.editing.model.TextEdit
import org.enso.text.editing.model.{IdMap, TextEdit}
/** The text editing JSON RPC API provided by the language server.
* See [[https://github.com/enso-org/enso/blob/develop/docs/language-server/README.md]]
* for message specifications.
*/
object TextApi {
object TextApi extends TextProtocol.Codecs {
type Version = String
@ -61,7 +61,11 @@ object TextApi {
}
case object ApplyEdit extends Method("text/applyEdit") {
case class Params(edit: FileEdit, execute: Option[Boolean])
case class Params(
edit: FileEdit,
execute: Option[Boolean],
idMap: Option[IdMap]
)
implicit val hasParams: HasParams.Aux[this.type, ApplyEdit.Params] =
new HasParams[this.type] {
type Params = ApplyEdit.Params

View File

@ -1,5 +1,7 @@
package org.enso.languageserver.text
import io.circe.{Decoder, Encoder, Json}
import io.circe.syntax._
import org.enso.languageserver.data.{CapabilityRegistration, ClientId}
import org.enso.languageserver.filemanager.{
FileAttributes,
@ -9,7 +11,9 @@ import org.enso.languageserver.filemanager.{
}
import org.enso.languageserver.session.JsonSession
import org.enso.polyglot.runtime.Runtime.Api.ExpressionId
import org.enso.text.editing.model.TextEdit
import org.enso.text.editing.model.{IdMap, Span, TextEdit}
import java.util.UUID
object TextProtocol {
@ -69,11 +73,13 @@ object TextProtocol {
* @param clientId the client requesting edits.
* @param edit a diff describing changes made to a file
* @param execute whether to execute the program after applying the edits
* @param idMap external identifiers
*/
case class ApplyEdit(
clientId: Option[ClientId],
edit: FileEdit,
execute: Boolean
execute: Boolean,
idMap: Option[IdMap]
)
/** Signals the result of applying a series of edits.
@ -214,4 +220,67 @@ object TextProtocol {
*/
case class ReadCollaborativeBufferResult(buffer: Option[Buffer])
trait Codecs {
private object IdMapOldCodecs {
private object CodecField {
val Index = "index"
val Size = "size"
val Value = "value"
}
implicit private val spanDecoder: Decoder[Span] =
Decoder.instance { cursor =>
for {
index <- cursor
.downField(CodecField.Index)
.downField(CodecField.Value)
.as[Int]
size <- cursor
.downField(CodecField.Size)
.downField(CodecField.Value)
.as[Int]
} yield Span(index, index + size)
}
implicit def idMapDecoder: Decoder[IdMap] =
Decoder.instance { cursor =>
for {
pairs <- Decoder[Vector[(Span, UUID)]].tryDecode(cursor)
} yield IdMap(pairs)
}
}
private object IdMapCodecs {
implicit def idMapEncoder: Encoder[IdMap] =
Encoder.instance { idMap =>
Json.fromValues(
idMap.values
.map({ case (span, uuid) =>
Json.arr((span.start, span.length, uuid).asJson)
})
)
}
implicit def idMapDecoder: Decoder[IdMap] =
Decoder.instance { cursor =>
for {
triples <- Decoder[Vector[(Int, Int, UUID)]].tryDecode(cursor)
} yield {
val pairs = triples.map({ case (start, length, uuid) =>
Span(start, start + length) -> uuid
})
IdMap(pairs)
}
}
}
implicit val encoder: Encoder[IdMap] =
IdMapCodecs.idMapEncoder
implicit val decoder: Decoder[IdMap] =
IdMapCodecs.idMapDecoder.or(IdMapOldCodecs.idMapDecoder)
}
}

View File

@ -187,7 +187,8 @@ class FileNotificationsTest
Api.EditFileNotification(
file("foo.txt"),
Seq(TextEdit(Range(Position(0, 0), Position(0, 0)), "bar")),
execute = true
execute = true,
None
)
)
)

View File

@ -2076,6 +2076,103 @@ class TextOperationsTest
}
""")
}
"apply edit with IdMap" in {
val client = getInitialisedWsClient()
client.send(json"""
{ "jsonrpc": "2.0",
"method": "file/write",
"id": 0,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "fooIdMap.txt" ]
},
"contents": "123456789"
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 0,
"result": null
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/openFile",
"id": 1,
"params": {
"path": {
"rootId": $testContentRootId,
"segments": [ "fooIdMap.txt" ]
}
}
}
""")
receiveAndReplyToOpenFile("fooIdMap.txt")
client.expectJson(json"""
{
"jsonrpc" : "2.0",
"id" : 1,
"result" : {
"writeCapability" : {
"method" : "text/canEdit",
"registerOptions" : {
"path" : {
"rootId" : $testContentRootId,
"segments" : [
"fooIdMap.txt"
]
}
}
},
"content" : "123456789",
"currentVersion" : "5795c3d628fd638c9835a4c79a55809f265068c88729a1a3fcdf8522"
}
}
""")
client.send(json"""
{ "jsonrpc": "2.0",
"method": "text/applyEdit",
"id": 2,
"params": {
"edit": {
"path": {
"rootId": $testContentRootId,
"segments": [ "fooIdMap.txt" ]
},
"oldVersion": "5795c3d628fd638c9835a4c79a55809f265068c88729a1a3fcdf8522",
"newVersion": "ebe55342f9c8b86857402797dd723fb4a2174e0b56d6ace0a6929ec3",
"edits": [
{
"range": {
"start": { "line": 0, "character": 0 },
"end": { "line": 0, "character": 0 }
},
"text": "bar"
},
{
"range": {
"start": { "line": 0, "character": 12 },
"end": { "line": 0, "character": 12 }
},
"text": "foo"
}
]
},
"idMap": [[0, 1, "00000000-0000-0000-0000-000000000001"]]
}
}
""")
client.expectJson(json"""
{ "jsonrpc": "2.0",
"id": 2,
"result": null
}
""")
}
}
"text/applyExpressionValue" must {

View File

@ -7,7 +7,7 @@ import org.enso.pkg.{ComponentGroups, QualifiedName}
import org.enso.polyglot.{ModuleExports, Suggestion}
import org.enso.polyglot.data.{Tree, TypeGraph}
import org.enso.text.editing.model
import org.enso.text.editing.model.{Range, TextEdit}
import org.enso.text.editing.model.{IdMap, Range, TextEdit}
import java.io.File
import java.util.UUID
@ -1091,7 +1091,8 @@ object Runtime {
final case class EditFileNotification(
path: File,
edits: Seq[TextEdit],
execute: Boolean
execute: Boolean,
idMap: Option[IdMap]
) extends ApiRequest
with ToLogString {
@ -1100,7 +1101,9 @@ object Runtime {
"EditFileNotification(" +
s"path=${MaskedPath(path.toPath).toLogString(shouldMask)},edits=" +
(if (shouldMask) edits.map(_ => STUB) else edits) +
",execute=" + execute + ")"
",execute=" + execute +
"idMap=" + idMap.map(_ => STUB) +
")"
}
/** A notification sent to the server about in-memory file contents being

View File

@ -14,6 +14,7 @@ import org.enso.compiler.core.CompilerStub;
import org.enso.compiler.core.ir.Diagnostic;
import org.enso.compiler.data.BindingsMap;
import org.enso.compiler.data.CompilerConfig;
import org.enso.compiler.data.IdMap;
import org.enso.editions.LibraryName;
import org.enso.pkg.Package;
import org.enso.pkg.QualifiedName;
@ -25,6 +26,7 @@ import org.enso.pkg.QualifiedName;
* {@link Compiler} & co. classes separately without any dependency on Truffle API.
*/
public interface CompilerContext extends CompilerStub {
boolean isIrCachingDisabled();
boolean isPrivateCheckDisabled();
@ -52,9 +54,9 @@ public interface CompilerContext extends CompilerStub {
/**
* Format the given diagnostic into a string. The returned string might have ANSI colors.
*
* @param module May be null if inline diagnostics is required.
* @param diagnostic
* @param isOutputRedirected True if the output is not system's out. If true, no ANSI color escape
* @param module may be null if inline diagnostics is required.
* @param diagnostic an IR node representing diagnostic information
* @param isOutputRedirected true if the output is not system's out. If true, no ANSI color escape
* characters will be inside the returned string.
* @return exception with a message to display or to throw
*/
@ -86,6 +88,8 @@ public interface CompilerContext extends CompilerStub {
CharSequence getCharacters(Module module) throws IOException;
IdMap getIdMap(Module module);
void updateModule(Module module, Consumer<Updater> callback);
boolean isSynthetic(Module module);
@ -117,6 +121,8 @@ public interface CompilerContext extends CompilerStub {
public static interface Updater {
void bindingsMap(BindingsMap map);
void idMap(IdMap idMap);
void ir(org.enso.compiler.core.ir.Module ir);
void compilationStage(CompilationStage stage);
@ -129,6 +135,7 @@ public interface CompilerContext extends CompilerStub {
}
public abstract static class Module {
public abstract CharSequence getCharacters() throws IOException;
public abstract String getPath();
@ -139,6 +146,8 @@ public interface CompilerContext extends CompilerStub {
public abstract BindingsMap getBindingsMap();
public abstract IdMap getIdMap();
public abstract List<QualifiedName> getDirectModulesRefs();
public abstract CompilationStage getCompilationStage();

View File

@ -0,0 +1,14 @@
package org.enso.compiler.data;
import java.util.Collections;
import java.util.Map;
import java.util.UUID;
import org.enso.compiler.core.ir.Location;
/** A mapping between the code {@link Location}s and their identifiers. */
public record IdMap(Map<Location, UUID> values) {
public static IdMap empty() {
return new IdMap(Collections.emptyMap());
}
}

View File

@ -36,7 +36,7 @@ import org.enso.pkg.QualifiedName
import org.enso.common.CompilationStage
import org.enso.syntax2.Tree
import java.io.{PrintStream}
import java.io.PrintStream
import java.util.concurrent.{
CompletableFuture,
ExecutorService,
@ -289,8 +289,8 @@ class Compiler(
parseModule(module, irCachingEnabled && !context.isInteractive(module))
importedModules
.filter(isLoadedFromSource)
.map(m => {
if (m.getBindingsMap() == null) {
.foreach(m => {
if (m.getBindingsMap == null) {
parseModule(m, irCachingEnabled && !context.isInteractive(module))
}
})
@ -585,9 +585,10 @@ class Compiler(
isGeneratingDocs = isGenDocs
)
val src = context.getCharacters(module)
val tree = ensoCompiler.parse(src)
val expr = ensoCompiler.generateIR(tree)
val src = context.getCharacters(module)
val idMap = context.getIdMap(module)
val tree = ensoCompiler.parse(src)
val expr = ensoCompiler.generateModuleIr(tree, idMap.values)
val exprWithModuleExports =
if (context.isSynthetic(module))

View File

@ -18,7 +18,7 @@ import org.enso.compiler.suggestions.SimpleUpdate
import org.enso.compiler.core.ir.module.scope.definition
import org.enso.compiler.pass.analyse.DataflowAnalysis
import org.enso.interpreter.instrument.execution.model.PendingEdit
import org.enso.text.editing.model.TextEdit
import org.enso.text.editing.model.{IdMap, TextEdit}
import org.enso.text.editing.{IndexedSource, TextEditor}
import scala.collection.mutable
@ -31,13 +31,15 @@ import scala.util.Using
* @param ir the IR node of the module
* @param simpleUpdate description of a simple editing change (usually in a literal)
* @param invalidated the list of invalidated expressions
* @param idMap the external identifiers
* @tparam A the source type
*/
case class Changeset[A](
source: A,
ir: IR,
simpleUpdate: Option[SimpleUpdate],
invalidated: Set[UUID @ExternalID]
invalidated: Set[UUID @ExternalID],
idMap: Option[IdMap]
)
/** Compute invalidated expressions.
@ -116,7 +118,9 @@ final class ChangesetBuilder[A: TextEditor: IndexedSource](
}
}
Changeset(source, ir, simpleUpdateOption, compute(edits.map(_.edit)))
val idMap = edits.flatMap(_.idMap).lastOption
Changeset(source, ir, simpleUpdateOption, compute(edits.map(_.edit)), idMap)
}
/** Traverses the IR and returns a list of all IR nodes affected by the edit

View File

@ -38,7 +38,7 @@ class EditFileCmd(request: Api.EditFileNotification)
)
val edits =
request.edits.map(edit =>
PendingEdit.ApplyEdit(edit, request.execute)
PendingEdit.ApplyEdit(edit, request.execute, request.idMap)
)
ctx.state.pendingEdits.enqueue(request.path, edits)
if (request.execute) {

View File

@ -1,11 +1,12 @@
package org.enso.interpreter.instrument.execution.model
import org.enso.polyglot.runtime.Runtime.Api.ExpressionId
import org.enso.text.editing.model.TextEdit
import org.enso.text.editing.model.{IdMap, TextEdit}
sealed trait PendingEdit {
def edit: TextEdit
def execute: Boolean
def idMap: Option[IdMap]
}
object PendingEdit {
@ -13,12 +14,19 @@ object PendingEdit {
*
* @param edit a diff describing changes made to a file
* @param execute whether to execute the program after applying the edit
* @param idMap external identifiers
*/
case class ApplyEdit(edit: TextEdit, execute: Boolean) extends PendingEdit
case class ApplyEdit(edit: TextEdit, execute: Boolean, idMap: Option[IdMap])
extends PendingEdit
case class SetExpressionValue(edit: TextEdit, id: ExpressionId, value: String)
extends PendingEdit {
/** @inheritdoc */
override val execute: Boolean = true
/** @inheritdoc */
override val idMap: Option[IdMap] = None
}
}

View File

@ -2,17 +2,12 @@ package org.enso.interpreter.instrument.job
import com.oracle.truffle.api.TruffleLogger
import org.enso.common.CompilationStage
import org.enso.compiler.CompilerResult
import org.enso.compiler.{data, CompilerResult}
import org.enso.compiler.context._
import org.enso.compiler.core.Implicits.AsMetadata
import org.enso.compiler.core.{ExternalID, IR}
import org.enso.compiler.core.ir.{
expression,
Diagnostic,
IdentifiedLocation,
Warning
}
import org.enso.compiler.core.ir.expression.Error
import org.enso.compiler.core.ir
import org.enso.compiler.core.ir.{expression, Location}
import org.enso.compiler.data.BindingsMap
import org.enso.compiler.pass.analyse.{
CachePreferenceAnalysis,
@ -34,8 +29,10 @@ import org.enso.pkg.QualifiedName
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.polyglot.runtime.Runtime.Api.StackItem
import org.enso.text.buffer.Rope
import org.enso.text.editing.model.IdMap
import java.io.File
import java.util
import java.util.UUID
import java.util.logging.Level
@ -55,7 +52,7 @@ final class EnsureCompiledJob(
false
) {
import EnsureCompiledJob.CompilationStatus
import EnsureCompiledJob._
/** @inheritdoc */
override def run(implicit ctx: RuntimeContext): CompilationStatus = {
@ -114,7 +111,7 @@ final class EnsureCompiledJob(
case _ =>
}
applyEdits(new File(module.getPath)).map { changeset =>
compile(module)
compile(module, changeset.idMap)
.map { _ =>
// Side-effect: ensures that module's source is correctly initialized.
module.getSource()
@ -181,8 +178,7 @@ final class EnsureCompiledJob(
(
modules
.addAll(
compilerResult.compiledModules
.map(Module.fromCompilerModule(_))
compilerResult.compiledModules.map(Module.fromCompilerModule)
)
.addOne(module),
statuses += status
@ -230,9 +226,9 @@ final class EnsureCompiledJob(
)
.diagnostics
val diagnostics = pass.collect {
case warn: Warning =>
case warn: ir.Warning =>
createDiagnostic(Api.DiagnosticType.Warning, module, warn)
case error: Error =>
case error: expression.Error =>
createDiagnostic(Api.DiagnosticType.Error, module, error)
}
sendDiagnosticUpdates(diagnostics)
@ -249,11 +245,11 @@ final class EnsureCompiledJob(
private def createDiagnostic(
kind: Api.DiagnosticType,
module: Module,
diagnostic: Diagnostic
diagnostic: ir.Diagnostic
): Api.ExecutionResult.Diagnostic = {
val source = module.getSource
def fileLocationFromSection(loc: IdentifiedLocation) = {
def fileLocationFromSection(loc: ir.IdentifiedLocation) = {
val section =
source.createSection(loc.location().start(), loc.location().length());
val locStr = "" + section.getStartLine() + ":" + section
@ -280,19 +276,33 @@ final class EnsureCompiledJob(
/** Compile the module.
*
* @param module the module to compile.
* @param idMapOpt the external identifiers
* @param ctx the runtime context
* @return the compiled module
*/
private def compile(
module: Module
module: Module,
idMapOpt: Option[IdMap] = None
)(implicit ctx: RuntimeContext): Either[Throwable, CompilerResult] =
try {
val compilationStage = module.getCompilationStage
if (!compilationStage.isAtLeast(CompilationStage.AFTER_CODEGEN)) {
if (
!compilationStage.isAtLeast(CompilationStage.AFTER_CODEGEN)
|| idMapOpt.isDefined
) {
ctx.executionService.getLogger
.log(Level.FINEST, s"Compiling ${module.getName}.")
val result = ctx.executionService.getContext.getCompiler
.run(module.asCompilerModule())
val compiler = ctx.executionService.getContext.getCompiler
idMapOpt.foreach { idMap =>
compiler.context.updateModule(
module.asCompilerModule(),
_.idMap(toCompilerIdMap(idMap))
)
}
val result = compiler.run(module.asCompilerModule())
Right(
result.copy(compiledModules =
result.compiledModules.filter(_.getName != module.getName)
@ -662,4 +672,18 @@ object EnsureCompiledJob {
None
}
/** Convert the identifiers map to its compiler equivalent.
*
* @param idMap the identifiers map
* @return the compiler representation of identifiers map
*/
private def toCompilerIdMap(idMap: IdMap): data.IdMap = {
val values =
idMap.values.foldLeft(new util.HashMap[Location, UUID]()) {
case (map, (span, id)) =>
map.put(new Location(span.start, span.end), id)
map
}
new data.IdMap(values)
}
}

View File

@ -140,7 +140,9 @@ final class RefactoringRenameJob(
this.getClass,
() => {
val pendingEdits =
fileEdit.edits.map(PendingEdit.ApplyEdit(_, execute = true))
fileEdit.edits.map(
PendingEdit.ApplyEdit(_, execute = true, idMap = None)
)
ctx.state.pendingEdits.enqueue(fileEdit.path, pendingEdits)
}
)

View File

@ -326,7 +326,8 @@ public class IncrementalUpdatesTest {
new model.Position(3, 8),
new model.Position(3, 8 + originalText.length())),
newText)),
true)));
true,
None())));
return context.receiveNIgnorePendingExpressionUpdates(1, 60, emptySet());
}

View File

@ -873,7 +873,8 @@ class RuntimeErrorsTest
"1234567890123456789"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -922,7 +923,8 @@ class RuntimeErrorsTest
"1000000000000.div 0"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -971,7 +973,8 @@ class RuntimeErrorsTest
"1000000000000.div 2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1096,7 +1099,8 @@ class RuntimeErrorsTest
"MyError2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1220,7 +1224,8 @@ class RuntimeErrorsTest
"MyError2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1443,7 +1448,8 @@ class RuntimeErrorsTest
"1234567890123456789"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1574,7 +1580,8 @@ class RuntimeErrorsTest
"101"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1714,7 +1721,8 @@ class RuntimeErrorsTest
"MyError2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1857,7 +1865,8 @@ class RuntimeErrorsTest
"10002 - 10000"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -2054,7 +2063,8 @@ class RuntimeErrorsTest
"10002 - 10000"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -2188,7 +2198,8 @@ class RuntimeErrorsTest
s"import Standard.Base.IO$newline$newline"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -2301,7 +2312,8 @@ class RuntimeErrorsTest
s"import Standard.Base.IO$newline$newline"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -2401,7 +2413,8 @@ class RuntimeErrorsTest
s"y = x - 1${newline} y"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -2490,7 +2503,8 @@ class RuntimeErrorsTest
"2"
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -396,7 +396,8 @@ class RuntimeRefactoringTest
"42"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -718,7 +719,8 @@ class RuntimeRefactoringTest
"42"
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -3072,7 +3072,8 @@ class RuntimeServerTest
"5"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3189,7 +3190,8 @@ class RuntimeServerTest
"5"
)
),
execute = false
execute = false,
idMap = None
)
)
)
@ -3206,7 +3208,8 @@ class RuntimeServerTest
"6"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3373,7 +3376,8 @@ class RuntimeServerTest
"\"Hi\""
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3485,7 +3489,8 @@ class RuntimeServerTest
"1234.x 4"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3520,7 +3525,8 @@ class RuntimeServerTest
"1000.x 5"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3557,7 +3563,8 @@ class RuntimeServerTest
"Main.pie"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3594,7 +3601,8 @@ class RuntimeServerTest
"Main.uwu"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3631,7 +3639,8 @@ class RuntimeServerTest
"Main.hie"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3666,7 +3675,8 @@ class RuntimeServerTest
"\"Hello!\""
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4126,7 +4136,8 @@ class RuntimeServerTest
"modified"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4221,7 +4232,8 @@ class RuntimeServerTest
"modified"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4288,7 +4300,8 @@ class RuntimeServerTest
"main = 42"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4458,7 +4471,8 @@ class RuntimeServerTest
s"Number.lucky = 42$newline$newline"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4550,7 +4564,8 @@ class RuntimeServerTest
code2
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -6856,7 +6871,8 @@ class RuntimeServerTest
"2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -7098,4 +7114,78 @@ class RuntimeServerTest
)
}
it should "support file edit notification with IdMap" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Enso_Test.Test.Main"
val xId = new UUID(0, 1)
val code =
"""from Standard.Base import all
|
|main =
| x = 0
| IO.println x
|""".stripMargin.linesIterator.mkString("\n")
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Create a new file
val mainFile = context.writeMain(code)
// Set sources for the module
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, code))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
// Push new item on the stack to trigger the re-execution
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem
.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
)
)
)
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("0")
// Modify the file
context.send(
Api.Request(
Api.EditFileNotification(
mainFile,
Seq(
TextEdit(
model.Range(model.Position(3, 8), model.Position(3, 9)),
"\"Hello World!\""
)
),
execute = true,
idMap = Some(model.IdMap(Vector(model.Span(46, 60) -> xId)))
)
)
)
context.receiveN(2) shouldEqual Seq(
TestMessages.update(contextId, xId, ConstantsGen.TEXT),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
}
}

View File

@ -212,7 +212,8 @@ class RuntimeSuggestionUpdatesTest
"\n x = 42\n IO.println x"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -284,7 +285,8 @@ class RuntimeSuggestionUpdatesTest
"x+y"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -376,7 +378,8 @@ class RuntimeSuggestionUpdatesTest
"\n y : Number"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -476,7 +479,8 @@ class RuntimeSuggestionUpdatesTest
"\nfoo x = x * 10\n"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -603,7 +607,8 @@ class RuntimeSuggestionUpdatesTest
"a b = a * b"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1265,7 +1270,8 @@ class RuntimeSuggestionUpdatesTest
" hiding hello"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1303,7 +1309,8 @@ class RuntimeSuggestionUpdatesTest
""
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -221,7 +221,8 @@ class RuntimeTextEditsTest
"Meh"
)
),
execute = false
execute = false,
idMap = None
)
)
)
@ -236,7 +237,8 @@ class RuntimeTextEditsTest
"Welcome!"
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -182,7 +182,8 @@ class RuntimeTypesTest
".id"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -262,7 +263,8 @@ class RuntimeTypesTest
" . id"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -345,7 +347,8 @@ class RuntimeTypesTest
"S"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -391,7 +394,8 @@ class RuntimeTypesTest
"T"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -479,7 +483,8 @@ class RuntimeTypesTest
"S"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -518,7 +523,8 @@ class RuntimeTypesTest
"S"
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -705,7 +705,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"5"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -840,7 +841,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"5"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1029,7 +1031,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"5"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1358,7 +1361,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"7"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -1710,7 +1714,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"6"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -3429,7 +3434,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"2"
)
),
execute = true
execute = true,
idMap = None
)
)
)
@ -4192,7 +4198,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
"x"
)
),
execute = true
execute = true,
idMap = None
)
)
)

View File

@ -1,6 +1,9 @@
package org.enso.compiler.core;
import java.util.Map;
import java.util.UUID;
import org.enso.compiler.core.ir.Expression;
import org.enso.compiler.core.ir.Location;
import org.enso.compiler.core.ir.Module;
import org.enso.syntax2.Parser;
import org.enso.syntax2.Tree;
@ -39,6 +42,11 @@ public final class EnsoParser implements AutoCloseable {
return TreeToIr.MODULE.translate(t);
}
public Module generateModuleIr(Tree t, Map<Location, UUID> idMap) {
var treeToIr = new TreeToIr(idMap);
return treeToIr.translate(t);
}
public scala.Option<Expression> generateIRInline(Tree t) {
return TreeToIr.MODULE.translateInline(t);
}

View File

@ -1,6 +1,8 @@
package org.enso.compiler.core;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Map;
import java.util.UUID;
import org.enso.compiler.core.ir.CallArgument;
import org.enso.compiler.core.ir.DefinitionArgument;
@ -49,7 +51,14 @@ final class TreeToIr {
static final String SKIP_MACRO_IDENTIFIER = "SKIP";
static final String FREEZE_MACRO_IDENTIFIER = "FREEZE";
private final Map<Location, UUID> idMap;
private TreeToIr() {
this.idMap = Collections.emptyMap();
}
public TreeToIr(Map<Location, UUID> idMap) {
this.idMap = idMap;
}
/**
@ -1947,15 +1956,14 @@ final class TreeToIr {
private Option<IdentifiedLocation> getIdentifiedLocation(Tree ast, int b, int e,
Option<UUID> someId) {
if (someId == null) {
someId = Option.apply(ast.uuid());
}
return Option.apply(switch (ast) {
case null -> null;
default -> {
var begin = castToInt(ast.getStartCode()) + b;
var end = castToInt(ast.getEndCode()) + e;
yield IdentifiedLocation.create(new Location(begin, end), someId);
var location = new Location(begin, end);
var uuid = Option.apply(idMap.get(location)).orElse(() -> someId == null ? Option.apply(ast.uuid()) : someId);
yield IdentifiedLocation.create(location, uuid);
}
});
}
@ -1990,7 +1998,10 @@ final class TreeToIr {
end = ast.getPattern().getEndCode();
}
int end_ = castToInt(end);
return Option.apply(IdentifiedLocation.create(new Location(begin_, end_), Option.empty()));
var location = new Location(begin_, end_);
var uuid = Option.apply(idMap.get(location));
return Option.apply(IdentifiedLocation.create(location, uuid));
}
private Option<IdentifiedLocation> getIdentifiedLocation(Token ast) {

View File

@ -30,6 +30,7 @@ import org.enso.compiler.context.CompilerContext;
import org.enso.compiler.context.LocalScope;
import org.enso.compiler.core.IR;
import org.enso.compiler.core.ir.Expression;
import org.enso.compiler.data.IdMap;
import org.enso.compiler.suggestions.SimpleUpdate;
import org.enso.interpreter.caches.Cache;
import org.enso.interpreter.caches.ModuleCache;
@ -65,6 +66,7 @@ public final class Module implements EnsoObject {
private CompilationStage compilationStage = CompilationStage.INITIAL;
private org.enso.compiler.core.ir.Module ir;
private Map<UUID, IR> uuidsMap;
private IdMap idMap;
/**
* This list is filled in case there is a directory with the same name as this module. The
@ -452,6 +454,15 @@ public final class Module implements EnsoObject {
this.uuidsMap = null;
}
/**
* Sets the identifiers map for this module.
*
* @param idMap the identifiers map
*/
void unsafeSetIdMap(IdMap idMap) {
this.idMap = idMap;
}
/**
* @return the runtime scope of this module.
*/
@ -499,6 +510,13 @@ public final class Module implements EnsoObject {
return sources.file();
}
/**
* @return the IdMap of this module.
*/
public IdMap getIdMap() {
return idMap;
}
/**
* @return {@code true} if the module is interactive, {@code false} otherwise
*/

View File

@ -29,6 +29,7 @@ import org.enso.compiler.core.ir.Diagnostic;
import org.enso.compiler.core.ir.IdentifiedLocation;
import org.enso.compiler.data.BindingsMap;
import org.enso.compiler.data.CompilerConfig;
import org.enso.compiler.data.IdMap;
import org.enso.compiler.pass.analyse.BindingAnalysis$;
import org.enso.compiler.suggestions.ExportsBuilder;
import org.enso.compiler.suggestions.ExportsMap;
@ -160,6 +161,12 @@ final class TruffleCompilerContext implements CompilerContext {
return module.getCharacters();
}
@Override
public IdMap getIdMap(CompilerContext.Module module) {
var moduleIdMap = module.getIdMap();
return moduleIdMap == null ? IdMap.empty() : moduleIdMap;
}
@Override
public boolean isSynthetic(CompilerContext.Module module) {
return module.isSynthetic();
@ -669,6 +676,7 @@ final class TruffleCompilerContext implements CompilerContext {
private final class ModuleUpdater implements Updater, AutoCloseable {
private final Module module;
private BindingsMap[] map;
private IdMap idMap;
private org.enso.compiler.core.ir.Module[] ir;
private CompilationStage stage;
private Boolean loadedFromCache;
@ -684,6 +692,11 @@ final class TruffleCompilerContext implements CompilerContext {
this.map = new BindingsMap[] {map};
}
@Override
public void idMap(IdMap idMap) {
this.idMap = idMap;
}
@Override
public void ir(org.enso.compiler.core.ir.Module ir) {
this.ir = new org.enso.compiler.core.ir.Module[] {ir};
@ -713,13 +726,16 @@ final class TruffleCompilerContext implements CompilerContext {
public void close() {
if (map != null) {
if (module.bindings != null && map[0] != null) {
loggerCompiler.log(Level.FINEST, "Reassigining bindings to {0}", module);
loggerCompiler.log(Level.FINEST, "Reassigning bindings to {0}", module);
}
module.bindings = map[0];
}
if (ir != null) {
module.module.unsafeSetIr(ir[0]);
}
if (idMap != null) {
module.module.unsafeSetIdMap(idMap);
}
if (stage != null) {
module.module.unsafeSetCompilationStage(stage);
}
@ -736,6 +752,7 @@ final class TruffleCompilerContext implements CompilerContext {
}
public static final class Module extends CompilerContext.Module {
private final org.enso.interpreter.runtime.Module module;
private BindingsMap bindings;
@ -759,7 +776,7 @@ final class TruffleCompilerContext implements CompilerContext {
}
/** Intentionally not public. */
final org.enso.interpreter.runtime.Module unsafeModule() {
org.enso.interpreter.runtime.Module unsafeModule() {
return module;
}
@ -787,7 +804,12 @@ final class TruffleCompilerContext implements CompilerContext {
return bindings;
}
final TruffleFile getSourceFile() {
@Override
public IdMap getIdMap() {
return module.getIdMap();
}
TruffleFile getSourceFile() {
return module.getSourceFile();
}

View File

@ -1,5 +1,7 @@
package org.enso.text.editing
import java.util.UUID
object model {
/** A representation of a position in a text file.
@ -58,4 +60,19 @@ object model {
*/
case class TextEdit(range: Range, text: String)
/** A text span denoted by its absolute start and end indexes.
*
* @param start the start index of a span
* @param end the end index of a span
*/
case class Span(start: Int, end: Int) {
def length: Int = end - start
}
/** The mapping between the text spans and their identifiers.
*
* @param values the list of span-uuid pairs
*/
case class IdMap(values: Vector[(Span, UUID)])
}