Initialize suggestions database only once (#8116)

close #8033

Changelog:
- update: run language server initialization once
- fix: issues with async `getSuggestionDatabase` message handling in new IDE
- update: implement unique background jobs
- refactor: initialization logic to Java
- refactor: `UniqueJob` to a marker interface
This commit is contained in:
Dmitry Bushev 2023-10-21 21:32:13 +01:00 committed by GitHub
parent 8fc720a1a2
commit b1df8b1889
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
45 changed files with 871 additions and 714 deletions

View File

@ -0,0 +1,37 @@
package org.enso.languageserver.boot.resource;
import java.util.Arrays;
import java.util.concurrent.CompletableFuture;
/** Component that initializes resources in parallel. */
public class AsyncResourcesInitialization implements InitializationComponent {
private final InitializationComponent[] resources;
/**
* Create async initialization component.
*
* @param resources the list of resources to initialize
*/
public AsyncResourcesInitialization(InitializationComponent... resources) {
this.resources = resources;
}
@Override
public boolean isInitialized() {
return Arrays.stream(resources).allMatch(InitializationComponent::isInitialized);
}
@Override
public CompletableFuture<Void> init() {
return CompletableFuture.allOf(
Arrays.stream(resources)
.map(
component ->
component.isInitialized()
? CompletableFuture.completedFuture(null)
: component.init())
.toArray(CompletableFuture<?>[]::new))
.thenRun(() -> {});
}
}

View File

@ -0,0 +1,35 @@
package org.enso.languageserver.boot.resource;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Semaphore;
/** Initialization component ensuring that only one initialization sequence is running at a time. */
public final class BlockingInitialization implements InitializationComponent {
private final InitializationComponent component;
private final Semaphore lock = new Semaphore(1);
/**
* Create blocking initialization component.
*
* @param component the underlying initialization component to run
*/
public BlockingInitialization(InitializationComponent component) {
this.component = component;
}
@Override
public boolean isInitialized() {
return component.isInitialized();
}
@Override
public CompletableFuture<Void> init() {
try {
lock.acquire();
} catch (InterruptedException e) {
return CompletableFuture.failedFuture(e);
}
return component.init().whenComplete((res, err) -> lock.release());
}
}

View File

@ -0,0 +1,46 @@
package org.enso.languageserver.boot.resource;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import org.enso.languageserver.data.ProjectDirectoriesConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Directories initialization. */
public class DirectoriesInitialization implements InitializationComponent {
private final Executor executor;
private final ProjectDirectoriesConfig projectDirectoriesConfig;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private volatile boolean isInitialized = false;
/**
* Creates the directories initialization component.
*
* @param executor the executor that runs the initialization
* @param projectDirectoriesConfig the directories config
*/
public DirectoriesInitialization(
Executor executor, ProjectDirectoriesConfig projectDirectoriesConfig) {
this.executor = executor;
this.projectDirectoriesConfig = projectDirectoriesConfig;
}
@Override
public boolean isInitialized() {
return isInitialized;
}
@Override
public CompletableFuture<Void> init() {
return CompletableFuture.runAsync(
() -> {
logger.info("Initializing directories...");
projectDirectoriesConfig.createDirectories();
logger.info("Initialized directories.");
isInitialized = true;
},
executor);
}
}

View File

@ -0,0 +1,13 @@
package org.enso.languageserver.boot.resource;
import java.util.concurrent.CompletableFuture;
/** A component that should be initialized. */
public interface InitializationComponent {
/** @return `true` if the component is initialized */
boolean isInitialized();
/** Initialize the component. */
CompletableFuture<Void> init();
}

View File

@ -0,0 +1,19 @@
package org.enso.languageserver.boot.resource;
/** Object indicating that the initialization is complete. */
public final class InitializationComponentInitialized {
private static final class InstanceHolder {
private static final InitializationComponentInitialized INSTANCE =
new InitializationComponentInitialized();
}
/**
* Get the initialized marker object.
*
* @return the instance of {@link InitializationComponentInitialized}.
*/
public static InitializationComponentInitialized getInstance() {
return InstanceHolder.INSTANCE;
}
}

View File

@ -0,0 +1,45 @@
package org.enso.languageserver.boot.resource;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import org.enso.jsonrpc.ProtocolFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Initialization of JSON-RPC protocol. */
public class JsonRpcInitialization implements InitializationComponent {
private final Executor executor;
private final ProtocolFactory protocolFactory;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private volatile boolean isInitialized = false;
/**
* Create an instance of JSON-RPC initialization component.
*
* @param executor the executor that runs the initialization
* @param protocolFactory the JSON-RPC protocol factory
*/
public JsonRpcInitialization(Executor executor, ProtocolFactory protocolFactory) {
this.executor = executor;
this.protocolFactory = protocolFactory;
}
@Override
public boolean isInitialized() {
return isInitialized;
}
@Override
public CompletableFuture<Void> init() {
return CompletableFuture.runAsync(
() -> {
logger.info("Initializing JSON-RPC protocol.");
protocolFactory.init();
logger.info("JSON-RPC protocol initialized.");
isInitialized = true;
},
executor);
}
}

View File

@ -0,0 +1,176 @@
package org.enso.languageserver.boot.resource;
import akka.event.EventStream;
import java.io.IOException;
import java.nio.file.FileSystemException;
import java.nio.file.Files;
import java.nio.file.NoSuchFileException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.CompletionStage;
import java.util.concurrent.Executor;
import org.apache.commons.io.FileUtils;
import org.enso.languageserver.data.ProjectDirectoriesConfig;
import org.enso.languageserver.event.InitializedEvent;
import org.enso.logger.masking.MaskedPath;
import org.enso.searcher.sql.SqlDatabase;
import org.enso.searcher.sql.SqlSuggestionsRepo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.jdk.javaapi.FutureConverters;
/** Initialization of the Language Server suggestions database. */
public class RepoInitialization implements InitializationComponent {
private static final int MAX_RETRIES = 3;
private static final long RETRY_DELAY_MILLIS = 1000;
private final Executor executor;
private final ProjectDirectoriesConfig projectDirectoriesConfig;
private final EventStream eventStream;
private final SqlDatabase sqlDatabase;
private final SqlSuggestionsRepo sqlSuggestionsRepo;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private volatile boolean isInitialized = false;
/**
* Create an instance of repo initialization component.
*
* @param executor the executor that runs the initialization
* @param projectDirectoriesConfig configuration of language server directories
* @param eventStream the events stream
* @param sqlDatabase the sql database
* @param sqlSuggestionsRepo the suggestions repo
*/
public RepoInitialization(
Executor executor,
ProjectDirectoriesConfig projectDirectoriesConfig,
EventStream eventStream,
SqlDatabase sqlDatabase,
SqlSuggestionsRepo sqlSuggestionsRepo) {
this.executor = executor;
this.projectDirectoriesConfig = projectDirectoriesConfig;
this.eventStream = eventStream;
this.sqlDatabase = sqlDatabase;
this.sqlSuggestionsRepo = sqlSuggestionsRepo;
}
@Override
public boolean isInitialized() {
return isInitialized;
}
@Override
public CompletableFuture<Void> init() {
return initSqlDatabase()
.thenComposeAsync(v -> initSuggestionsRepo(), executor)
.thenRun(() -> isInitialized = true);
}
private CompletableFuture<Void> initSqlDatabase() {
return CompletableFuture.runAsync(
() -> {
logger.info("Initializing sql database [{}]...", sqlDatabase);
sqlDatabase.open();
logger.info("Initialized sql database [{}].", sqlDatabase);
},
executor)
.whenCompleteAsync(
(res, err) -> {
if (err != null) {
logger.error("Failed to initialize sql database [{}].", sqlDatabase, err);
}
},
executor);
}
private CompletableFuture<Void> initSuggestionsRepo() {
return CompletableFuture.runAsync(
() -> logger.info("Initializing suggestions repo [{}]...", sqlDatabase), executor)
.thenComposeAsync(
v ->
doInitSuggestionsRepo().exceptionallyComposeAsync(this::recoverInitializationError),
executor)
.thenRunAsync(
() -> logger.info("Initialized Suggestions repo [{}].", sqlDatabase), executor)
.whenCompleteAsync(
(res, err) -> {
if (err != null) {
logger.error("Failed to initialize SQL suggestions repo [{}].", sqlDatabase, err);
} else {
eventStream.publish(InitializedEvent.SuggestionsRepoInitialized$.MODULE$);
}
});
}
private CompletableFuture<Void> recoverInitializationError(Throwable error) {
return CompletableFuture.runAsync(
() ->
logger.warn(
"Failed to initialize the suggestions database [{}].", sqlDatabase, error),
executor)
.thenRunAsync(sqlDatabase::close, executor)
.thenComposeAsync(v -> clearDatabaseFile(0), executor)
.thenRunAsync(sqlDatabase::open, executor)
.thenRunAsync(() -> logger.info("Retrying database initialization."), executor)
.thenComposeAsync(v -> doInitSuggestionsRepo(), executor);
}
private CompletableFuture<Void> clearDatabaseFile(int retries) {
return CompletableFuture.runAsync(
() -> {
logger.info("Clear database file. Attempt #{}.", retries + 1);
try {
Files.delete(projectDirectoriesConfig.suggestionsDatabaseFile().toPath());
} catch (IOException e) {
throw new CompletionException(e);
}
},
executor)
.exceptionallyComposeAsync(error -> recoverClearDatabaseFile(error, retries), executor);
}
private CompletableFuture<Void> recoverClearDatabaseFile(Throwable error, int retries) {
if (error instanceof CompletionException) {
return recoverClearDatabaseFile(error.getCause(), retries);
} else if (error instanceof NoSuchFileException) {
logger.warn(
"Failed to delete the database file. Attempt #{}. File does not exist [{}].",
retries + 1,
new MaskedPath(projectDirectoriesConfig.suggestionsDatabaseFile().toPath()));
return CompletableFuture.completedFuture(null);
} else if (error instanceof FileSystemException) {
logger.error(
"Failed to delete the database file. Attempt #{}. The file will be removed during the shutdown.",
retries + 1,
error);
Runtime.getRuntime()
.addShutdownHook(
new Thread(
() ->
FileUtils.deleteQuietly(projectDirectoriesConfig.suggestionsDatabaseFile())));
return CompletableFuture.failedFuture(error);
} else if (error instanceof IOException) {
logger.error("Failed to delete the database file. Attempt #{}.", retries + 1, error);
if (retries < MAX_RETRIES) {
try {
Thread.sleep(RETRY_DELAY_MILLIS);
} catch (InterruptedException e) {
throw new CompletionException(e);
}
return clearDatabaseFile(retries + 1);
} else {
return CompletableFuture.failedFuture(error);
}
}
return CompletableFuture.completedFuture(null);
}
private CompletionStage<Void> doInitSuggestionsRepo() {
return FutureConverters.asJava(sqlSuggestionsRepo.init()).thenAccept(res -> {});
}
}

View File

@ -0,0 +1,46 @@
package org.enso.languageserver.boot.resource;
import java.util.Arrays;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
/** Initializes resources in sequence. */
public class SequentialResourcesInitialization implements InitializationComponent {
private final InitializationComponent[] resources;
private final Executor executor;
/**
* Create an instance of sequential initialization component.
*
* @param executor the executor that runs the initialization
* @param resources the list of resources to initialize
*/
public SequentialResourcesInitialization(
Executor executor, InitializationComponent... resources) {
this.resources = resources;
this.executor = executor;
}
@Override
public boolean isInitialized() {
return Arrays.stream(resources).allMatch(InitializationComponent::isInitialized);
}
@Override
public CompletableFuture<Void> init() {
CompletableFuture<Void> result = CompletableFuture.completedFuture(null);
for (InitializationComponent component : resources) {
result =
result.thenComposeAsync(
res ->
component.isInitialized()
? CompletableFuture.completedFuture(null)
: component.init(),
executor);
}
return result;
}
}

View File

@ -0,0 +1,54 @@
package org.enso.languageserver.boot.resource;
import akka.event.EventStream;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import org.enso.languageserver.event.InitializedEvent;
import org.enso.polyglot.LanguageInfo;
import org.graalvm.polyglot.Context;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Initialize the Truffle context. */
public class TruffleContextInitialization implements InitializationComponent {
private final Executor executor;
private final Context truffleContext;
private final EventStream eventStream;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private volatile boolean isInitialized = false;
/**
* Creates an instance of Truffle initialization component.
*
* @param executor the executor that runs the initialization
* @param eventStream the events stream
* @param truffleContext the Truffle context
*/
public TruffleContextInitialization(
Executor executor, Context truffleContext, EventStream eventStream) {
this.executor = executor;
this.truffleContext = truffleContext;
this.eventStream = eventStream;
}
@Override
public boolean isInitialized() {
return isInitialized;
}
@Override
public CompletableFuture<Void> init() {
return CompletableFuture.runAsync(
() -> {
logger.info("Initializing Runtime context [{}]...", truffleContext);
truffleContext.initialize(LanguageInfo.ID);
eventStream.publish(InitializedEvent.TruffleContextInitialized$.MODULE$);
logger.info("Initialized Runtime context [{}].", truffleContext);
isInitialized = true;
},
executor);
}
}

View File

@ -0,0 +1,51 @@
package org.enso.languageserver.boot.resource;
import akka.event.EventStream;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Executor;
import org.enso.languageserver.effect.Runtime;
import org.enso.languageserver.event.InitializedEvent;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Initialization of ZIO runtime. */
public class ZioRuntimeInitialization implements InitializationComponent {
private final Executor executor;
private final Runtime runtime;
private final EventStream eventStream;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private volatile boolean isInitialized = false;
/**
* Create an instance of ZIO runtime initialization component.
*
* @param executor the executor that runs the initialization
* @param runtime the runtime to initialize
* @param eventStream the events stream
*/
public ZioRuntimeInitialization(Executor executor, Runtime runtime, EventStream eventStream) {
this.executor = executor;
this.runtime = runtime;
this.eventStream = eventStream;
}
@Override
public boolean isInitialized() {
return isInitialized;
}
@Override
public CompletableFuture<Void> init() {
return CompletableFuture.runAsync(
() -> {
logger.info("Initializing ZIO runtime...");
runtime.init();
logger.info("ZIO runtime initialized [{}].", runtime);
isInitialized = true;
eventStream.publish(InitializedEvent.ZioRuntimeInitialized$.MODULE$);
},
executor);
}
}

View File

@ -4,9 +4,10 @@ import akka.event.EventStream
import org.enso.jsonrpc.ProtocolFactory
import org.enso.languageserver.boot.resource.{
AsyncResourcesInitialization,
BlockingInitialization,
DirectoriesInitialization,
InitializationComponent,
JsonRpcInitializationComponent,
JsonRpcInitialization,
RepoInitialization,
SequentialResourcesInitialization,
TruffleContextInitialization,
@ -17,7 +18,7 @@ import org.enso.languageserver.effect
import org.enso.searcher.sql.{SqlDatabase, SqlSuggestionsRepo}
import org.graalvm.polyglot.Context
import scala.concurrent.ExecutionContext
import scala.concurrent.ExecutionContextExecutor
/** Helper object for the initialization of the Language Server resources.
* Creates the directories, initializes the databases, and the Truffle context.
@ -43,19 +44,23 @@ object ResourcesInitialization {
suggestionsRepo: SqlSuggestionsRepo,
truffleContext: Context,
runtime: effect.Runtime
)(implicit ec: ExecutionContext): InitializationComponent = {
SequentialResourcesInitialization(
new DirectoriesInitialization(directoriesConfig),
AsyncResourcesInitialization(
new JsonRpcInitializationComponent(protocolFactory),
new ZioRuntimeInitialization(runtime, eventStream),
new RepoInitialization(
directoriesConfig,
eventStream,
sqlDatabase,
suggestionsRepo
),
new TruffleContextInitialization(eventStream, truffleContext)
)(implicit ec: ExecutionContextExecutor): InitializationComponent = {
new BlockingInitialization(
new SequentialResourcesInitialization(
ec,
new DirectoriesInitialization(ec, directoriesConfig),
new AsyncResourcesInitialization(
new JsonRpcInitialization(ec, protocolFactory),
new ZioRuntimeInitialization(ec, runtime, eventStream),
new RepoInitialization(
ec,
directoriesConfig,
eventStream,
sqlDatabase,
suggestionsRepo
),
new TruffleContextInitialization(ec, truffleContext, eventStream)
)
)
)
}

View File

@ -1,34 +0,0 @@
package org.enso.languageserver.boot.resource
import scala.concurrent.{ExecutionContext, Future}
/** Initializes resources in parallel.
*
* @param resources the list of resources to initialize
* @param ec the execution context
*/
class AsyncResourcesInitialization(
resources: Iterable[InitializationComponent]
)(implicit ec: ExecutionContext)
extends InitializationComponent {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
Future
.traverse(resources)(_.init())
.map { _ => InitializationComponent.Initialized }
}
object AsyncResourcesInitialization {
/** Create [[AsyncResourcesInitialization]] component.
*
* @param resources the list of resources to initialize
* @param ec the execution context
* @return new async initialization component
*/
def apply(resources: InitializationComponent*)(implicit
ec: ExecutionContext
): AsyncResourcesInitialization =
new AsyncResourcesInitialization(resources)
}

View File

@ -1,25 +0,0 @@
package org.enso.languageserver.boot.resource
import com.typesafe.scalalogging.LazyLogging
import org.enso.languageserver.data.ProjectDirectoriesConfig
import scala.concurrent.{ExecutionContext, Future}
/** Directories initialization.
*
* @param directoriesConfig the directories config
*/
class DirectoriesInitialization(directoriesConfig: ProjectDirectoriesConfig)(
implicit ec: ExecutionContext
) extends InitializationComponent
with LazyLogging {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
Future {
logger.info("Initializing directories...")
directoriesConfig.createDirectories()
logger.info("Initialized directories.")
InitializationComponent.Initialized
}
}

View File

@ -1,15 +0,0 @@
package org.enso.languageserver.boot.resource
import scala.concurrent.Future
/** A component that should be initialized. */
trait InitializationComponent {
/** Initialize the component. */
def init(): Future[InitializationComponent.Initialized.type]
}
object InitializationComponent {
case object Initialized
}

View File

@ -1,25 +0,0 @@
package org.enso.languageserver.boot.resource
import com.typesafe.scalalogging.LazyLogging
import org.enso.jsonrpc.ProtocolFactory
import scala.concurrent.{ExecutionContext, Future}
/** Initialization of JSON-RPC protocol.
*
* @param protocolFactory the JSON-RPC protocol factory
*/
class JsonRpcInitializationComponent(protocolFactory: ProtocolFactory)(implicit
ec: ExecutionContext
) extends InitializationComponent
with LazyLogging {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
Future {
logger.info("Initializing JSON-RPC protocol.")
protocolFactory.init()
logger.info("JSON-RPC protocol initialized.")
InitializationComponent.Initialized
}
}

View File

@ -1,152 +0,0 @@
package org.enso.languageserver.boot.resource
import java.io.IOException
import java.nio.file.{FileSystemException, Files, NoSuchFileException}
import akka.event.EventStream
import com.typesafe.scalalogging.LazyLogging
import org.apache.commons.io.FileUtils
import org.enso.languageserver.data.ProjectDirectoriesConfig
import org.enso.languageserver.event.InitializedEvent
import org.enso.logger.masking.MaskedPath
import org.enso.searcher.sql.{SqlDatabase, SqlSuggestionsRepo}
import scala.concurrent.{ExecutionContext, Future}
import scala.util.control.NonFatal
import scala.util.{Failure, Success}
/** Initialization of the Language Server repositories.
*
* @param directoriesConfig configuration of language server directories
* @param eventStream akka events stream
* @param sqlDatabase the sql database
* @param suggestionsRepo the suggestions repo
*/
class RepoInitialization(
directoriesConfig: ProjectDirectoriesConfig,
eventStream: EventStream,
sqlDatabase: SqlDatabase,
suggestionsRepo: SqlSuggestionsRepo
)(implicit ec: ExecutionContext)
extends InitializationComponent
with LazyLogging {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
for {
_ <- sqlDatabaseInit
_ <- suggestionsRepoInit
} yield InitializationComponent.Initialized
private def sqlDatabaseInit: Future[Unit] = {
val initAction = Future {
logger.info("Initializing sql database [{}]...", sqlDatabase)
sqlDatabase.open()
logger.info("Initialized sql database [{}].", sqlDatabase)
}
initAction.onComplete {
case Success(()) =>
case Failure(ex) =>
logger.error("Failed to initialize sql database [{}].", sqlDatabase, ex)
}
initAction
}
private def suggestionsRepoInit: Future[Unit] = {
val initAction =
for {
_ <- Future {
logger.info(
"Initializing suggestions repo [{}]...",
MaskedPath(directoriesConfig.suggestionsDatabaseFile.toPath)
)
}
_ <- suggestionsRepo.init.recoverWith { case NonFatal(error) =>
recoverInitError(error, suggestionsRepo.db)
}
_ <- Future {
logger.info(
"Initialized Suggestions repo [{}].",
MaskedPath(directoriesConfig.suggestionsDatabaseFile.toPath)
)
}
} yield ()
initAction.onComplete {
case Success(()) =>
eventStream.publish(InitializedEvent.SuggestionsRepoInitialized)
case Failure(ex) =>
logger.error(
"Failed to initialize SQL suggestions repo [{}].",
MaskedPath(directoriesConfig.suggestionsDatabaseFile.toPath),
ex
)
}
initAction
}
private def recoverInitError(
error: Throwable,
db: SqlDatabase
): Future[Unit] =
for {
_ <- Future {
logger.warn(
"Failed to initialize the suggestions database [{}].",
MaskedPath(directoriesConfig.suggestionsDatabaseFile.toPath),
error
)
}
_ <- Future(db.close())
_ <- clearDatabaseFile()
_ <- Future(db.open())
_ <- Future {
logger.info("Retrying database initialization.")
}
_ <- suggestionsRepo.init
} yield ()
private def clearDatabaseFile(retries: Int = 0): Future[Unit] = {
Future {
logger.info("Clear database file. Attempt #{}.", retries + 1)
Files.delete(directoriesConfig.suggestionsDatabaseFile.toPath)
}.recoverWith {
case _: NoSuchFileException =>
logger.warn(
"Failed to delete the database file. Attempt #{}. " +
"File does not exist [{}].",
retries + 1,
MaskedPath(directoriesConfig.suggestionsDatabaseFile.toPath)
)
Future.successful(())
case error: FileSystemException =>
logger.error(
"Failed to delete the database file. Attempt #{}. " +
"The file will be removed during the shutdown.",
retries + 1,
error
)
sys.addShutdownHook(
FileUtils.deleteQuietly(directoriesConfig.suggestionsDatabaseFile)
)
Future.failed(error)
case error: IOException =>
logger.error(
"Failed to delete the database file. Attempt #{}.",
retries + 1,
error
)
if (retries < RepoInitialization.MaxRetries) {
Thread.sleep(1000)
clearDatabaseFile(retries + 1)
} else {
Future.failed(error)
}
}
}
}
object RepoInitialization {
val MaxRetries = 3
}

View File

@ -1,26 +0,0 @@
package org.enso.languageserver.boot.resource
import scala.concurrent.{ExecutionContext, Future}
/** Initializes resources in sequence.
*
* @param resources the list of resources to initialize
*/
class SequentialResourcesInitialization(
resources: Seq[InitializationComponent]
)(implicit ec: ExecutionContext)
extends InitializationComponent {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
resources.foldLeft(Future.successful(InitializationComponent.Initialized)) {
(action, resource) => action.flatMap(_ => resource.init())
}
}
object SequentialResourcesInitialization {
def apply(resources: InitializationComponent*)(implicit
ec: ExecutionContext
): SequentialResourcesInitialization =
new SequentialResourcesInitialization(resources)
}

View File

@ -1,33 +0,0 @@
package org.enso.languageserver.boot.resource
import akka.event.EventStream
import com.typesafe.scalalogging.LazyLogging
import org.enso.languageserver.event.InitializedEvent
import org.enso.polyglot.LanguageInfo
import org.graalvm.polyglot.Context
import scala.concurrent.{ExecutionContext, Future}
/** Initialize the Truffle context.
*
* @param eventStream akka events stream
* @param truffleContext the Truffle context
*/
class TruffleContextInitialization(
eventStream: EventStream,
truffleContext: Context
)(implicit
ec: ExecutionContext
) extends InitializationComponent
with LazyLogging {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
Future {
logger.info("Initializing Runtime context...")
truffleContext.initialize(LanguageInfo.ID)
eventStream.publish(InitializedEvent.TruffleContextInitialized)
logger.info("Initialized Runtime context.")
InitializationComponent.Initialized
}
}

View File

@ -1,33 +0,0 @@
package org.enso.languageserver.boot.resource
import akka.event.EventStream
import com.typesafe.scalalogging.LazyLogging
import org.enso.languageserver.effect
import org.enso.languageserver.event.InitializedEvent
import scala.concurrent.{ExecutionContext, Future}
/** Initialization of ZIO runtime.
*
* @param runtime the runtime to initialize
* @param eventStream events stream
* @param ec the execution context
*/
class ZioRuntimeInitialization(
runtime: effect.Runtime,
eventStream: EventStream
)(implicit
ec: ExecutionContext
) extends InitializationComponent
with LazyLogging {
/** @inheritdoc */
override def init(): Future[InitializationComponent.Initialized.type] =
Future {
logger.info("Initializing ZIO runtime...")
runtime.init()
logger.info("ZIO runtime initialized [{}].", runtime)
eventStream.publish(InitializedEvent.ZioRuntimeInitialized)
InitializationComponent.Initialized
}
}

View File

@ -1,14 +1,17 @@
package org.enso.languageserver.protocol.json
import akka.actor.{Actor, ActorRef, Cancellable, Props, Stash, Status}
import akka.pattern.pipe
import akka.pattern.pipeCompletionStage
import akka.util.Timeout
import com.typesafe.scalalogging.LazyLogging
import org.enso.cli.task.ProgressUnit
import org.enso.cli.task.notifications.TaskNotificationApi
import org.enso.jsonrpc._
import org.enso.languageserver.ai.AICompletion
import org.enso.languageserver.boot.resource.InitializationComponent
import org.enso.languageserver.boot.resource.{
InitializationComponent,
InitializationComponentInitialized
}
import org.enso.languageserver.capability.CapabilityApi.{
AcquireCapability,
ForceReleaseCapability,
@ -173,8 +176,15 @@ class JsonConnectionController(
_,
InitProtocolConnection.Params(clientId)
) =>
logger.info("Initializing resources.")
mainComponent.init().pipeTo(self)
logger.info(
"Initializing resources for [{}] [{}].",
clientId,
mainComponent
)
mainComponent
.init()
.thenApply(_ => InitializationComponentInitialized.getInstance)
.pipeTo(self)
context.become(initializing(webActor, clientId, req, sender()))
case Request(_, id, _) =>
@ -190,7 +200,7 @@ class JsonConnectionController(
request: Request[_, _],
receiver: ActorRef
): Receive = {
case InitializationComponent.Initialized =>
case _: InitializationComponentInitialized =>
logger.info("RPC session initialized for client [{}].", clientId)
val session = JsonSession(clientId, self)
context.system.eventStream.publish(JsonSessionInitialized(session))

View File

@ -331,17 +331,14 @@ final class SuggestionsHandler(
handlerAction.pipeTo(handler)
if (state.shouldStartBackgroundProcessing) {
runtimeConnector ! Api.Request(Api.StartBackgroundProcessing())
context.become(
initialized(
projectName,
graph,
clients,
state.backgroundProcessingStarted()
)
context.become(
initialized(
projectName,
graph,
clients,
state.backgroundProcessingStarted()
)
}
)
case Completion(path, pos, selfType, returnType, tags, isStatic) =>
val selfTypes = selfType.toList.flatMap(ty => ty :: graph.getParents(ty))
@ -426,14 +423,7 @@ final class SuggestionsHandler(
)
)
action.pipeTo(handler)(sender())
context.become(
initialized(
projectName,
graph,
clients,
state.backgroundProcessingStopped()
)
)
context.become(initialized(projectName, graph, clients, state))
case ProjectNameUpdated(name, updates) =>
updates.foreach(sessionRouter ! _)

View File

@ -8,7 +8,7 @@ import org.enso.languageserver.data._
import org.enso.languageserver.event.InitializedEvent
import org.enso.languageserver.filemanager.{ContentRoot, ContentRootWithFile}
import org.enso.searcher.sql.{SchemaVersion, SqlDatabase, SqlSuggestionsRepo}
import org.enso.testkit.FlakySpec
import org.enso.testkit.{FlakySpec, ToScalaFutureConversions}
import org.scalatest.BeforeAndAfterAll
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpecLike
@ -16,6 +16,7 @@ import org.sqlite.SQLiteException
import java.nio.file.{Files, StandardOpenOption}
import java.util.UUID
import scala.concurrent.Await
import scala.concurrent.duration._
@ -25,6 +26,7 @@ class RepoInitializationSpec
with AnyWordSpecLike
with Matchers
with BeforeAndAfterAll
with ToScalaFutureConversions
with FlakySpec {
import system.dispatcher
@ -43,6 +45,7 @@ class RepoInitializationSpec
val component =
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,
@ -68,6 +71,7 @@ class RepoInitializationSpec
val testSchemaVersion = Long.MaxValue
val component =
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,
@ -96,6 +100,7 @@ class RepoInitializationSpec
val component =
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,
@ -132,6 +137,7 @@ class RepoInitializationSpec
withRepos(config) { (sqlDatabase, suggestionsRepo) =>
val component =
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,
@ -168,6 +174,7 @@ class RepoInitializationSpec
withRepos(config) { (sqlDatabase, suggestionsRepo) =>
val component =
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,

View File

@ -60,7 +60,6 @@ import org.slf4j.event.Level
import java.nio.file.{Files, Path}
import java.util.UUID
import scala.concurrent.Await
import scala.concurrent.duration._
class BaseServerTest
@ -70,8 +69,6 @@ class BaseServerTest
with WithTemporaryDirectory
with FakeEnvironment {
import system.dispatcher
val timeout: FiniteDuration = 10.seconds
def isFileWatcherEnabled: Boolean = false
@ -141,16 +138,23 @@ class BaseServerTest
val sqlDatabase = SqlDatabase(config.directories.suggestionsDatabaseFile)
val suggestionsRepo = new SqlSuggestionsRepo(sqlDatabase)(system.dispatcher)
val initializationComponent = SequentialResourcesInitialization(
new DirectoriesInitialization(config.directories),
new ZioRuntimeInitialization(zioRuntime, system.eventStream),
new RepoInitialization(
config.directories,
system.eventStream,
sqlDatabase,
suggestionsRepo
private def initializationComponent =
new SequentialResourcesInitialization(
system.dispatcher,
new DirectoriesInitialization(system.dispatcher, config.directories),
new ZioRuntimeInitialization(
system.dispatcher,
zioRuntime,
system.eventStream
),
new RepoInitialization(
system.dispatcher,
config.directories,
system.eventStream,
sqlDatabase,
suggestionsRepo
)
)
)
val contentRootManagerActor =
system.actorOf(ContentRootManagerActor.props(config))
@ -262,7 +266,7 @@ class BaseServerTest
UUID.randomUUID(),
Api.GetTypeGraphResponse(typeGraph)
)
Await.ready(initializationComponent.init(), timeout)
initializationComponent.init().get(timeout.length, timeout.unit)
suggestionsHandler ! ProjectNameUpdated("Test")
val environment = fakeInstalledEnvironment()

View File

@ -5,6 +5,7 @@ import java.util.UUID;
import java.util.logging.Level;
import org.enso.interpreter.instrument.execution.RuntimeContext;
import org.enso.interpreter.instrument.job.DeserializeLibrarySuggestionsJob;
import org.enso.interpreter.instrument.job.StartBackgroundProcessingJob;
import org.enso.interpreter.runtime.EnsoContext;
import org.enso.polyglot.runtime.Runtime$Api$InvalidateModulesIndexResponse;
import scala.Option;
@ -25,17 +26,17 @@ public final class InvalidateModulesIndexCommand extends AsynchronousCommand {
}
@Override
@SuppressWarnings("unchecked")
public Future<BoxedUnit> executeAsynchronously(RuntimeContext ctx, ExecutionContext ec) {
return Future.apply(
() -> {
TruffleLogger logger = ctx.executionService().getLogger();
long writeCompilationLockTimestamp = ctx.locking().acquireWriteCompilationLock();
try {
ctx.jobControlPlane().abortAllJobs();
ctx.jobControlPlane().abortBackgroundJobs(DeserializeLibrarySuggestionsJob.class);
EnsoContext context = ctx.executionService().getContext();
context.getTopScope().getModules().forEach(module -> module.setIndexed(false));
ctx.jobControlPlane().stopBackgroundJobs();
context
.getPackageRepository()
@ -47,6 +48,7 @@ public final class InvalidateModulesIndexCommand extends AsynchronousCommand {
return BoxedUnit.UNIT;
});
StartBackgroundProcessingJob.startBackgroundJobs(ctx);
reply(new Runtime$Api$InvalidateModulesIndexResponse(), ctx);
} finally {
ctx.locking().releaseWriteCompilationLock();

View File

@ -1,19 +1,24 @@
package org.enso.interpreter.instrument.job;
import java.util.Comparator;
import scala.collection.immutable.List$;
/** The job that runs in the background. */
public abstract class BackgroundJob<A> extends Job<A> implements Comparable<BackgroundJob<?>> {
public abstract class BackgroundJob<A> extends Job<A> {
private final int priority;
/** Comparator defining the order of jobs in the background jobs queue. */
public static final Comparator<BackgroundJob<?>> BACKGROUND_JOBS_QUEUE_ORDER =
Comparator.comparingInt(BackgroundJob::getPriority);
/**
* Create a background job with priority.
*
* @param priority the job priority. Lower number indicates higher priority.
*/
public BackgroundJob(int priority) {
super(List$.MODULE$.empty(), false, false);
super(List$.MODULE$.empty(), true, false);
this.priority = priority;
}
@ -21,9 +26,4 @@ public abstract class BackgroundJob<A> extends Job<A> implements Comparable<Back
public int getPriority() {
return priority;
}
@Override
public int compareTo(BackgroundJob<?> that) {
return Integer.compare(this.priority, that.getPriority());
}
}

View File

@ -55,4 +55,9 @@ public final class SerializeModuleJob extends BackgroundJob<Void> {
}
return null;
}
@Override
public String toString() {
return "SerializeModuleJob(" + moduleName.toString() + ")";
}
}

View File

@ -38,7 +38,7 @@ class ModifyVisualizationCmd(
val jobFilter: PartialFunction[Job[_], Option[ExpressionId]] = {
case upsert: UpsertVisualizationJob
if upsert.visualizationId == request.visualizationId =>
Some(upsert.key)
Some(upsert.expressionId)
}
ctx.jobControlPlane.jobInProgress(jobFilter)
}

View File

@ -4,6 +4,8 @@ import org.enso.interpreter.instrument.job.Job
import java.util.UUID
import scala.annotation.varargs
/** Controls running jobs.
*/
trait JobControlPlane {
@ -15,6 +17,7 @@ trait JobControlPlane {
*
* @param ignoredJobs the list of jobs to keep in the execution queue
*/
@varargs
def abortAllExcept(ignoredJobs: Class[_ <: Job[_]]*): Unit
/** Aborts all jobs that relates to the specified execution context.
@ -23,6 +26,13 @@ trait JobControlPlane {
*/
def abortJobs(contextId: UUID): Unit
/** Abort provided background jobs.
*
* @param toAbort the list of jobs to abort
*/
@varargs
def abortBackgroundJobs(toAbort: Class[_ <: Job[_]]*): Unit
/** Starts background jobs processing.
*
* @return `true` if the background jobs were started and `false` if they are

View File

@ -66,27 +66,38 @@ final class JobExecutionEngine(
override def runBackground[A](job: BackgroundJob[A]): Unit =
synchronized {
if (isBackgroundJobsStarted) {
cancelDuplicateJobs(job, backgroundJobsRef)
runInternal(job, backgroundJobExecutor, backgroundJobsRef)
} else {
job match {
case job: UniqueJob[_] =>
delayedBackgroundJobsQueue.removeIf {
case that: UniqueJob[_] => that.equalsTo(job)
case _ => false
}
case _ =>
}
delayedBackgroundJobsQueue.add(job)
}
}
/** @inheritdoc */
override def run[A](job: Job[A]): Future[A] = {
cancelDuplicateJobs(job)
cancelDuplicateJobs(job, runningJobsRef)
runInternal(job, jobExecutor, runningJobsRef)
}
private def cancelDuplicateJobs[A](job: Job[A]): Unit = {
private def cancelDuplicateJobs[A](
job: Job[A],
runningJobsRef: AtomicReference[Vector[RunningJob]]
): Unit = {
job match {
case job: UniqueJob[_] =>
val allJobs =
runningJobsRef.updateAndGet(_.filterNot(_.future.isCancelled))
allJobs.foreach { runningJob =>
runningJob.job match {
case jobRef: UniqueJob[_]
if jobRef.getClass == job.getClass && jobRef.key == job.key =>
case jobRef: UniqueJob[_] if jobRef.equalsTo(job) =>
runtimeContext.executionService.getLogger
.log(Level.FINEST, s"Cancelling duplicate job [$jobRef].")
runningJob.future.cancel(jobRef.mayInterruptIfRunning)
@ -164,6 +175,19 @@ final class JobExecutionEngine(
.interruptThreads()
}
override def abortBackgroundJobs(toAbort: Class[_ <: Job[_]]*): Unit = {
val allJobs =
backgroundJobsRef.updateAndGet(_.filterNot(_.future.isCancelled))
val cancellableJobs = allJobs
.filter { runningJob =>
runningJob.job.isCancellable &&
toAbort.contains(runningJob.job.getClass)
}
cancellableJobs.foreach { runningJob =>
runningJob.future.cancel(runningJob.job.mayInterruptIfRunning)
}
}
/** @inheritdoc */
override def startBackgroundJobs(): Boolean =
synchronized {
@ -193,7 +217,18 @@ final class JobExecutionEngine(
/** Submit background jobs preserving the stable order. */
private def submitBackgroundJobsOrdered(): Unit = {
Collections.sort(delayedBackgroundJobsQueue)
Collections.sort(
delayedBackgroundJobsQueue,
BackgroundJob.BACKGROUND_JOBS_QUEUE_ORDER
)
runtimeContext.executionService.getLogger.log(
Level.FINE,
"Submitting {0} background jobs [{1}]",
Array[AnyRef](
delayedBackgroundJobsQueue.size(): Integer,
delayedBackgroundJobsQueue
)
)
delayedBackgroundJobsQueue.forEach(job => runBackground(job))
delayedBackgroundJobsQueue.clear()
}

View File

@ -13,8 +13,17 @@ import scala.jdk.CollectionConverters._
* @param libraryName the name of loaded library
*/
final class DeserializeLibrarySuggestionsJob(
libraryName: LibraryName
) extends BackgroundJob[Unit](DeserializeLibrarySuggestionsJob.Priority) {
val libraryName: LibraryName
) extends BackgroundJob[Unit](DeserializeLibrarySuggestionsJob.Priority)
with UniqueJob[Unit] {
/** @inheritdoc */
override def equalsTo(that: UniqueJob[_]): Boolean =
that match {
case that: DeserializeLibrarySuggestionsJob =>
this.libraryName == that.libraryName
case _ => false
}
/** @inheritdoc */
override def run(implicit ctx: RuntimeContext): Unit = {

View File

@ -17,9 +17,18 @@ import java.util.logging.Level
*/
class DetachVisualizationJob(
visualizationId: VisualizationId,
expressionId: ExpressionId,
val expressionId: ExpressionId,
contextId: ContextId
) extends UniqueJob[Unit](expressionId, List(contextId), false) {
) extends Job[Unit](List(contextId), false, false)
with UniqueJob[Unit] {
/** @inheritdoc */
override def equalsTo(that: UniqueJob[_]): Boolean =
that match {
case that: DetachVisualizationJob =>
this.expressionId == that.expressionId
case _ => false
}
/** @inheritdoc */
override def run(implicit ctx: RuntimeContext): Unit = {

View File

@ -88,9 +88,7 @@ class ExecuteJob(
Level.FINEST,
s"Kept context lock [ExecuteJob] for ${contextId} for ${System.currentTimeMillis() - acquiredLock} milliseconds"
)
}
StartBackgroundProcessingJob.startBackgroundJobs()
}
override def toString(): String = {

View File

@ -25,17 +25,16 @@ abstract class Job[+A](
def run(implicit ctx: RuntimeContext): A
}
/** The job queue can contain only one job of this type with the same `key`.
* When a job of this type is added to the job queue, previous duplicate jobs
* are cancelled.
*
* @param key a unique job key
* @param contextIds affected executions contests' ids
* @param mayInterruptIfRunning determines if the job may be interruptd when
* running
/** The job queue can contain only one job of this type decided by the
* `equalsTo` method. When a job of this type is added to the job queue,
* previous duplicate jobs are cancelled.
*/
abstract class UniqueJob[+A](
val key: UUID,
contextIds: List[UUID],
mayInterruptIfRunning: Boolean
) extends Job[A](contextIds, isCancellable = false, mayInterruptIfRunning)
trait UniqueJob[A] { self: Job[A] =>
/** Decide if this job is the same as the other job.
*
* @param that the other job to compare with
* @return `true` if `this` job is considered the same as `that` job
*/
def equalsTo(that: UniqueJob[_]): Boolean
}

View File

@ -38,13 +38,22 @@ import java.util.logging.Level
class UpsertVisualizationJob(
requestId: Option[Api.RequestId],
val visualizationId: Api.VisualizationId,
expressionId: Api.ExpressionId,
val expressionId: Api.ExpressionId,
config: Api.VisualizationConfiguration
) extends UniqueJob[Option[Executable]](
expressionId,
) extends Job[Option[Executable]](
List(config.executionContextId),
false,
false
) {
)
with UniqueJob[Option[Executable]] {
/** @inheritdoc */
override def equalsTo(that: UniqueJob[_]): Boolean =
that match {
case that: UpsertVisualizationJob =>
this.expressionId == that.expressionId
case _ => false
}
/** @inheritdoc */
override def run(implicit ctx: RuntimeContext): Option[Executable] = {

View File

@ -12,7 +12,6 @@ import org.enso.interpreter.runtime.type.ConstantsGen;
import org.enso.interpreter.test.Metadata;
import org.enso.interpreter.test.NodeCountingTestInstrument;
import org.enso.interpreter.test.instrument.RuntimeServerTest.TestContext;
import org.enso.polyglot.runtime.Runtime$Api$BackgroundJobsStartedNotification;
import org.enso.polyglot.runtime.Runtime$Api$CreateContextRequest;
import org.enso.polyglot.runtime.Runtime$Api$CreateContextResponse;
import org.enso.polyglot.runtime.Runtime$Api$EditFileNotification;
@ -220,12 +219,11 @@ public class IncrementalUpdatesTest {
)
);
assertSameElements(context.receiveNIgnorePendingExpressionUpdates(5, 10, emptySet()),
assertSameElements(context.receiveNIgnorePendingExpressionUpdates(4, 10, emptySet()),
Response(requestId, new Runtime$Api$PushContextResponse(contextId)),
TestMessages.update(contextId, mainFoo, exprType, new Runtime$Api$MethodCall(new Runtime$Api$MethodPointer("Enso_Test.Test.Main", "Enso_Test.Test.Main", "foo"), Vector$.MODULE$.empty())),
TestMessages.update(contextId, mainRes, ConstantsGen.NOTHING),
context.executionComplete(contextId),
Response(new Runtime$Api$BackgroundJobsStartedNotification())
context.executionComplete(contextId)
);
assertEquals(List.newBuilder().addOne(originalOutput), context.consumeOut());

View File

@ -139,9 +139,8 @@ class RuntimeAsyncCommandsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreExpressionUpdates(
3
2
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -224,7 +223,7 @@ class RuntimeAsyncCommandsTest
Api.Request(requestId, Api.InterruptContextRequest(contextId))
)
context.receiveNIgnoreExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.InterruptContextResponse(contextId)),
Api.Response(
@ -244,8 +243,7 @@ class RuntimeAsyncCommandsTest
),
Api.Response(
Api.ExecutionComplete(contextId)
),
Api.Response(Api.BackgroundJobsStartedNotification())
)
)
}
}

View File

@ -167,9 +167,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
7
6
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -264,9 +263,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
7
6
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -354,9 +352,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
5
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -431,9 +428,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
5
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -503,9 +499,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
5
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -588,9 +583,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -665,9 +659,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
7
6
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -751,9 +744,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
7
6
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -835,9 +827,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -1010,9 +1001,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -1113,9 +1103,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -1203,9 +1192,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, xId, ConstantsGen.VECTOR),
TestMessages.update(
@ -1284,9 +1272,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -1398,9 +1385,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
7
6
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -1520,9 +1506,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -1668,9 +1653,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -1787,9 +1771,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -1856,9 +1839,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -1961,9 +1943,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -2073,9 +2054,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -2186,9 +2166,8 @@ class RuntimeErrorsTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
5
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -2285,8 +2264,7 @@ class RuntimeErrorsTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,

View File

@ -179,9 +179,8 @@ class RuntimeExecutionEnvironmentTest
)
)
)
val responses1 = context.receiveNIgnoreStdLib(4)
val responses1 = context.receiveNIgnoreStdLib(3)
responses1 should contain allOf (
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -279,9 +278,8 @@ class RuntimeExecutionEnvironmentTest
)
)
)
val responses1 = context.receiveNIgnoreStdLib(4)
val responses1 = context.receiveNIgnoreStdLib(3)
responses1 should contain allOf (
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,

View File

@ -149,8 +149,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -233,8 +232,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -319,8 +317,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, exprOperator1, ConstantsGen.INTEGER),
TestMessages.update(contextId, exprOperator2, ConstantsGen.INTEGER),
@ -451,8 +448,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -537,8 +533,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -625,8 +620,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, exprOperator1, ConstantsGen.INTEGER),
TestMessages.update(
@ -764,8 +758,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -836,8 +829,7 @@ class RuntimeRefactoringTest
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, exprOperator1, ConstantsGen.INTEGER),
context.executionComplete(contextId)

View File

@ -409,8 +409,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId, typeChanged = true),
context.Main.Update.mainY(contextId, typeChanged = true),
@ -512,8 +511,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idFoo, ConstantsGen.INTEGER),
context.executionComplete(contextId)
@ -567,8 +565,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -674,8 +671,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(10) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(9) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -778,8 +774,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -853,8 +848,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -923,8 +917,7 @@ class RuntimeServerTest
"Standard.Base.Data.Text.Text",
"+"
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -995,8 +988,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1068,8 +1060,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1154,8 +1145,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1257,8 +1247,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1374,8 +1363,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1471,8 +1459,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1588,8 +1575,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1705,8 +1691,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1822,8 +1807,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -1935,8 +1919,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2034,8 +2017,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2129,8 +2111,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2194,8 +2175,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, id_x, ConstantsGen.DATE),
TestMessages.update(
@ -2258,8 +2238,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, id_x, "Standard.Base.Data.Time.Date.Date"),
context.executionComplete(contextId)
@ -2310,8 +2289,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2370,8 +2348,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2431,8 +2408,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2494,8 +2470,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2622,8 +2597,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages
.update(
@ -2754,8 +2728,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2855,8 +2828,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(7) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -2939,8 +2911,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idResult, ConstantsGen.INTEGER),
TestMessages.update(contextId, idPrintln, ConstantsGen.NOTHING),
@ -3039,8 +3010,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idMainA, ConstantsGen.INTEGER),
TestMessages.update(contextId, idMainP, ConstantsGen.NOTHING),
@ -3325,8 +3295,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(7) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idMain, ConstantsGen.NOTHING),
TestMessages.update(
@ -3577,8 +3546,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, xId, ConstantsGen.FUNCTION),
TestMessages.update(contextId, mainRes, ConstantsGen.NOTHING),
@ -3676,8 +3644,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -3747,8 +3714,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -3828,8 +3794,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idMain, ConstantsGen.INTEGER_BUILTIN),
context.executionComplete(contextId)
@ -3891,8 +3856,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(7) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -3978,8 +3942,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId)
)
@ -4074,8 +4037,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idText, ConstantsGen.TEXT),
TestMessages.update(contextId, idRes, ConstantsGen.NOTHING),
@ -4151,8 +4113,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -4198,8 +4159,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -4263,8 +4223,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -4321,8 +4280,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -4396,8 +4354,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4444,8 +4401,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4495,8 +4451,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4549,8 +4504,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4616,8 +4570,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4682,8 +4635,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4757,8 +4709,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4831,7 +4782,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(3) should contain theSameElementsAs Seq(
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4861,8 +4812,7 @@ class RuntimeServerTest
)
)
)
),
Api.Response(Api.BackgroundJobsStartedNotification())
)
)
}
@ -4906,7 +4856,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -4936,8 +4886,7 @@ class RuntimeServerTest
)
)
)
),
Api.Response(Api.BackgroundJobsStartedNotification())
)
)
}
@ -4981,8 +4930,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -5049,8 +4997,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -5125,7 +5072,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(3) should contain theSameElementsAs Seq(
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -5171,8 +5118,7 @@ class RuntimeServerTest
)
)
)
),
Api.Response(Api.BackgroundJobsStartedNotification())
)
)
}
@ -5226,7 +5172,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
context.receiveNIgnoreStdLib(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionFailed(
@ -5272,8 +5218,7 @@ class RuntimeServerTest
)
)
)
),
Api.Response(Api.BackgroundJobsStartedNotification())
)
)
}
@ -5316,8 +5261,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -5376,8 +5320,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -5500,8 +5443,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
@ -5673,8 +5615,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main2.Update.mainY(contextId),
context.Main2.Update.mainZ(contextId),
@ -5728,8 +5669,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -5844,8 +5784,7 @@ class RuntimeServerTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -5993,9 +5932,8 @@ class RuntimeServerTest
)
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages
.update(
@ -6089,8 +6027,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(6) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages
.update(
@ -6172,7 +6109,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -6184,8 +6121,7 @@ class RuntimeServerTest
)
)
),
context.executionComplete(contextId),
Api.Response(Api.BackgroundJobsStartedNotification())
context.executionComplete(contextId)
)
}
@ -6239,8 +6175,7 @@ class RuntimeServerTest
)
)
)
context.receiveN(8) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(7) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, x, ConstantsGen.INTEGER_BUILTIN),
TestMessages.update(contextId, `y_inc`, Constants.UNRESOLVED_SYMBOL),
@ -6406,8 +6341,7 @@ class RuntimeServerTest
)
)
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idX, ConstantsGen.TEXT),
context.executionComplete(contextId)

