When renaming the project clean old modules instead of updating (#6148)

close #6139
close #6137

When the project is renamed, the engine cleans up affected modules and initiates modules re-indexing to fill the suggestions database with new records. This way it reduces the amount of information stored in the suggestions database and helps implement #6080 optimization.

Changelog:
- remove: rename features from the suggestions database
- update: rename command to initiate modules cleanup and project re-execution
- fix: #6137
This commit is contained in:
Dmitry Bushev 2023-03-31 12:40:21 +03:00 committed by GitHub
parent d89de84988
commit 2338e5d8e6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 382 additions and 599 deletions

View File

@ -1,11 +0,0 @@
package org.enso.languageserver.refactoring
import org.enso.languageserver.event.Event
/** An event notifying that project name has changed.
*
* @param oldName the old name of the project
* @param newName the new project name
*/
case class ProjectNameChangedEvent(oldName: String, newName: String)
extends Event

View File

@ -5,7 +5,6 @@ import java.util.UUID
import akka.actor.{Actor, ActorRef, Cancellable, Props} import akka.actor.{Actor, ActorRef, Cancellable, Props}
import com.typesafe.scalalogging.LazyLogging import com.typesafe.scalalogging.LazyLogging
import org.enso.jsonrpc._ import org.enso.jsonrpc._
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.refactoring.RefactoringApi.RenameProject import org.enso.languageserver.refactoring.RefactoringApi.RenameProject
import org.enso.languageserver.requesthandler.RequestTimeout import org.enso.languageserver.requesthandler.RequestTimeout
import org.enso.languageserver.util.UnhandledLogging import org.enso.languageserver.util.UnhandledLogging
@ -37,7 +36,6 @@ class RenameProjectHandler(timeout: FiniteDuration, runtimeConnector: ActorRef)
context.become( context.become(
responseStage( responseStage(
id, id,
ProjectNameChangedEvent(params.oldName, params.newName),
sender(), sender(),
cancellable cancellable
) )
@ -46,7 +44,6 @@ class RenameProjectHandler(timeout: FiniteDuration, runtimeConnector: ActorRef)
private def responseStage( private def responseStage(
id: Id, id: Id,
nameChangedEvent: ProjectNameChangedEvent,
replyTo: ActorRef, replyTo: ActorRef,
cancellable: Cancellable cancellable: Cancellable
): Receive = { ): Receive = {
@ -56,7 +53,6 @@ class RenameProjectHandler(timeout: FiniteDuration, runtimeConnector: ActorRef)
context.stop(self) context.stop(self)
case Api.Response(_, Api.ProjectRenamed(_, _)) => case Api.Response(_, Api.ProjectRenamed(_, _)) =>
context.system.eventStream.publish(nameChangedEvent)
replyTo ! ResponseResult(RenameProject, id, Unused) replyTo ! ResponseResult(RenameProject, id, Unused)
cancellable.cancel() cancellable.cancel()
context.stop(self) context.stop(self)

View File

@ -23,7 +23,6 @@ import org.enso.languageserver.filemanager.{
FileDeletedEvent, FileDeletedEvent,
Path Path
} }
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.runtime.RuntimeFailureMapper import org.enso.languageserver.runtime.RuntimeFailureMapper
import org.enso.languageserver.search.SearchProtocol._ import org.enso.languageserver.search.SearchProtocol._
import org.enso.languageserver.search.handler.InvalidateModulesIndexHandler import org.enso.languageserver.search.handler.InvalidateModulesIndexHandler
@ -36,7 +35,6 @@ import org.enso.polyglot.data.TypeGraph
import org.enso.polyglot.runtime.Runtime.Api import org.enso.polyglot.runtime.Runtime.Api
import org.enso.searcher.data.QueryResult import org.enso.searcher.data.QueryResult
import org.enso.searcher.{SuggestionsRepo, VersionsRepo} import org.enso.searcher.{SuggestionsRepo, VersionsRepo}
import org.enso.text.ContentVersion
import org.enso.text.editing.model.Position import org.enso.text.editing.model.Position
import scala.collection.mutable import scala.collection.mutable
@ -115,7 +113,6 @@ final class SuggestionsHandler(
) )
context.system.eventStream context.system.eventStream
.subscribe(self, classOf[Api.LibraryLoaded]) .subscribe(self, classOf[Api.LibraryLoaded])
context.system.eventStream.subscribe(self, classOf[ProjectNameChangedEvent])
context.system.eventStream.subscribe(self, classOf[FileDeletedEvent]) context.system.eventStream.subscribe(self, classOf[FileDeletedEvent])
context.system.eventStream context.system.eventStream
.subscribe(self, InitializedEvent.SuggestionsRepoInitialized.getClass) .subscribe(self, InitializedEvent.SuggestionsRepoInitialized.getClass)
@ -127,17 +124,6 @@ final class SuggestionsHandler(
initializing(SuggestionsHandler.Initialization()) initializing(SuggestionsHandler.Initialization())
private def initializing(init: SuggestionsHandler.Initialization): Receive = { private def initializing(init: SuggestionsHandler.Initialization): Receive = {
case ProjectNameChangedEvent(oldName, newName) =>
logger.info(
"Initializing: project name changed from [{}] to [{}].",
oldName,
newName
)
suggestionsRepo
.renameProject(oldName, newName)
.map(_ => ProjectNameUpdated(newName))
.pipeTo(self)
case ProjectNameUpdated(name, updates) => case ProjectNameUpdated(name, updates) =>
logger.info("Initializing: project name is updated to [{}].", name) logger.info("Initializing: project name is updated to [{}].", name)
updates.foreach(sessionRouter ! _) updates.foreach(sessionRouter ! _)
@ -249,7 +235,7 @@ final class SuggestionsHandler(
case SuggestionUpdatesBatch(updates) => case SuggestionUpdatesBatch(updates) =>
val modules = updates.map(_.module) val modules = updates.map(_.module)
traverseSeq(updates)(applyUpdateIfVersionChanged) traverseSeq(updates)(applyDatabaseUpdates)
.onComplete { .onComplete {
case Success(results) => case Success(results) =>
logger.debug( logger.debug(
@ -257,17 +243,15 @@ final class SuggestionsHandler(
modules.length, modules.length,
modules.mkString(", ") modules.mkString(", ")
) )
results.foreach { results.foreach { notification =>
case Some(notification) => if (notification.updates.nonEmpty) {
if (notification.updates.nonEmpty) { clients.foreach { clientId =>
clients.foreach { clientId => sessionRouter ! DeliverToJsonController(
sessionRouter ! DeliverToJsonController( clientId,
clientId, notification
notification )
)
}
} }
case None => }
} }
self ! SuggestionsHandler.SuggestionUpdatesCompleted self ! SuggestionsHandler.SuggestionUpdatesCompleted
case Failure(ex) => case Failure(ex) =>
@ -290,9 +274,9 @@ final class SuggestionsHandler(
case msg: Api.SuggestionsDatabaseModuleUpdateNotification => case msg: Api.SuggestionsDatabaseModuleUpdateNotification =>
logger.debug("Got module update [{}].", msg.module) logger.debug("Got module update [{}].", msg.module)
applyUpdateIfVersionChanged(msg) applyDatabaseUpdates(msg)
.onComplete { .onComplete {
case Success(Some(notification)) => case Success(notification) =>
logger.debug("Complete module update [{}].", msg.module) logger.debug("Complete module update [{}].", msg.module)
if (notification.updates.nonEmpty) { if (notification.updates.nonEmpty) {
clients.foreach { clientId => clients.foreach { clientId =>
@ -300,12 +284,6 @@ final class SuggestionsHandler(
} }
} }
self ! SuggestionsHandler.SuggestionUpdatesCompleted self ! SuggestionsHandler.SuggestionUpdatesCompleted
case Success(None) =>
logger.debug(
"Skip module update, version not changed [{}].",
msg.module
)
self ! SuggestionsHandler.SuggestionUpdatesCompleted
case Failure(ex) => case Failure(ex) =>
logger.error( logger.error(
"Error applying suggestion database updates [{}, {}].", "Error applying suggestion database updates [{}, {}].",
@ -458,55 +436,6 @@ final class SuggestionsHandler(
) )
action.pipeTo(handler)(sender()) action.pipeTo(handler)(sender())
case ProjectNameChangedEvent(oldName, newName) =>
suggestionsRepo
.renameProject(oldName, newName)
.map {
case (version, moduleIds, selfTypeIds, returnTypeIds, argumentIds) =>
val suggestionModuleUpdates = moduleIds.map {
case (suggestionId, moduleName) =>
SuggestionsDatabaseUpdate.Modify(
id = suggestionId,
module = Some(fieldUpdate(moduleName))
)
}
val selfTypeUpdates = selfTypeIds.map {
case (suggestionId, selfType) =>
SuggestionsDatabaseUpdate.Modify(
id = suggestionId,
selfType = Some(fieldUpdate(selfType))
)
}
val returnTypeUpdates = returnTypeIds.map {
case (suggestionId, returnType) =>
SuggestionsDatabaseUpdate.Modify(
id = suggestionId,
returnType = Some(fieldUpdate(returnType))
)
}
val argumentUpdates =
argumentIds.groupBy(_._1).map { case (suggestionId, grouped) =>
val argumentUpdates = grouped.map { case (_, index, typeName) =>
SuggestionArgumentUpdate.Modify(
index = index,
reprType = Some(fieldUpdate(typeName))
)
}
SuggestionsDatabaseUpdate.Modify(
id = suggestionId,
arguments = Some(argumentUpdates)
)
}
val notification =
SuggestionsDatabaseUpdateNotification(
version,
suggestionModuleUpdates ++ selfTypeUpdates ++ returnTypeUpdates ++ argumentUpdates
)
val updates = clients.map(DeliverToJsonController(_, notification))
ProjectNameUpdated(newName, updates)
}
.pipeTo(self)
case ProjectNameUpdated(name, updates) => case ProjectNameUpdated(name, updates) =>
updates.foreach(sessionRouter ! _) updates.foreach(sessionRouter ! _)
context.become(initialized(name, graph, clients, state)) context.become(initialized(name, graph, clients, state))
@ -559,19 +488,6 @@ final class SuggestionsHandler(
} }
} }
private def applyUpdateIfVersionChanged(
msg: Api.SuggestionsDatabaseModuleUpdateNotification
): Future[Option[SuggestionsDatabaseUpdateNotification]] = {
val isVersionChanged =
versionsRepo.getVersion(msg.module).map { digestOpt =>
!digestOpt.map(ContentVersion(_)).contains(msg.version)
}
isVersionChanged.flatMap { isChanged =>
if (isChanged) applyDatabaseUpdates(msg).map(Some(_))
else Future.successful(None)
}
}
/** Handle the suggestions of the loaded library. /** Handle the suggestions of the loaded library.
* *
* Adds the new suggestions to the suggestions database and sends the * Adds the new suggestions to the suggestions database and sends the

View File

@ -11,7 +11,6 @@ import org.enso.languageserver.capability.CapabilityProtocol.{
import org.enso.languageserver.data._ import org.enso.languageserver.data._
import org.enso.languageserver.event.InitializedEvent import org.enso.languageserver.event.InitializedEvent
import org.enso.languageserver.filemanager._ import org.enso.languageserver.filemanager._
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.search.SearchProtocol.SuggestionDatabaseEntry import org.enso.languageserver.search.SearchProtocol.SuggestionDatabaseEntry
import org.enso.languageserver.session.JsonSession import org.enso.languageserver.session.JsonSession
import org.enso.languageserver.session.SessionRouter.DeliverToJsonController import org.enso.languageserver.session.SessionRouter.DeliverToJsonController
@ -745,89 +744,6 @@ class SuggestionsHandlerSpec
expectMsg(SearchProtocol.InvalidateSuggestionsDatabaseResult) expectMsg(SearchProtocol.InvalidateSuggestionsDatabaseResult)
} }
"rename module when renaming project" taggedAs Retry in withDb {
(_, repo, router, _, handler) =>
Await.ready(repo.insert(Suggestions.constructor), Timeout)
val clientId = UUID.randomUUID()
val newModuleName = "Vest"
// acquire capability
handler ! AcquireCapability(
newJsonSession(clientId),
CapabilityRegistration(ReceivesSuggestionsDatabaseUpdates())
)
expectMsg(CapabilityAcquired)
handler ! ProjectNameChangedEvent("Test", newModuleName)
router.expectMsg(
DeliverToJsonController(
clientId,
SearchProtocol.SuggestionsDatabaseUpdateNotification(
2,
Seq(
SearchProtocol.SuggestionsDatabaseUpdate.Modify(
id = 1,
module = Some(fieldUpdate("local.Vest.Main"))
)
)
)
)
)
}
"rename types when renaming project" taggedAs Retry in withDb {
(_, repo, router, _, handler) =>
val method = Suggestions.method.copy(
selfType = "local.Test.MyType",
arguments = Suggestions.method.arguments.map(arg =>
arg.copy(reprType = "local.Test.MyType")
)
)
Await.ready(repo.insert(method), Timeout)
val clientId = UUID.randomUUID()
val newModuleName = "Vest"
// acquire capability
handler ! AcquireCapability(
newJsonSession(clientId),
CapabilityRegistration(ReceivesSuggestionsDatabaseUpdates())
)
expectMsg(CapabilityAcquired)
handler ! ProjectNameChangedEvent("Test", newModuleName)
router.expectMsg(
DeliverToJsonController(
clientId,
SearchProtocol.SuggestionsDatabaseUpdateNotification(
2,
Seq(
SearchProtocol.SuggestionsDatabaseUpdate.Modify(
id = 1,
module = Some(fieldUpdate("local.Vest.Main"))
),
SearchProtocol.SuggestionsDatabaseUpdate.Modify(
id = 1,
selfType = Some(fieldUpdate("local.Vest.MyType"))
),
SearchProtocol.SuggestionsDatabaseUpdate.Modify(
id = 1,
arguments = Some(
method.arguments.zipWithIndex.map { case (_, index) =>
SearchProtocol.SuggestionArgumentUpdate.Modify(
index = index,
reprType = Some(fieldUpdate("local.Vest.MyType"))
)
}
)
)
)
)
)
)
}
"search entries by empty search query" taggedAs Retry in withDb { "search entries by empty search query" taggedAs Retry in withDb {
(config, repo, _, _, handler) => (config, repo, _, _, handler) =>
val (_, inserted) = val (_, inserted) =

View File

@ -34,9 +34,9 @@ import org.enso.languageserver.protocol.json.{
JsonConnectionControllerFactory, JsonConnectionControllerFactory,
JsonRpcProtocolFactory JsonRpcProtocolFactory
} }
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.runtime.{ContextRegistry, RuntimeFailureMapper} import org.enso.languageserver.runtime.{ContextRegistry, RuntimeFailureMapper}
import org.enso.languageserver.search.SuggestionsHandler import org.enso.languageserver.search.SuggestionsHandler
import org.enso.languageserver.search.SuggestionsHandler.ProjectNameUpdated
import org.enso.languageserver.session.SessionRouter import org.enso.languageserver.session.SessionRouter
import org.enso.languageserver.text.BufferRegistry import org.enso.languageserver.text.BufferRegistry
import org.enso.languageserver.vcsmanager.{Git, VcsManager} import org.enso.languageserver.vcsmanager.{Git, VcsManager}
@ -255,7 +255,7 @@ class BaseServerTest
Api.GetTypeGraphResponse(typeGraph) Api.GetTypeGraphResponse(typeGraph)
) )
Await.ready(initializationComponent.init(), timeout) Await.ready(initializationComponent.init(), timeout)
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test")) suggestionsHandler ! ProjectNameUpdated("Test")
val environment = fakeInstalledEnvironment() val environment = fakeInstalledEnvironment()
val languageHome = LanguageHome.detectFromExecutableLocation(environment) val languageHome = LanguageHome.detectFromExecutableLocation(environment)

View File

@ -3662,9 +3662,29 @@ class RuntimeServerTest
context.send( context.send(
Api.Request(requestId, Api.RenameProject("Enso_Test", "Test", "Foo")) Api.Request(requestId, Api.RenameProject("Enso_Test", "Test", "Foo"))
) )
context.receiveN(1) should contain theSameElementsAs Seq( val renameProjectResponses = context.receiveN(6)
Api.Response(requestId, Api.ProjectRenamed("Enso_Test", "Foo")) renameProjectResponses should contain allOf (
Api.Response(requestId, Api.ProjectRenamed("Enso_Test", "Foo")),
context.Main.Update.mainX(contextId),
TestMessages.update(
contextId,
context.Main.idMainY,
ConstantsGen.INTEGER,
Api.MethodPointer("Enso_Test.Foo.Main", ConstantsGen.NUMBER, "foo")
),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
) )
renameProjectResponses.collect {
case Api.Response(
_,
notification: Api.SuggestionsDatabaseModuleUpdateNotification
) =>
notification.module shouldEqual moduleName
notification.actions should contain theSameElementsAs Vector(
Api.SuggestionsDatabaseAction.Clean(moduleName)
)
}
// recompute existing stack // recompute existing stack
context.send( context.send(
@ -3707,6 +3727,111 @@ class RuntimeServerTest
) )
} }
it should "push and pop functions after renaming the project" in {
val contents = context.Main.code
val mainFile = context.writeMain(contents)
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// open file
context.send(
Api.Request(Api.OpenFileNotification(mainFile, contents))
)
context.receiveNone shouldEqual None
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
// rename Test -> Foo
context.pkg.rename("Foo")
context.send(
Api.Request(requestId, Api.RenameProject("Enso_Test", "Test", "Foo"))
)
val renameProjectResponses = context.receiveN(6)
renameProjectResponses should contain allOf (
Api.Response(requestId, Api.ProjectRenamed("Enso_Test", "Foo")),
context.Main.Update.mainX(contextId),
TestMessages.update(
contextId,
context.Main.idMainY,
ConstantsGen.INTEGER,
Api.MethodPointer("Enso_Test.Foo.Main", ConstantsGen.NUMBER, "foo")
),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
renameProjectResponses.collect {
case Api.Response(
_,
notification: Api.SuggestionsDatabaseModuleUpdateNotification
) =>
notification.module shouldEqual moduleName
notification.actions should contain theSameElementsAs Vector(
Api.SuggestionsDatabaseAction.Clean(moduleName)
)
}
// push foo call
val item2 = Api.StackItem.LocalCall(context.Main.idMainY)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item2))
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.fooY(contextId),
context.Main.Update.fooZ(contextId),
context.executionComplete(contextId)
)
// pop foo call
context.send(Api.Request(requestId, Api.PopContextRequest(contextId)))
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PopContextResponse(contextId)),
TestMessages.update(
contextId,
context.Main.idMainY,
ConstantsGen.INTEGER,
Api.MethodPointer("Enso_Test.Foo.Main", ConstantsGen.NUMBER, "foo"),
fromCache = true
),
context.executionComplete(contextId)
)
// pop main
context.send(Api.Request(requestId, Api.PopContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.PopContextResponse(contextId))
)
// pop empty stack
context.send(Api.Request(requestId, Api.PopContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.EmptyStackError(contextId))
)
}
it should "send the type graph" in { it should "send the type graph" in {
val requestId = UUID.randomUUID() val requestId = UUID.randomUUID()
val expectedGraph: TypeGraph = Types.getTypeHierarchy val expectedGraph: TypeGraph = Types.getTypeHierarchy

View File

@ -698,6 +698,152 @@ class RuntimeSuggestionUpdatesTest
context.consumeOut shouldEqual List("51") context.consumeOut shouldEqual List("51")
} }
it should "send suggestion updates after renaming the project" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Enso_Test.Test.Main"
val code =
"""from Standard.Base import all
|
|main = IO.println "Hello World!"
|""".stripMargin.linesIterator.mkString("\n")
val version = contentsVersion(code)
val mainFile = context.writeMain(code)
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// open file
context.send(
Api.Request(Api.OpenFileNotification(mainFile, code))
)
context.receiveNone shouldEqual None
// push main
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, "Enso_Test.Test.Main", "main"),
None,
Vector()
)
)
)
)
context.receiveNIgnoreExpressionUpdates(
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
module = moduleName,
version = version,
actions = Vector(Api.SuggestionsDatabaseAction.Clean(moduleName)),
exports = Vector(),
updates = Tree.Root(
Vector(
Tree.Node(
Api.SuggestionUpdate(
Suggestion.Module(
moduleName,
None
),
Api.SuggestionAction.Add()
),
Vector()
),
Tree.Node(
Api.SuggestionUpdate(
Suggestion.Method(
None,
moduleName,
"main",
List(),
"Enso_Test.Test.Main",
ConstantsGen.ANY,
true,
None
),
Api.SuggestionAction.Add()
),
Vector()
)
)
)
)
),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
// rename Test -> Foo
context.pkg.rename("Foo")
context.send(
Api.Request(requestId, Api.RenameProject("Enso_Test", "Test", "Foo"))
)
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.ProjectRenamed("Enso_Test", "Foo")),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
module = moduleName,
version = version,
actions = Vector(Api.SuggestionsDatabaseAction.Clean(moduleName)),
exports = Vector(),
updates = Tree.empty
)
),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
module = "Enso_Test.Foo.Main",
version = version,
actions =
Vector(Api.SuggestionsDatabaseAction.Clean("Enso_Test.Foo.Main")),
exports = Vector(),
updates = Tree.Root(
Vector(
Tree.Node(
Api.SuggestionUpdate(
Suggestion.Module(
"Enso_Test.Foo.Main",
None
),
Api.SuggestionAction.Add()
),
Vector()
),
Tree.Node(
Api.SuggestionUpdate(
Suggestion.Method(
None,
"Enso_Test.Foo.Main",
"main",
List(),
"Enso_Test.Foo.Main",
ConstantsGen.ANY,
true,
None
),
Api.SuggestionAction.Add()
),
Vector()
)
)
)
)
),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
}
it should "index overloaded functions" in { it should "index overloaded functions" in {
val contextId = UUID.randomUUID() val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID() val requestId = UUID.randomUUID()

View File

@ -109,7 +109,7 @@ class RecomputeContextCmd(
val stack = ctx.contextManager.getStack(request.contextId) val stack = ctx.contextManager.getStack(request.contextId)
val executable = Executable(request.contextId, stack) val executable = Executable(request.contextId, stack)
for { for {
_ <- Future(ctx.jobProcessor.run(EnsureCompiledJob(executable.stack))) _ <- ctx.jobProcessor.run(EnsureCompiledJob(executable.stack))
_ <- ctx.jobProcessor.run(new ExecuteJob(executable)) _ <- ctx.jobProcessor.run(new ExecuteJob(executable))
} yield () } yield ()
} else { } else {

View File

@ -1,10 +1,12 @@
package org.enso.interpreter.instrument.command package org.enso.interpreter.instrument.command
import java.util.logging.Level import java.util.logging.Level
import org.enso.interpreter.instrument.{CacheInvalidation, InstrumentFrame}
import org.enso.interpreter.instrument.InstrumentFrame
import org.enso.interpreter.instrument.execution.RuntimeContext import org.enso.interpreter.instrument.execution.RuntimeContext
import org.enso.interpreter.instrument.job.{EnsureCompiledJob, ExecuteJob}
import org.enso.interpreter.runtime.Module
import org.enso.pkg.QualifiedName import org.enso.pkg.QualifiedName
import org.enso.polyglot.data.Tree
import org.enso.polyglot.runtime.Runtime.Api import org.enso.polyglot.runtime.Runtime.Api
import scala.collection.mutable import scala.collection.mutable
@ -25,31 +27,75 @@ class RenameProjectCmd(
ctx: RuntimeContext, ctx: RuntimeContext,
ec: ExecutionContext ec: ExecutionContext
): Future[Unit] = ): Future[Unit] =
Future { for {
ctx.locking.acquireWriteCompilationLock() _ <- Future { doRename }
try { _ <- reExecute
val logger = ctx.executionService.getLogger } yield ()
logger.log(
Level.FINE, private def doRename(implicit ctx: RuntimeContext): Unit = {
s"Renaming project [old:${request.namespace}.${request.oldName},new:${request.namespace}.${request.newName}]..." ctx.locking.acquireWriteCompilationLock()
try {
val logger = ctx.executionService.getLogger
logger.log(
Level.FINE,
s"Renaming project [old:${request.namespace}.${request.oldName},new:${request.namespace}.${request.newName}]..."
)
val projectModules = getProjectModules
projectModules.foreach { module =>
module.setIndexed(false)
ctx.endpoint.sendToClient(
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
module = module.getName.toString,
version =
ctx.versioning.evalVersion(module.getSource.getCharacters),
actions = Vector(
Api.SuggestionsDatabaseAction.Clean(module.getName.toString)
),
exports = Vector(),
updates = Tree.empty
)
)
) )
val context = ctx.executionService.getContext
context.renameProject(
request.namespace,
request.oldName,
request.newName
)
ctx.contextManager.getAllContexts.values
.foreach(updateMethodPointers(request.newName, _))
reply(Api.ProjectRenamed(request.namespace, request.newName))
logger.log(
Level.INFO,
s"Project renamed to ${request.namespace}.${request.newName}"
)
} finally {
ctx.locking.releaseWriteCompilationLock()
} }
val context = ctx.executionService.getContext
context.renameProject(
request.namespace,
request.oldName,
request.newName
)
ctx.contextManager.getAllContexts.values.foreach { stack =>
updateMethodPointers(request.newName, stack)
clearCache(stack)
}
reply(Api.ProjectRenamed(request.namespace, request.newName))
logger.log(
Level.INFO,
s"Project renamed to ${request.namespace}.${request.newName}"
)
} finally {
ctx.locking.releaseWriteCompilationLock()
} }
}
private def reExecute(implicit
ctx: RuntimeContext,
ec: ExecutionContext
): Future[Unit] =
for {
_ <- Future.sequence {
ctx.contextManager.getAllContexts.toVector.map {
case (contextId, stack) =>
for {
_ <- ctx.jobProcessor.run(EnsureCompiledJob(stack))
_ <- ctx.jobProcessor.run(new ExecuteJob(contextId, stack.toList))
} yield ()
}
}
} yield ()
/** Update module name of method pointers in the stack. /** Update module name of method pointers in the stack.
* *
@ -66,9 +112,33 @@ class RenameProjectCmd(
.fromString(call.methodPointer.module) .fromString(call.methodPointer.module)
.renameProject(projectName) .renameProject(projectName)
.toString .toString
val methodPointer = call.methodPointer.copy(module = moduleName) val typeName = QualifiedName
.fromString(call.methodPointer.definedOnType)
.renameProject(projectName)
.toString
val methodPointer =
call.methodPointer.copy(module = moduleName, definedOnType = typeName)
InstrumentFrame(call.copy(methodPointer = methodPointer), cache, sync) InstrumentFrame(call.copy(methodPointer = methodPointer), cache, sync)
case item => item case item => item
} }
} }
private def getProjectModules(implicit ctx: RuntimeContext): Seq[Module] = {
val packageRepository = ctx.executionService.getContext.getPackageRepository
packageRepository.getMainProjectPackage
.map { pkg => packageRepository.getModulesForLibrary(pkg.libraryName) }
.getOrElse(List())
}
private def clearCache(stack: Iterable[InstrumentFrame]): Unit = {
stack.foreach(_.syncState.clearMethodPointersState())
CacheInvalidation.run(
stack,
CacheInvalidation(
CacheInvalidation.StackSelector.All,
CacheInvalidation.Command.InvalidateAll
)
)
}
} }

View File

@ -157,24 +157,4 @@ trait SuggestionsRepo[F[_]] {
/** Cleans the repo resetting the version. */ /** Cleans the repo resetting the version. */
def clean: F[Unit] def clean: F[Unit]
/** Update the suggestions with the new project name.
*
* @param oldName the old name of the project
* @param newName the new project name
* @return the current database version and lists of suggestion ids with
* updated module name, self type, return type and arguments
*/
def renameProject(
oldName: String,
newName: String
): F[
(
Long,
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, Int, String)]
)
]
} }

View File

@ -144,21 +144,6 @@ final class SqlSuggestionsRepo(val db: SqlDatabase)(implicit
): Future[(Long, Seq[Option[Long]])] = ): Future[(Long, Seq[Option[Long]])] =
db.run(updateAllQuery(expressions).transactionally) db.run(updateAllQuery(expressions).transactionally)
/** @inheritdoc */
override def renameProject(
oldName: String,
newName: String
): Future[
(
Long,
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, Int, String)]
)
] =
db.run(renameProjectQuery(oldName, newName).transactionally)
/** @inheritdoc */ /** @inheritdoc */
override def currentVersion: Future[Long] = override def currentVersion: Future[Long] =
db.run(currentVersionQuery) db.run(currentVersionQuery)
@ -787,73 +772,6 @@ final class SqlSuggestionsRepo(val db: SqlDatabase)(implicit
query query
} }
/** The query to update the project name.
*
* @param oldName the old name of the project
* @param newName the new project name
* @return the current database version and lists of suggestion ids
* with updated module name, self type, return type and arguments
*/
private def renameProjectQuery(
oldName: String,
newName: String
): DBIO[
(
Long,
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, String)],
Seq[(Long, Int, String)]
)
] = {
def updateQuery(column: String) =
sqlu"""update suggestions
set #$column =
substr(#$column, 0, instr(#$column, $oldName)) ||
$newName ||
substr(#$column, instr(#$column, $oldName) + length($oldName))
where #$column like '%.#$oldName.%'"""
val argumentsUpdateQuery =
sqlu"""update arguments
set type =
substr(type, 0, instr(type, $oldName)) ||
$newName ||
substr(type, instr(type, $oldName) + length($oldName))
where type like '%.#$oldName.%'"""
def noop[A] = DBIO.successful(Seq[A]())
val selectUpdatedModulesQuery = Suggestions
.filter(row => row.module.like(s"%.$newName.%"))
.map(row => (row.id, row.module))
.result
val selectUpdatedSelfTypesQuery = Suggestions
.filter(_.selfType.like(s"%.$newName.%"))
.map(row => (row.id, row.selfType))
.result
val selectUpdatedReturnTypesQuery = Suggestions
.filter(_.returnType.like(s"%.$newName.%"))
.map(row => (row.id, row.returnType))
.result
val selectUpdatedArgumentsQuery = Arguments
.filter(_.tpe.like(s"%.$newName.%"))
.map(row => (row.suggestionId, row.index, row.tpe))
.result
for {
n1 <- updateQuery("module")
moduleIds <- if (n1 > 0) selectUpdatedModulesQuery else noop
n2 <- updateQuery("self_type")
selfTypeIds <- if (n2 > 0) selectUpdatedSelfTypesQuery else noop
n3 <- updateQuery("return_type")
returnTypeIds <- if (n3 > 0) selectUpdatedReturnTypesQuery else noop
n4 <- argumentsUpdateQuery
argumentIds <- if (n4 > 0) selectUpdatedArgumentsQuery else noop
version <-
if (n1 > 0 || n2 > 0 || n3 > 0 || n4 > 0) incrementVersionQuery
else currentVersionQuery
} yield (version, moduleIds, selfTypeIds, returnTypeIds, argumentIds)
}
/** The query to get current version of the repo. */ /** The query to get current version of the repo. */
private def currentVersionQuery: DBIO[Long] = { private def currentVersionQuery: DBIO[Long] = {
for { for {

View File

@ -998,279 +998,6 @@ class SuggestionsRepoTest extends AnyWordSpec with Matchers with RetrySpec {
v1 shouldEqual v2 v1 shouldEqual v2
} }
"rename the project name" taggedAs Retry in withRepo { repo =>
val newModuleName = "local.Best.Main"
val newSelfType = "local.Best.Main"
val newReturnType = "local.Best.Main.MyType"
val action = for {
(_, ids) <- repo.insertAll(
Seq(
suggestion.module,
suggestion.tpe,
suggestion.constructor,
suggestion.method,
suggestion.conversion,
suggestion.function,
suggestion.local
)
)
(_, xs1, xs2, xs3, xs4) <- repo.renameProject("Test", "Best")
(_, res) <- repo.getAll
} yield (ids, xs1, xs2, xs3, xs4, res)
val (ids, xs1, xs2, xs3, xs4, res) = Await.result(action, Timeout)
xs1 should contain theSameElementsAs ids.flatten.map((_, newModuleName))
xs2 should contain theSameElementsAs Seq(ids(3)).flatten
.map((_, newSelfType))
xs3 should contain theSameElementsAs Seq(ids(4), ids(5), ids(6)).flatten
.map {
case id if ids(4).get == id => (id, "local.Best.Main.Bar")
case id => (id, newReturnType)
}
xs4 should contain theSameElementsAs Seq(
(ids(4).get, 0, "local.Best.Main.Foo")
)
res.map(_.suggestion) should contain theSameElementsAs Seq(
suggestion.module.copy(module = newModuleName),
suggestion.tpe.copy(module = newModuleName),
suggestion.constructor.copy(module = newModuleName),
suggestion.method
.copy(module = newModuleName, selfType = newSelfType),
suggestion.conversion.copy(
module = newModuleName,
sourceType = "local.Best.Main.Foo",
returnType = "local.Best.Main.Bar"
),
suggestion.function
.copy(module = newModuleName, returnType = newReturnType),
suggestion.local
.copy(module = newModuleName, returnType = newReturnType)
)
}
"rename the module containing project name" taggedAs Retry in withRepo {
repo =>
val newModuleName = "local.Best.Main"
val newSelfType = "local.Best.Main"
val newReturnType = "local.Best.Main.MyType"
val constructor =
suggestion.constructor.copy(module = "local.Test.Main.Test.Main")
val all =
Seq(
suggestion.module,
suggestion.tpe,
constructor,
suggestion.method,
suggestion.conversion,
suggestion.function,
suggestion.local
)
val action = for {
(_, ids) <- repo.insertAll(all)
(_, xs1, xs2, xs3, xs4) <- repo.renameProject("Test", "Best")
(_, res) <- repo.getAll
} yield (ids, xs1, xs2, xs3, xs4, res)
val (ids, xs1, xs2, xs3, xs4, res) = Await.result(action, Timeout)
xs1 should contain theSameElementsAs ids.zip(all).flatMap {
case (idOpt, _: Suggestion.Constructor) =>
idOpt.map((_, "local.Best.Main.Test.Main"))
case (idOpt, _) =>
idOpt.map((_, newModuleName))
}
xs2 should contain theSameElementsAs Seq(ids(3)).flatten
.map((_, newSelfType))
xs3 should contain theSameElementsAs Seq(ids(4), ids(5), ids(6)).flatten
.map {
case id if ids(4).get == id => (id, "local.Best.Main.Bar")
case id => (id, newReturnType)
}
xs4 should contain theSameElementsAs Seq(
(ids(4).get, 0, "local.Best.Main.Foo")
)
res.map(_.suggestion) should contain theSameElementsAs Seq(
suggestion.module.copy(module = newModuleName),
suggestion.tpe.copy(module = newModuleName),
constructor.copy(module = "local.Best.Main.Test.Main"),
suggestion.method
.copy(module = newModuleName, selfType = newSelfType),
suggestion.conversion.copy(
module = newModuleName,
sourceType = "local.Best.Main.Foo",
returnType = "local.Best.Main.Bar"
),
suggestion.function
.copy(module = newModuleName, returnType = newReturnType),
suggestion.local
.copy(module = newModuleName, returnType = newReturnType)
)
}
"rename multiple modules containing project name" taggedAs Retry in withRepo {
repo =>
val newMainModuleName = "local.Best.Main"
val newFooModuleName = "local.Best.Foo"
val newReturnTypeName = "local.Best.Main.MyType"
val module = suggestion.module.copy(module = "local.Test.Main")
val tpe = suggestion.tpe.copy(module = "local.Test.Main")
val constructor =
suggestion.constructor.copy(module = "local.Test.Main")
val method = suggestion.method.copy(module = "local.Test.Foo")
val conversion = suggestion.conversion.copy(module = "local.Test.Foo")
val function = suggestion.function.copy(module = "local.Bar.Main")
val local = suggestion.local.copy(module = "local.Bar.Main")
val all =
Seq(module, tpe, constructor, method, conversion, function, local)
val action = for {
(_, ids) <- repo.insertAll(all)
(_, xs1, xs2, xs3, xs4) <- repo.renameProject("Test", "Best")
(_, res) <- repo.getAll
} yield (ids, xs1, xs2, xs3, xs4, res)
val (ids, xs1, xs2, xs3, xs4, res) = Await.result(action, Timeout)
xs1 should contain theSameElementsAs ids
.zip(Seq(module, tpe, constructor, method, conversion))
.flatMap {
case (idOpt, _: Suggestion.Module) =>
idOpt.map((_, newMainModuleName))
case (idOpt, _: Suggestion.Type) =>
idOpt.map((_, newMainModuleName))
case (idOpt, _: Suggestion.Constructor) =>
idOpt.map((_, newMainModuleName))
case (idOpt, _) =>
idOpt.map((_, newFooModuleName))
}
xs2 should contain theSameElementsAs Seq(ids(3)).flatten
.map((_, newMainModuleName))
xs3 should contain theSameElementsAs Seq(ids(4), ids(5), ids(6)).flatten
.map {
case id if ids(4).get == id => (id, "local.Best.Main.Bar")
case id => (id, newReturnTypeName)
}
xs4 should contain theSameElementsAs Seq(
(ids(4).get, 0, "local.Best.Main.Foo")
)
res.map(_.suggestion) should contain theSameElementsAs Seq(
module.copy(module = newMainModuleName),
tpe.copy(module = newMainModuleName),
constructor.copy(module = newMainModuleName),
method.copy(module = newFooModuleName, selfType = newMainModuleName),
suggestion.conversion.copy(
module = newFooModuleName,
sourceType = "local.Best.Main.Foo",
returnType = "local.Best.Main.Bar"
),
function.copy(returnType = newReturnTypeName),
local.copy(returnType = newReturnTypeName)
)
}
"rename arguments containing project name" taggedAs Retry in withRepo {
repo =>
val newModuleName = "local.Best.Main"
val newSelfType = "local.Best.Main"
val newReturnType = "local.Best.Main.MyType"
val newArgumentType = "local.Best.Main.Test.MyType"
val method = suggestion.method.copy(arguments =
Seq(
Suggestion.Argument("x", "Number", false, true, Some("0")),
Suggestion.Argument(
"y",
"local.Test.Main.Test.MyType",
false,
false,
None
)
)
)
val all =
Seq(
suggestion.module,
suggestion.tpe,
suggestion.constructor,
method,
suggestion.conversion,
suggestion.function,
suggestion.local
)
val action = for {
(_, ids) <- repo.insertAll(all)
(_, xs1, xs2, xs3, xs4) <- repo.renameProject("Test", "Best")
(_, res) <- repo.getAll
} yield (ids, xs1, xs2, xs3, xs4, res)
val (ids, xs1, xs2, xs3, xs4, res) = Await.result(action, Timeout)
xs1 should contain theSameElementsAs ids.flatten.map((_, newModuleName))
xs2 should contain theSameElementsAs Seq(ids(3)).flatten
.map((_, newSelfType))
xs3 should contain theSameElementsAs Seq(ids(4), ids(5), ids(6)).flatten
.map {
case id if ids(4).get == id => (id, "local.Best.Main.Bar")
case id => (id, newReturnType)
}
xs4 should contain theSameElementsAs Seq(ids(3), ids(4)).flatMap {
_.map {
case id if ids(4).get == id => (id, 0, "local.Best.Main.Foo")
case id => (id, 1, newArgumentType)
}
}
res.map(_.suggestion) should contain theSameElementsAs Seq(
suggestion.module.copy(module = newModuleName),
suggestion.tpe.copy(module = newModuleName),
suggestion.constructor.copy(module = newModuleName),
method
.copy(
module = newModuleName,
selfType = newSelfType,
arguments = method.arguments.map { argument =>
argument.copy(reprType =
if (argument.reprType.startsWith("local.Test."))
newArgumentType
else argument.reprType
)
}
),
suggestion.conversion.copy(
module = newModuleName,
sourceType = "local.Best.Main.Foo",
returnType = "local.Best.Main.Bar"
),
suggestion.function
.copy(module = newModuleName, returnType = newReturnType),
suggestion.local
.copy(module = newModuleName, returnType = newReturnType)
)
}
"change version after renaming the module" taggedAs Retry in withRepo {
repo =>
val action = for {
v1 <- repo.insert(suggestion.constructor)
(v2, _, _, _, _) <- repo.renameProject("Test", "Zest")
} yield (v1, v2)
val (v1, v2) = Await.result(action, Timeout)
v1 should not equal Some(v2)
}
"not change version when not renamed the module" taggedAs Retry in withRepo {
repo =>
val action = for {
v1 <- repo.insert(suggestion.constructor)
(v2, _, _, _, _) <- repo.renameProject("Zest", "Best")
} yield (v1, v2)
val (v1, v2) = Await.result(action, Timeout)
v1 shouldEqual Some(v2)
}
"apply export updates" taggedAs Retry in withRepo { repo => "apply export updates" taggedAs Retry in withRepo { repo =>
val reexport = "Foo.Bar" val reexport = "Foo.Bar"
val method = suggestion.method.copy(reexport = Some(reexport)) val method = suggestion.method.copy(reexport = Some(reexport))