Handle Rename Command Properly (#1041)

In the current workflow, at first, the default Unnamed project is
created, and the Suggestions database is populated with entries from the
Unnamed.* modules. When the user changes the name of the project, we
should update all modules in the Suggestion Database with the new
project name.

This PR implements module renaming in the Suggestions database and fixes
the initialization issues.

- add: search/invalidateSuggestionsDatabase JSON-RPC command that resets
  the corrupted Suggestions database
- update: SuggestionsHandler to rename the modules in the
  SuggestionsDatabase when the project is renamed
- fix: MainModule initialization
This commit is contained in:
Dmitry Bushev 2020-07-29 14:51:00 +03:00 committed by GitHub
parent 93c4453299
commit 2f9e7f51af
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 535 additions and 76 deletions

View File

@ -105,6 +105,7 @@ transport formats, please look [here](./protocol-architecture).
- [Search Operations](#search-operations)
- [Suggestions Database Example](#suggestionsdatabaseexample)
- [`search/getSuggestionsDatabase`](#searchgetsuggestionsdatabase)
- [`search/invalidateSuggestionsDatabase`](#invalidatesuggestionsdatabase)
- [`search/getSuggestionsDatabaseVersion`](#searchgetsuggestionsdatabaseversion)
- [`search/suggestionsDatabaseUpdate`](#searchsuggestionsdatabaseupdate)
- [`search/completion`](#searchcompletion)
@ -2595,7 +2596,7 @@ Sent from client to the server to receive the full suggestions database.
- **Type:** Request
- **Direction:** Client -> Server
- **Connection:** Binary
- **Connection:** Protocol
- **Visibility:** Public
#### Parameters
@ -2622,6 +2623,33 @@ null;
- [`ProjectNotFoundError`](#projectnotfounderror) project is not found in the
root directory
### `search/invalidateSuggestionsDatabase`
Sent from client to the server to clean the suggestions database resetting the
version.
- **Type:** Request
- **Direction:** Client -> Server
- **Connection:** Protocol
- **Visibility:** Public
#### Parameters
```typescript
null;
```
#### Result
```typescript
null;
```
#### Errors
- [`SuggestionsDatabaseError`](#suggestionsdatabaseerror) an error accessing the
suggestions database
### `search/getSuggestionsDatabaseVersion`
Sent from client to the server to receive the current version of the suggestions
@ -2629,7 +2657,7 @@ database.
- **Type:** Request
- **Direction:** Client -> Server
- **Connection:** Binary
- **Connection:** Protocol
- **Visibility:** Public
#### Parameters
@ -2661,7 +2689,7 @@ database.
- **Type:** Notification
- **Direction:** Server -> Client
- **Connection:** Binary
- **Connection:** Protocol
- **Visibility:** Public
#### Parameters
@ -2683,7 +2711,7 @@ Sent from client to the server to receive the autocomplete suggestion.
- **Type:** Request
- **Direction:** Client -> Server
- **Connection:** Binary
- **Connection:** Protocol
- **Visibility:** Public
#### Parameters

View File

@ -101,7 +101,13 @@ class MainModule(serverConfig: LanguageServerConfig) {
lazy val suggestionsHandler =
system.actorOf(
SuggestionsHandler
.props(languageServerConfig, suggestionsRepo, sessionRouter),
.props(
languageServerConfig,
suggestionsRepo,
versionsRepo,
sessionRouter,
runtimeConnector
),
"suggestions-handler"
)
@ -198,7 +204,7 @@ class MainModule(serverConfig: LanguageServerConfig) {
runtimeConnector
)
lazy val jsonRpcServer =
val jsonRpcServer =
new JsonRpcServer(
JsonRpc.protocol,
jsonRpcControllerFactory,
@ -206,7 +212,7 @@ class MainModule(serverConfig: LanguageServerConfig) {
.Config(outgoingBufferSize = 10000, lazyMessageTimeout = 10.seconds)
)
lazy val binaryServer =
val binaryServer =
new BinaryWebSocketServer(
InboundMessageDecoder,
BinaryEncoder.empty,

View File

@ -40,7 +40,8 @@ import org.enso.languageserver.runtime.ExecutionApi._
import org.enso.languageserver.search.SearchApi.{
Completion,
GetSuggestionsDatabase,
GetSuggestionsDatabaseVersion
GetSuggestionsDatabaseVersion,
InvalidateSuggestionsDatabase
}
import org.enso.languageserver.runtime.VisualisationApi.{
AttachVisualisation,
@ -267,6 +268,8 @@ class JsonConnectionController(
.props(requestTimeout, suggestionsHandler),
GetSuggestionsDatabase -> search.GetSuggestionsDatabaseHandler
.props(requestTimeout, suggestionsHandler),
InvalidateSuggestionsDatabase -> search.InvalidateSuggestionsDatabaseHandler
.props(requestTimeout, suggestionsHandler),
Completion -> search.CompletionHandler
.props(requestTimeout, suggestionsHandler),
AttachVisualisation -> AttachVisualisationHandler

View File

@ -57,6 +57,7 @@ object JsonRpc {
.registerRequest(ModifyVisualisation)
.registerRequest(GetSuggestionsDatabase)
.registerRequest(GetSuggestionsDatabaseVersion)
.registerRequest(InvalidateSuggestionsDatabase)
.registerRequest(Completion)
.registerRequest(RenameProject)
.registerNotification(ForceReleaseCapability)

View File

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

View File

@ -34,11 +34,19 @@ class RenameProjectHandler(timeout: FiniteDuration, runtimeConnector: ActorRef)
runtimeConnector ! Api.Request(UUID.randomUUID(), payload)
val cancellable =
context.system.scheduler.scheduleOnce(timeout, self, RequestTimeout)
context.become(responseStage(id, sender(), cancellable))
context.become(
responseStage(
id,
ProjectNameChangedEvent(params.oldName, params.newName),
sender(),
cancellable
)
)
}
private def responseStage(
id: Id,
nameChangedEvent: ProjectNameChangedEvent,
replyTo: ActorRef,
cancellable: Cancellable
): Receive = {
@ -47,8 +55,8 @@ class RenameProjectHandler(timeout: FiniteDuration, runtimeConnector: ActorRef)
replyTo ! ResponseError(Some(id), ServiceError)
context.stop(self)
case Api.Response(_, Api.ProjectRenamed(name)) =>
context.system.eventStream.publish(ProjectNameChangedEvent(name))
case Api.Response(_, Api.ProjectRenamed(_)) =>
context.system.eventStream.publish(nameChangedEvent)
replyTo ! ResponseResult(RenameProject, id, Unused)
cancellable.cancel()
context.stop(self)

View File

@ -0,0 +1,91 @@
package org.enso.languageserver.requesthandler.search
import akka.actor.{Actor, ActorLogging, ActorRef, Cancellable, Props, Status}
import org.enso.jsonrpc.Errors.ServiceError
import org.enso.jsonrpc._
import org.enso.languageserver.requesthandler.RequestTimeout
import org.enso.languageserver.runtime.{
ContextRegistryProtocol,
RuntimeFailureMapper
}
import org.enso.languageserver.search.SearchApi.{
InvalidateSuggestionsDatabase,
SuggestionsDatabaseError
}
import org.enso.languageserver.search.{SearchFailureMapper, SearchProtocol}
import org.enso.languageserver.util.UnhandledLogging
import scala.concurrent.duration.FiniteDuration
/**
* A request handler for `search/invalidateSuggestionsDatabase` command.
*
* @param timeout request timeout
* @param suggestionsHandler a reference to the suggestions handler
*/
class InvalidateSuggestionsDatabaseHandler(
timeout: FiniteDuration,
suggestionsHandler: ActorRef
) extends Actor
with ActorLogging
with UnhandledLogging {
import context.dispatcher
override def receive: Receive = requestStage
private def requestStage: Receive = {
case Request(InvalidateSuggestionsDatabase, id, _) =>
suggestionsHandler ! SearchProtocol.InvalidateSuggestionsDatabase
val cancellable =
context.system.scheduler.scheduleOnce(timeout, self, RequestTimeout)
context.become(responseStage(id, sender(), cancellable))
}
private def responseStage(
id: Id,
replyTo: ActorRef,
cancellable: Cancellable
): Receive = {
case Status.Failure(ex) =>
log.error(ex, "InvalidateSuggestionsDatabase error")
replyTo ! ResponseError(Some(id), SuggestionsDatabaseError)
cancellable.cancel()
context.stop(self)
case RequestTimeout =>
log.error(s"Request $id timed out")
replyTo ! ResponseError(Some(id), ServiceError)
context.stop(self)
case msg: SearchProtocol.SearchFailure =>
replyTo ! ResponseError(Some(id), SearchFailureMapper.mapFailure(msg))
case error: ContextRegistryProtocol.Failure =>
replyTo ! ResponseError(Some(id), RuntimeFailureMapper.mapFailure(error))
cancellable.cancel()
context.stop(self)
case SearchProtocol.InvalidateSuggestionsDatabaseResult =>
replyTo ! ResponseResult(InvalidateSuggestionsDatabase, id, Unused)
cancellable.cancel()
context.stop(self)
}
}
object InvalidateSuggestionsDatabaseHandler {
/**
* Creates configuration object used to create a
* [[InvalidateSuggestionsDatabaseHandler]].
*
* @param timeout request timeout
* @param suggestionsHandler a reference to the suggestions handler
*/
def props(
timeout: FiniteDuration,
suggestionsHandler: ActorRef
): Props =
Props(new InvalidateSuggestionsDatabaseHandler(timeout, suggestionsHandler))
}

View File

@ -60,6 +60,17 @@ object SearchApi {
}
}
case object InvalidateSuggestionsDatabase
extends Method("search/invalidateSuggestionsDatabase") {
implicit val hasParams = new HasParams[this.type] {
type Params = Unused.type
}
implicit val hasResult = new HasResult[this.type] {
type Result = Unused.type
}
}
case object Completion extends Method("search/completion") {
case class Params(

View File

@ -291,6 +291,15 @@ object SearchProtocol {
*/
case class CompletionResult(currentVersion: Long, results: Seq[SuggestionId])
/** The request to invalidate the modules index. */
case object InvalidateModulesIndex
/** The request to invalidate the suggestions database. */
case object InvalidateSuggestionsDatabase
/** The reply to the invalidate request. */
case object InvalidateSuggestionsDatabaseResult
/** Base trait for search request errors. */
sealed trait SearchFailure

View File

@ -18,12 +18,13 @@ import org.enso.languageserver.event.InitializedEvent
import org.enso.languageserver.filemanager.{FileDeletedEvent, Path}
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.search.SearchProtocol._
import org.enso.languageserver.search.handler.InvalidateModulesIndexHandler
import org.enso.languageserver.session.SessionRouter.DeliverToJsonController
import org.enso.languageserver.util.UnhandledLogging
import org.enso.pkg.PackageManager
import org.enso.polyglot.Suggestion
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.searcher.SuggestionsRepo
import org.enso.searcher.{FileVersionsRepo, SuggestionsRepo}
import org.enso.text.editing.model.Position
import scala.concurrent.Future
@ -63,20 +64,26 @@ import scala.util.{Failure, Success}
* }}
*
* @param config the server configuration
* @param repo the suggestions repo
* @param suggestionsRepo the suggestions repo
* @param sessionRouter the session router
* @param runtimeConnector the runtime connector
*/
final class SuggestionsHandler(
config: Config,
repo: SuggestionsRepo[Future],
sessionRouter: ActorRef
suggestionsRepo: SuggestionsRepo[Future],
fileVersionsRepo: FileVersionsRepo[Future],
sessionRouter: ActorRef,
runtimeConnector: ActorRef
) extends Actor
with Stash
with ActorLogging
with UnhandledLogging {
import SuggestionsHandler.ProjectNameUpdated
import context.dispatcher
private val timeout = config.executionContext.requestTimeout
override def preStart(): Unit = {
context.system.eventStream
.subscribe(self, classOf[Api.ExpressionValuesComputed])
@ -93,7 +100,7 @@ final class SuggestionsHandler(
case (_, contentRoot) =>
PackageManager.Default
.fromDirectory(contentRoot)
.foreach(pkg => self ! ProjectNameChangedEvent(pkg.config.name))
.foreach(pkg => self ! ProjectNameUpdated(pkg.config.name))
}
}
@ -101,7 +108,12 @@ final class SuggestionsHandler(
initializing(SuggestionsHandler.Initialization())
def initializing(init: SuggestionsHandler.Initialization): Receive = {
case ProjectNameChangedEvent(name) =>
case ProjectNameChangedEvent(oldName, newName) =>
suggestionsRepo
.renameProject(oldName, newName)
.map(_ => ProjectNameUpdated(newName))
.pipeTo(self)
case ProjectNameUpdated(name) =>
tryInitialize(init.copy(project = Some(name)))
case InitializedEvent.SuggestionsRepoInitialized =>
tryInitialize(
@ -165,7 +177,7 @@ final class SuggestionsHandler(
val types = updates.flatMap(update =>
update.expressionType.map(update.expressionId -> _)
)
repo
suggestionsRepo
.updateAll(types)
.map {
case (version, updatedIds) =>
@ -177,12 +189,12 @@ final class SuggestionsHandler(
}
case GetSuggestionsDatabaseVersion =>
repo.currentVersion
suggestionsRepo.currentVersion
.map(GetSuggestionsDatabaseVersionResult)
.pipeTo(sender())
case GetSuggestionsDatabase =>
repo.getAll
suggestionsRepo.getAll
.map {
case (version, entries) =>
GetSuggestionsDatabaseResult(
@ -193,11 +205,11 @@ final class SuggestionsHandler(
.pipeTo(sender())
case Completion(path, pos, selfType, returnType, tags) =>
getModule(projectName, path)
getModuleName(projectName, path)
.fold(
Future.successful,
module =>
repo
suggestionsRepo
.search(
Some(module),
selfType,
@ -210,11 +222,11 @@ final class SuggestionsHandler(
.pipeTo(sender())
case FileDeletedEvent(path) =>
getModule(projectName, path)
getModuleName(projectName, path)
.fold(
err => Future.successful(Left(err)),
module =>
repo
suggestionsRepo
.removeByModule(module)
.map {
case (version, ids) =>
@ -245,7 +257,23 @@ final class SuggestionsHandler(
)
}
case ProjectNameChangedEvent(name) =>
case InvalidateSuggestionsDatabase =>
val action = for {
_ <- suggestionsRepo.clean
_ <- fileVersionsRepo.clean
} yield SearchProtocol.InvalidateModulesIndex
val handler = context.system
.actorOf(InvalidateModulesIndexHandler.props(timeout, runtimeConnector))
action.pipeTo(handler)(sender())
case ProjectNameChangedEvent(oldName, newName) =>
suggestionsRepo
.renameProject(oldName, newName)
.map(_ => ProjectNameUpdated(newName))
.pipeTo(self)
case ProjectNameUpdated(name) =>
context.become(initialized(name, clients))
}
@ -256,6 +284,7 @@ final class SuggestionsHandler(
*/
private def tryInitialize(state: SuggestionsHandler.Initialization): Unit = {
state.initialized.fold(context.become(initializing(state))) { name =>
log.debug("Initialized")
context.become(initialized(name, Set()))
unstashAll()
}
@ -278,8 +307,8 @@ final class SuggestionsHandler(
): Future[SuggestionsDatabaseUpdateNotification] = {
val added = updates.map(_.suggestion)
for {
(_, removedIds) <- repo.removeByModule(moduleName)
(version, addedIds) <- repo.insertAll(added)
(_, removedIds) <- suggestionsRepo.removeByModule(moduleName)
(version, addedIds) <- suggestionsRepo.insertAll(added)
} yield {
val updatesRemoved = removedIds.map(SuggestionsDatabaseUpdate.Remove)
val updatesAdded = (addedIds zip added).flatMap {
@ -317,8 +346,8 @@ final class SuggestionsHandler(
}
for {
(_, removedIds) <- repo.removeAll(removed)
(version, addedIds) <- repo.insertAll(added)
(_, removedIds) <- suggestionsRepo.removeAll(removed)
(version, addedIds) <- suggestionsRepo.insertAll(added)
} yield {
val updatesRemoved = removedIds.collect {
case Some(id) => SuggestionsDatabaseUpdate.Remove(id)
@ -345,7 +374,7 @@ final class SuggestionsHandler(
* @param path the requested file path
* @return the module name
*/
private def getModule(
private def getModuleName(
projectName: String,
path: Path
): Either[SearchFailure, String] =
@ -363,6 +392,13 @@ final class SuggestionsHandler(
object SuggestionsHandler {
/**
* The notification about the project name update.
*
* @param projectName the new project name
*/
case class ProjectNameUpdated(projectName: String)
/**
* The initialization state of the handler.
*
@ -390,14 +426,26 @@ object SuggestionsHandler {
* Creates a configuration object used to create a [[SuggestionsHandler]].
*
* @param config the server configuration
* @param repo the suggestions repo
* @param suggestionsRepo the suggestions repo
* @param fileVersionsRepo the file versions repo
* @param sessionRouter the session router
* @param runtimeConnector the runtime connector
*/
def props(
config: Config,
repo: SuggestionsRepo[Future],
sessionRouter: ActorRef
suggestionsRepo: SuggestionsRepo[Future],
fileVersionsRepo: FileVersionsRepo[Future],
sessionRouter: ActorRef,
runtimeConnector: ActorRef
): Props =
Props(new SuggestionsHandler(config, repo, sessionRouter))
Props(
new SuggestionsHandler(
config,
suggestionsRepo,
fileVersionsRepo,
sessionRouter,
runtimeConnector
)
)
}

View File

@ -0,0 +1,72 @@
package org.enso.languageserver.search.handler
import java.util.UUID
import akka.actor.{Actor, ActorLogging, ActorRef, Cancellable, Props}
import org.enso.languageserver.requesthandler.RequestTimeout
import org.enso.languageserver.runtime.RuntimeFailureMapper
import org.enso.languageserver.search.SearchProtocol
import org.enso.languageserver.util.UnhandledLogging
import org.enso.polyglot.runtime.Runtime.Api
import scala.concurrent.duration.FiniteDuration
/**
* A request handler for invalidate modules index command.
*
* @param timeout request timeout
* @param runtime reference to the runtime connector
*/
final class InvalidateModulesIndexHandler(
timeout: FiniteDuration,
runtime: ActorRef
) extends Actor
with ActorLogging
with UnhandledLogging {
import context.dispatcher
override def receive: Receive = requestStage
private def requestStage: Receive = {
case SearchProtocol.InvalidateModulesIndex =>
runtime ! Api.Request(
UUID.randomUUID(),
Api.InvalidateModulesIndexRequest()
)
val cancellable =
context.system.scheduler.scheduleOnce(timeout, self, RequestTimeout)
context.become(responseStage(sender(), cancellable))
}
private def responseStage(
replyTo: ActorRef,
cancellable: Cancellable
): Receive = {
case RequestTimeout =>
replyTo ! RequestTimeout
context.stop(self)
case Api.Response(_, Api.InvalidateModulesIndexResponse()) =>
replyTo ! SearchProtocol.InvalidateSuggestionsDatabaseResult
cancellable.cancel()
context.stop(self)
case Api.Response(_, error: Api.Error) =>
replyTo ! RuntimeFailureMapper.mapApiError(error)
cancellable.cancel()
context.stop(self)
}
}
object InvalidateModulesIndexHandler {
/**
* Creates a configuration object used to create [[InvalidateModulesIndexHandler]].
*
* @param timeout request timeout
* @param runtime reference to the runtime conector
*/
def props(timeout: FiniteDuration, runtime: ActorRef): Props =
Props(new InvalidateModulesIndexHandler(timeout, runtime))
}

View File

@ -14,13 +14,12 @@ import org.enso.languageserver.capability.CapabilityProtocol.{
import org.enso.languageserver.data._
import org.enso.languageserver.event.InitializedEvent
import org.enso.languageserver.filemanager.Path
import org.enso.languageserver.refactoring.ProjectNameChangedEvent
import org.enso.languageserver.search.SearchProtocol.SuggestionDatabaseEntry
import org.enso.languageserver.session.JsonSession
import org.enso.languageserver.session.SessionRouter.DeliverToJsonController
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.searcher.SuggestionsRepo
import org.enso.searcher.sql.SqlSuggestionsRepo
import org.enso.searcher.{FileVersionsRepo, SuggestionsRepo}
import org.enso.searcher.sql.{SqlDatabase, SqlSuggestionsRepo, SqlVersionsRepo}
import org.enso.testkit.RetrySpec
import org.enso.text.editing.model.Position
import org.scalatest.BeforeAndAfterAll
@ -50,7 +49,7 @@ class SuggestionsHandlerSpec
"SuggestionsHandler" should {
"subscribe to notification updates" taggedAs Retry in withDb {
(_, _, _, handler) =>
(_, _, _, _, handler) =>
val clientId = UUID.randomUUID()
handler ! AcquireCapability(
@ -61,7 +60,7 @@ class SuggestionsHandlerSpec
}
"receive runtime updates" taggedAs Retry in withDb {
(_, repo, router, handler) =>
(_, repo, router, _, handler) =>
val clientId = UUID.randomUUID()
// acquire capability
@ -95,7 +94,7 @@ class SuggestionsHandlerSpec
}
"apply runtime updates in correct order" taggedAs Retry in withDb {
(_, repo, router, handler) =>
(_, repo, router, _, handler) =>
val clientId = UUID.randomUUID()
// acquire capability
@ -128,14 +127,14 @@ class SuggestionsHandlerSpec
}
"get initial suggestions database version" taggedAs Retry in withDb {
(_, _, _, handler) =>
(_, _, _, _, handler) =>
handler ! SearchProtocol.GetSuggestionsDatabaseVersion
expectMsg(SearchProtocol.GetSuggestionsDatabaseVersionResult(0))
}
"get suggestions database version" taggedAs Retry in withDb {
(_, repo, _, handler) =>
(_, repo, _, _, handler) =>
Await.ready(repo.insert(Suggestions.atom), Timeout)
handler ! SearchProtocol.GetSuggestionsDatabaseVersion
@ -143,14 +142,14 @@ class SuggestionsHandlerSpec
}
"get initial suggestions database" taggedAs Retry in withDb {
(_, _, _, handler) =>
(_, _, _, _, handler) =>
handler ! SearchProtocol.GetSuggestionsDatabase
expectMsg(SearchProtocol.GetSuggestionsDatabaseResult(0, Seq()))
}
"get suggestions database" taggedAs Retry in withDb {
(_, repo, _, handler) =>
(_, repo, _, _, handler) =>
Await.ready(repo.insert(Suggestions.atom), Timeout)
handler ! SearchProtocol.GetSuggestionsDatabase
@ -162,8 +161,23 @@ class SuggestionsHandlerSpec
)
}
"invalidate suggestions database" taggedAs Retry in withDb {
(_, repo, _, connector, handler) =>
Await.ready(repo.insert(Suggestions.atom), Timeout)
handler ! SearchProtocol.InvalidateSuggestionsDatabase
connector.expectMsgClass(classOf[Api.Request]) match {
case Api.Request(_, Api.InvalidateModulesIndexRequest()) =>
case Api.Request(_, msg) =>
fail(s"Runtime connector receive unexpected message: $msg")
}
connector.reply(Api.Response(Api.InvalidateModulesIndexResponse()))
expectMsg(SearchProtocol.InvalidateSuggestionsDatabaseResult)
}
"search entries by empty search query" taggedAs Retry in withDb {
(config, repo, _, handler) =>
(config, repo, _, _, handler) =>
Await.ready(repo.insertAll(Suggestions.all), Timeout)
handler ! SearchProtocol.Completion(
file = mkModulePath(config, "Foo", "Main.enso"),
@ -177,7 +191,7 @@ class SuggestionsHandlerSpec
}
"search entries by self type" taggedAs Retry in withDb {
(config, repo, _, handler) =>
(config, repo, _, _, handler) =>
val (_, Seq(_, methodId, _, _)) =
Await.result(repo.insertAll(Suggestions.all), Timeout)
handler ! SearchProtocol.Completion(
@ -192,7 +206,7 @@ class SuggestionsHandlerSpec
}
"search entries by return type" taggedAs Retry in withDb {
(config, repo, _, handler) =>
(config, repo, _, _, handler) =>
val (_, Seq(_, _, functionId, _)) =
Await.result(repo.insertAll(Suggestions.all), Timeout)
handler ! SearchProtocol.Completion(
@ -207,7 +221,7 @@ class SuggestionsHandlerSpec
}
"search entries by tags" taggedAs Retry in withDb {
(config, repo, _, handler) =>
(config, repo, _, _, handler) =>
val (_, Seq(_, _, _, localId)) =
Await.result(repo.insertAll(Suggestions.all), Timeout)
handler ! SearchProtocol.Completion(
@ -225,11 +239,21 @@ class SuggestionsHandlerSpec
def newSuggestionsHandler(
config: Config,
sessionRouter: TestProbe,
repo: SuggestionsRepo[Future]
runtimeConnector: TestProbe,
suggestionsRepo: SuggestionsRepo[Future],
fileVersionsRepo: FileVersionsRepo[Future]
): ActorRef = {
val handler =
system.actorOf(SuggestionsHandler.props(config, repo, sessionRouter.ref))
handler ! ProjectNameChangedEvent("Test")
system.actorOf(
SuggestionsHandler.props(
config,
suggestionsRepo,
fileVersionsRepo,
sessionRouter.ref,
runtimeConnector.ref
)
)
handler ! SuggestionsHandler.ProjectNameUpdated("Test")
handler
}
@ -252,25 +276,46 @@ class SuggestionsHandlerSpec
JsonSession(clientId, TestProbe().ref)
def withDb(
test: (Config, SuggestionsRepo[Future], TestProbe, ActorRef) => Any
test: (
Config,
SuggestionsRepo[Future],
TestProbe,
TestProbe,
ActorRef
) => Any
): Unit = {
val testContentRoot = Files.createTempDirectory(null).toRealPath()
sys.addShutdownHook(FileUtils.deleteQuietly(testContentRoot.toFile))
val config = newConfig(testContentRoot.toFile)
val router = TestProbe("session-router")
val repo = SqlSuggestionsRepo(config.directories.suggestionsDatabaseFile)
val handler = newSuggestionsHandler(config, router, repo)
repo.init.onComplete {
val config = newConfig(testContentRoot.toFile)
val router = TestProbe("session-router")
val connector = TestProbe("runtime-connector")
val sqlDatabase = SqlDatabase(config.directories.suggestionsDatabaseFile)
val suggestionsRepo = new SqlSuggestionsRepo(sqlDatabase)
val versionsRepo = new SqlVersionsRepo(sqlDatabase)
val handler = newSuggestionsHandler(
config,
router,
connector,
suggestionsRepo,
versionsRepo
)
suggestionsRepo.init.onComplete {
case Success(()) =>
system.eventStream.publish(InitializedEvent.SuggestionsRepoInitialized)
case Failure(ex) =>
system.log.error(ex, "Failed to initialize Suggestions repo")
}
versionsRepo.init.onComplete {
case Success(()) =>
system.eventStream.publish(InitializedEvent.FileVersionsRepoInitialized)
case Failure(ex) =>
system.log.error(ex, "Failed to initialize FileVersions repo")
}
try test(config, repo, router, handler)
try test(config, suggestionsRepo, router, connector, handler)
finally {
system.stop(handler)
repo.close()
sqlDatabase.close()
}
}

View File

@ -114,7 +114,13 @@ class BaseServerTest extends JsonRpcServerTestKit {
val suggestionsHandler =
system.actorOf(
SuggestionsHandler.props(config, suggestionsRepo, sessionRouter)
SuggestionsHandler.props(
config,
suggestionsRepo,
versionsRepo,
sessionRouter,
runtimeConnectorProbe.ref
)
)
val capabilityRouter =

View File

@ -13,7 +13,7 @@ class SuggestionsHandlerEventsTest extends BaseServerTest with FlakySpec {
"send suggestions database notifications" taggedAs Flaky in {
val client = getInitialisedWsClient()
system.eventStream.publish(ProjectNameChangedEvent("Test"))
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test"))
client.send(json.acquireSuggestionsDatabaseUpdatesCapability(0))
client.expectJson(json.ok(0))

View File

@ -19,7 +19,7 @@ class SuggestionsHandlerTest extends BaseServerTest with FlakySpec {
"get initial suggestions database version" in {
val client = getInitialisedWsClient()
system.eventStream.publish(ProjectNameChangedEvent("Test"))
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test"))
client.send(json.getSuggestionsDatabaseVersion(0))
client.expectJson(json"""
@ -34,7 +34,7 @@ class SuggestionsHandlerTest extends BaseServerTest with FlakySpec {
"get initial suggestions database" taggedAs Flaky in {
val client = getInitialisedWsClient()
system.eventStream.publish(ProjectNameChangedEvent("Test"))
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test"))
client.send(json.getSuggestionsDatabase(0))
client.expectJson(json"""
@ -51,7 +51,7 @@ class SuggestionsHandlerTest extends BaseServerTest with FlakySpec {
"reply to completion request" taggedAs Flaky in {
val client = getInitialisedWsClient()
system.eventStream.publish(ProjectNameChangedEvent("Test"))
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test"))
client.send(json"""
{ "jsonrpc": "2.0",
@ -110,7 +110,7 @@ class SuggestionsHandlerTest extends BaseServerTest with FlakySpec {
"reply with error when project root not found" taggedAs Flaky in {
val client = getInitialisedWsClient()
system.eventStream.publish(ProjectNameChangedEvent("Test"))
system.eventStream.publish(ProjectNameChangedEvent("Test", "Test"))
client.send(json"""
{ "jsonrpc": "2.0",

View File

@ -167,6 +167,14 @@ object Runtime {
new JsonSubTypes.Type(
value = classOf[Api.SuggestionsDatabaseReIndexNotification],
name = "suggestionsDatabaseReindexNotification"
),
new JsonSubTypes.Type(
value = classOf[Api.InvalidateModulesIndexRequest],
name = "invalidateModulesIndexRequest"
),
new JsonSubTypes.Type(
value = classOf[Api.InvalidateModulesIndexResponse],
name = "invalidateModulesIndexResponse"
)
)
)
@ -711,6 +719,12 @@ object Runtime {
mapper.registerModule(DefaultScalaModule)
}
/** A request to invalidate the indexed flag of the modules. */
case class InvalidateModulesIndexRequest() extends ApiRequest
/** Signals that the module indexes has been invalidated. */
case class InvalidateModulesIndexResponse() extends ApiResponse
/**
* Serializes a Request into a byte buffer.
*

View File

@ -9,9 +9,9 @@ import com.oracle.truffle.api.interop.*;
import com.oracle.truffle.api.library.ExportLibrary;
import com.oracle.truffle.api.library.ExportMessage;
import org.enso.interpreter.Language;
import org.enso.interpreter.runtime.builtin.Builtins;
import org.enso.interpreter.runtime.Context;
import org.enso.interpreter.runtime.Module;
import org.enso.interpreter.runtime.builtin.Builtins;
import org.enso.interpreter.runtime.data.Vector;
import org.enso.interpreter.runtime.type.Types;
import org.enso.pkg.QualifiedName;
@ -19,9 +19,7 @@ import org.enso.pkg.QualifiedName$;
import org.enso.polyglot.MethodNames;
import java.io.File;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.*;
import java.util.stream.Collectors;
/** Represents the top scope of Enso execution, containing all the importable modules. */
@ -51,6 +49,11 @@ public class TopLevelScope implements TruffleObject {
return scope;
}
/** @return the list of modules in the scope. */
public Collection<Module> getModules() {
return modules.values();
}
/**
* Looks up a module by name.
*

View File

@ -45,6 +45,10 @@ object CommandFactory {
case payload: Api.OpenFileNotification => new OpenFileCmd(payload)
case payload: Api.CloseFileNotification => new CloseFileCmd(payload)
case payload: Api.EditFileNotification => new EditFileCmd(payload)
case payload: Api.InvalidateModulesIndexRequest =>
new InvalidateModulesIndexCmd(request.requestId, payload)
case Api.ShutDownRuntimeServer() =>
throw new IllegalArgumentException(
"ShutDownRuntimeServer request is not convertible to command object"

View File

@ -0,0 +1,38 @@
package org.enso.interpreter.instrument.command
import org.enso.interpreter.instrument.execution.RuntimeContext
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.polyglot.runtime.Runtime.Api.RequestId
import scala.concurrent.{ExecutionContext, Future}
/**
* A command that invalidates the modules index.
*
* @param maybeRequestId an option with request id
* @param request a request for invalidation
*/
class InvalidateModulesIndexCmd(
maybeRequestId: Option[RequestId],
val request: Api.InvalidateModulesIndexRequest
) extends Command(maybeRequestId) {
/**
* Executes a request.
*
* @param ctx contains suppliers of services to perform a request
*/
override def execute(implicit
ctx: RuntimeContext,
ec: ExecutionContext
): Future[Unit] = {
for {
_ <- Future { ctx.jobControlPlane.abortAllJobs() }
} yield {
ctx.executionService.getContext.getTopScope.getModules
.forEach(_.setIndexed(false))
reply(Api.InvalidateModulesIndexResponse())
}
}
}

View File

@ -25,4 +25,7 @@ trait FileVersionsRepo[F[_]] {
* @param file the file path
*/
def remove(file: File): F[Unit]
/** Clean the repo. */
def clean: F[Unit]
}

View File

@ -88,4 +88,14 @@ trait SuggestionsRepo[F[_]] {
def updateAll(
expressions: Seq[(Suggestion.ExternalId, String)]
): F[(Long, Seq[Option[Long]])]
/** Cleans the repo resetting the version. */
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
*/
def renameProject(oldName: String, newName: String): F[Unit]
}

View File

@ -31,8 +31,8 @@ final class SqlSuggestionsRepo(db: SqlDatabase)(implicit ec: ExecutionContext)
def init: Future[Unit] =
db.run(initQuery)
/** Clean the repo. */
def clean: Future[Unit] =
/** @inheritdoc */
override def clean: Future[Unit] =
db.run(cleanQuery)
/** @inheritdoc */
@ -89,6 +89,10 @@ final class SqlSuggestionsRepo(db: SqlDatabase)(implicit ec: ExecutionContext)
): Future[(Long, Seq[Option[Long]])] =
db.run(updateAllQuery(expressions))
/** @inheritdoc */
override def renameProject(oldName: String, newName: String): Future[Unit] =
db.run(renameProjectQuery(oldName, newName))
/** @inheritdoc */
override def currentVersion: Future[Long] =
db.run(currentVersionQuery)
@ -116,12 +120,13 @@ final class SqlSuggestionsRepo(db: SqlDatabase)(implicit ec: ExecutionContext)
}
/** The query to clean the repo. */
private def cleanQuery: DBIO[Unit] =
private def cleanQuery: DBIO[Unit] = {
for {
_ <- Suggestions.delete
_ <- Arguments.delete
_ <- SuggestionsVersions.delete
} yield ()
}
/** Get all suggestions.
*
@ -342,6 +347,22 @@ final class SqlSuggestionsRepo(db: SqlDatabase)(implicit ec: ExecutionContext)
query.transactionally
}
/** The query to update the project name.
*
* @param oldName the old name of the project
* @param newName the new project name
*/
private def renameProjectQuery(
oldName: String,
newName: String
): DBIO[Unit] = {
val updateQuery =
sqlu"""update suggestions
set module = replace(module, $oldName, $newName)
where module like '#$oldName%'"""
updateQuery >> DBIO.successful(())
}
/** The query to get current version of the repo. */
private def currentVersionQuery: DBIO[Long] = {
for {

View File

@ -29,8 +29,8 @@ final class SqlVersionsRepo(db: SqlDatabase)(implicit ec: ExecutionContext)
override def remove(file: File): Future[Unit] =
db.run(removeQuery(file))
/** Clean the database. */
def clean: Future[Unit] =
/** @inheritdoc */
override def clean: Future[Unit] =
db.run(cleanQuery)
/** Close the database. */

View File

@ -367,6 +367,42 @@ class SuggestionsRepoTest extends AnyWordSpec with Matchers with RetrySpec {
v1 shouldEqual v2
}
"rename the module" taggedAs Retry in withRepo { repo =>
val newModuleName = "Best.Main"
val action = for {
_ <- repo.insertAll(
Seq(
suggestion.atom,
suggestion.method,
suggestion.function,
suggestion.local
)
)
_ <- repo.renameProject("Test", "Best")
(_, res) <- repo.getAll
} yield res
val res = Await.result(action, Timeout)
res.map(_.suggestion) should contain theSameElementsAs Seq(
suggestion.atom.copy(module = newModuleName),
suggestion.method.copy(module = newModuleName),
suggestion.function.copy(module = newModuleName),
suggestion.local.copy(module = newModuleName)
)
}
"not change version after renaming the module" taggedAs Retry in withRepo {
repo =>
val action = for {
v1 <- repo.insert(suggestion.atom)
_ <- repo.renameProject("Test", "Zest")
v2 <- repo.currentVersion
} yield (v1, v2)
val (v1, v2) = Await.result(action, Timeout)
v1 shouldEqual Some(v2)
}
"search suggestion by empty query" taggedAs Retry in withRepo { repo =>
val action = for {
_ <- repo.insert(suggestion.atom)