View File

@ -92,6 +92,13 @@ class RuntimeSuggestionUpdatesTest
override protected def beforeEach(): Unit = {
context = new TestContext("Test")
val Some(Api.Response(_, Api.InitializedNotification())) = context.receive
context.send(
Api.Request(UUID.randomUUID(), Api.StartBackgroundProcessing())
)
context.receive shouldEqual Some(
Api.Response(Api.BackgroundJobsStartedNotification())
)
}
override protected def afterEach(): Unit = {
@ -138,9 +145,8 @@ class RuntimeSuggestionUpdatesTest
)
)
context.receiveNIgnoreExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
@ -676,9 +682,8 @@ class RuntimeSuggestionUpdatesTest
)
)
context.receiveNIgnoreExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
@ -832,9 +837,8 @@ class RuntimeSuggestionUpdatesTest
)
)
context.receiveNIgnoreExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
@ -1010,9 +1014,8 @@ class RuntimeSuggestionUpdatesTest
)
)
context.receiveNIgnoreExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.SuggestionsDatabaseModuleUpdateNotification(
@ -1306,12 +1309,11 @@ class RuntimeSuggestionUpdatesTest
)
)
)
val updates1 = context.receiveNIgnoreExpressionUpdates(4)
updates1.length shouldEqual 4
val updates1 = context.receiveNIgnoreExpressionUpdates(3)
updates1.length shouldEqual 3
updates1 should contain allOf (
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.executionComplete(contextId),
Api.Response(Api.BackgroundJobsStartedNotification())
)
val indexedModules = updates1.collect {
case Api.Response(
@ -1333,12 +1335,11 @@ class RuntimeSuggestionUpdatesTest
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
)
val updates2 = context.receiveNIgnoreExpressionUpdates(4)
updates2.length shouldEqual 4
val updates2 = context.receiveNIgnoreExpressionUpdates(3)
updates2.length shouldEqual 3
updates2 should contain allOf (
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
context.executionComplete(contextId),
Api.Response(Api.BackgroundJobsStartedNotification())
context.executionComplete(contextId)
)
val indexedModules2 = updates1.collect {
case Api.Response(

View File

@ -358,9 +358,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -478,9 +477,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -614,9 +612,8 @@ class RuntimeVisualizationsTest
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -743,9 +740,8 @@ class RuntimeVisualizationsTest
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -872,9 +868,8 @@ class RuntimeVisualizationsTest
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -1110,9 +1105,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -1247,8 +1241,7 @@ class RuntimeVisualizationsTest
)
)
)
context.receiveN(3) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.VisualizationAttached()),
Api.Response(
Api.ExecutionFailed(
@ -1383,9 +1376,8 @@ class RuntimeVisualizationsTest
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -1496,9 +1488,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
6
5
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
@ -1629,9 +1620,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -1696,9 +1686,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -1793,9 +1782,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -1875,9 +1863,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -1988,9 +1975,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -2098,8 +2084,7 @@ class RuntimeVisualizationsTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -2196,9 +2181,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.panic(
contextId,
@ -2327,8 +2311,7 @@ class RuntimeVisualizationsTest
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(4) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
context.receiveNIgnoreStdLib(3) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.error(
contextId,
@ -2435,9 +2418,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -2534,9 +2516,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -2662,9 +2643,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -2837,9 +2817,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
7
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),
@ -2999,9 +2978,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -3100,7 +3078,7 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
4
3
) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
@ -3113,8 +3091,7 @@ class RuntimeVisualizationsTest
)
)
),
context.executionComplete(contextId),
Api.Response(Api.BackgroundJobsStartedNotification())
context.executionComplete(contextId)
)
// attach visualization
@ -3205,9 +3182,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
5
4
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
@ -3341,9 +3317,8 @@ class RuntimeVisualizationsTest
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnorePendingExpressionUpdates(
9
8
) should contain theSameElementsAs Seq(
Api.Response(Api.BackgroundJobsStartedNotification()),
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, idX, s"$moduleName.T"),
TestMessages.update(

View File

@ -15,7 +15,6 @@ import org.enso.compiler.codegen.IrToTruffle;
import org.enso.compiler.codegen.RuntimeStubsGenerator;
import org.enso.compiler.context.CompilerContext;
import org.enso.compiler.context.FreshNameSupply;
import org.enso.compiler.core.IR;
import org.enso.compiler.core.ir.Expression;
import org.enso.compiler.data.CompilerConfig;
import org.enso.interpreter.node.ExpressionNode;
@ -24,8 +23,6 @@ import org.enso.interpreter.runtime.scope.ModuleScope;
import org.enso.interpreter.runtime.scope.TopLevelScope;
import org.enso.pkg.QualifiedName;
import org.enso.polyglot.CompilationStage;
import org.enso.polyglot.RuntimeOptions;
import com.oracle.truffle.api.TruffleFile;
import com.oracle.truffle.api.TruffleLogger;
import com.oracle.truffle.api.source.Source;

View File

@ -314,9 +314,9 @@ final class SerializationManager(compiler: Compiler) {
libraryName
)
result
case _ =>
case None =>
compiler.context.logSerializationManager(
Level.FINEST,
Level.FINE,
"Unable to load suggestions for library [{0}].",
libraryName
)

View File

@ -0,0 +1,15 @@
package org.enso.testkit
import org.scalatest.TestSuite
import java.util.concurrent.CompletableFuture
import scala.concurrent.Future
import scala.jdk.FutureConverters._
trait ToScalaFutureConversions extends TestSuite {
/** Convert Java future to Scala. */
implicit final def toScalaFuture[A](f: CompletableFuture[A]): Future[A] =
f.asScala
}