Run node in a different execution environment (#11173)

close #10719

Changelog:
- add: optional `expressionConfigs` parameter to the `executionContext/recompute` request
- update: `IdExecutionInstrument` allowing to run a single node in a specified execution environment
- refactor: move tests related to the recompute request to a separate test suite. Otherwise the `RuntimeServerTest` is becoming too bloated

# Important Notes
The updated `executionContext/recompute` request.

```typescript
interface ExecutionContextRecomputeParameters {
/** The execution context identifier. */
contextId: ContextId;

/** The expressions that will be invalidated before the execution.
*
*  Only the provided expression ids are invalidated excluding the dependencies.
*/
invalidatedExpressions?: "all" | ExpressionId[];

/** The execution environment that will be used in the execution. */
executionEnvironment?: ExecutionEnvironment;

/** The execution configurations for particular expressions.
*
*  The provided expressions will be invalidated from the cache with the
*  dependencies. The result of the execution will stay in the cache until the
*  cache is invalidated by editing the node or other means.
*/
expressionConfigs?: ExpressionConfig[];
}

interface ExpressionConfig {
/** The expression identifier. */
expressionId: ExpressionId;
/** The execution environment that should be used to run this expression. */
executionEnvironment?: ExecutionEnvironment;
}
```

#### Use cases

- to re-run a single node without re-running the dependent nodes (subtree), put the node id in the `invalidatedExpressions` parameter.
- to re-run a node with dependent nodes (subtree), put the node id in the `expressionConfigs` parameter with empty `executionEnvironment`
- to re-run a node in a different execution environment, put the node  id in the `expressionConfigs` and specify the `executionEnvieronment`
This commit is contained in:
Dmitry Bushev 2024-10-09 15:09:45 +03:00 committed by GitHub
parent 9d9c03eeaa
commit 78993a0d1a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
32 changed files with 2033 additions and 628 deletions

View File

@ -25,6 +25,8 @@ transport formats, please look [here](./protocol-architecture).
- [`MethodCall`](#methodcall)
- [`MethodPointer`](#methodpointer)
- [`ProfilingInfo`](#profilinginfo)
- [`ExecutionEnvironment`](#executionenvironment)
- [`ExpressionConfig`](#expressionConfig)
- [`ExpressionUpdate`](#expressionupdate)
- [`ExpressionUpdatePayload`](#expressionupdatepayload)
- [`VisualizationConfiguration`](#visualizationconfiguration)
@ -343,6 +345,19 @@ The execution environment of Enso runtime.
type ExecutionEnvironment = Design | Live;
```
### `ExpressionConfig`
The expression configuration used in the recompute request.
```typescript
interface ExpressionConfig {
/** The expression identifier. */
expressionId: ExpressionId;
/** The execution environment that should be used to run this expression. */
executionEnvironment?: ExecutionEnvironment;
}
```
### `ExpressionUpdate`
An update about the computed expression.
@ -3633,10 +3648,23 @@ May include a list of expressions for which caches should be invalidated.
interface ExecutionContextRecomputeParameters {
/** The execution context identifier. */
contextId: ContextId;
/** The expressions that will be invalidated before the execution. */
/** The expressions that will be invalidated before the execution.
*
* Only the provided expression ids are invalidated excluding the dependencies.
*/
invalidatedExpressions?: "all" | ExpressionId[];
/** The execution environment that will be used in the execution. */
executionEnvironment?: ExecutionEnvironment;
/** The execution configurations for particular expressions.
*
* The provided expressions will be invalidated from the cache with the
* dependencies. The result of the execution will stay in the cache until the
* cache is invalidated by editing the node or other means.
*/
expressionConfigs?: ExpressionConfig[];
}
```

View File

@ -48,7 +48,8 @@ class RecomputeHandler(
session,
msg.params.contextId,
msg.params.invalidatedExpressions,
msg.params.executionEnvironment
msg.params.executionEnvironment,
msg.params.expressionConfigs.getOrElse(Seq())
)
override protected def positiveResponse(

View File

@ -193,7 +193,8 @@ final class ContextRegistry(
client,
contextId,
expressions,
environment
environment,
expressionConfigs
) =>
if (store.hasContext(client.clientId, contextId)) {
val handler =
@ -210,7 +211,8 @@ final class ContextRegistry(
Api.RecomputeContextRequest(
contextId,
invalidatedExpressions,
environment.map(ExecutionEnvironments.toApi)
environment.map(ExecutionEnvironments.toApi),
expressionConfigs.map(_.toApi)
)
)
} else {

View File

@ -98,12 +98,14 @@ object ContextRegistryProtocol {
* @param contextId execution context identifier
* @param invalidatedExpressions the expressions that should be invalidated
* @param executionEnvironment the environment that should be used for execution
* @param expressionConfigs the execution configurations for particular expressions
*/
case class RecomputeContextRequest(
rpcSession: JsonSession,
contextId: ContextId,
invalidatedExpressions: Option[InvalidatedExpressions],
executionEnvironment: Option[ExecutionEnvironments.ExecutionEnvironment]
executionEnvironment: Option[ExecutionEnvironments.ExecutionEnvironment],
expressionConfigs: Seq[ExpressionConfig]
)
/** A response about recomputing the context.

View File

@ -108,7 +108,8 @@ object ExecutionApi {
case class Params(
contextId: ContextId,
invalidatedExpressions: Option[InvalidatedExpressions],
executionEnvironment: Option[ExecutionEnvironments.ExecutionEnvironment]
executionEnvironment: Option[ExecutionEnvironments.ExecutionEnvironment],
expressionConfigs: Option[Seq[ExpressionConfig]]
)
implicit

View File

@ -0,0 +1,23 @@
package org.enso.languageserver.runtime
import org.enso.polyglot.runtime.Runtime.Api
import java.util.UUID
/** The expression configuration used in the recompute request.
*
* @param expressionId the expression identifier
* @param executionEnvironment the execution environment used to run this expression
*/
case class ExpressionConfig(
expressionId: UUID,
executionEnvironment: Option[ExecutionEnvironments.ExecutionEnvironment]
) {
/** Convert this expression config to the runtime API. */
def toApi: Api.ExpressionConfig =
Api.ExpressionConfig(
expressionId,
executionEnvironment.map(ExecutionEnvironments.toApi)
)
}

View File

@ -3,8 +3,7 @@ package org.enso.languageserver.runtime
import io.circe.syntax._
import io.circe.{Decoder, DecodingFailure, Encoder}
/** A request to invalidate expressions.
*/
/** A request to invalidate expressions. */
sealed trait InvalidatedExpressions
object InvalidatedExpressions {

View File

@ -449,7 +449,8 @@ class ContextRegistryTest extends BaseServerTest with ReportLogsOnFailure {
Api.RecomputeContextRequest(
`contextId`,
Some(Api.InvalidatedExpressions.All()),
None
None,
Seq()
)
) =>
requestId
@ -527,7 +528,92 @@ class ContextRegistryTest extends BaseServerTest with ReportLogsOnFailure {
Vector(`expressionId`)
)
),
None
None,
Seq()
)
) =>
requestId
case msg =>
fail(s"Unexpected message: $msg")
}
runtimeConnectorProbe.lastSender ! Api.Response(
requestId3,
Api.RecomputeContextResponse(contextId)
)
client.expectJson(json.ok(3))
}
"recompute with expression configs" in {
val client = getInitialisedWsClient()
// create context
client.send(json.executionContextCreateRequest(1))
val (requestId, contextId) =
runtimeConnectorProbe.receiveN(1).head match {
case Api.Request(requestId, Api.CreateContextRequest(contextId)) =>
(requestId, contextId)
case msg =>
fail(s"Unexpected message: $msg")
}
runtimeConnectorProbe.lastSender ! Api.Response(
requestId,
Api.CreateContextResponse(contextId)
)
client.expectJson(json.executionContextCreateResponse(1, contextId))
// push stack item
val expressionId = UUID.randomUUID()
client.send(json.executionContextPushRequest(2, contextId, expressionId))
val requestId2 =
runtimeConnectorProbe.receiveN(1).head match {
case Api.Request(
requestId,
Api.PushContextRequest(
`contextId`,
Api.StackItem.LocalCall(`expressionId`)
)
) =>
requestId
case msg =>
fail(s"Unexpected message: $msg")
}
runtimeConnectorProbe.lastSender ! Api.Response(
requestId2,
Api.PushContextResponse(contextId)
)
client.expectJson(json.ok(2))
// recompute
client.send(
json"""
{ "jsonrpc": "2.0",
"method": "executionContext/recompute",
"id": 3,
"params": {
"contextId": $contextId,
"expressionConfigs": [
{ "expressionId": $expressionId,
"executionEnvironment": "Live"
}
]
}
}
"""
)
val requestId3 =
runtimeConnectorProbe.receiveN(1).head match {
case Api.Request(
requestId,
Api.RecomputeContextRequest(
`contextId`,
None,
None,
Seq(
Api.ExpressionConfig(
`expressionId`,
Some(Api.ExecutionEnvironment.Live())
)
)
)
) =>
requestId

View File

@ -69,6 +69,8 @@ public interface IdExecutionService {
* the execution and return the value as a result.
*/
Object onFunctionReturn(Info info);
Object getExecutionEnvironment(Info info);
}
/**

View File

@ -691,6 +691,16 @@ object Runtime {
}
}
/** The configuration of how to execute the expression.
*
* @param expressionId the expression identifier
* @param executionEnvironment the execution environment for the expression
*/
sealed case class ExpressionConfig(
expressionId: ExpressionId,
executionEnvironment: Option[ExecutionEnvironment]
)
/** The notification about the execution status.
*
* @param contextId the context's id
@ -914,12 +924,14 @@ object Runtime {
* @param expressions the selector specifying which expressions should be
* recomputed.
* @param executionEnvironment the environment used for execution
* @param expressionConfigs execution configurations for selected expressions
*/
@named("recomputeContextRequest")
final case class RecomputeContextRequest(
contextId: ContextId,
expressions: Option[InvalidatedExpressions],
executionEnvironment: Option[ExecutionEnvironment]
executionEnvironment: Option[ExecutionEnvironment],
expressionConfigs: Seq[ExpressionConfig]
) extends ApiRequest
/** A response sent from the server upon handling the

View File

@ -0,0 +1,55 @@
package org.enso.interpreter.instrument;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import org.enso.interpreter.runtime.state.ExecutionEnvironment;
import org.enso.polyglot.runtime.Runtime$Api$ExecutionEnvironment;
import org.enso.polyglot.runtime.Runtime$Api$ExpressionConfig;
import scala.Option;
import scala.collection.immutable.Seq;
/**
* The program execution config.
*
* @param executionEnvironment the global execution environment of the program
* @param expressionConfigs execution configs for each expression
*/
public record ExecutionConfig(
ExecutionEnvironment executionEnvironment, Map<UUID, ExecutionEnvironment> expressionConfigs) {
public static ExecutionConfig empty() {
return new ExecutionConfig(null, Collections.emptyMap());
}
@SuppressWarnings("unchecked")
public static ExecutionConfig create(
Object executionEnvironmentOption1, Object expressionConfigs1) {
Map<UUID, ExecutionEnvironment> expressionConfigsBuilder = new HashMap<>();
Option<Runtime$Api$ExecutionEnvironment> executionEnvironmentOption =
(Option<Runtime$Api$ExecutionEnvironment>) executionEnvironmentOption1;
Seq<Runtime$Api$ExpressionConfig> expressionConfigs =
(Seq<Runtime$Api$ExpressionConfig>) expressionConfigs1;
expressionConfigs.foreach(
expressionConfig -> {
expressionConfig
.executionEnvironment()
.foreach(
executionEnvironment -> {
expressionConfigsBuilder.put(
expressionConfig.expressionId(),
ExecutionEnvironment.forName(executionEnvironment.name()));
return null;
});
return null;
});
ExecutionEnvironment executionEnvironment =
executionEnvironmentOption
.map(env -> ExecutionEnvironment.forName(env.name()))
.getOrElse(() -> null);
return new ExecutionConfig(executionEnvironment, expressionConfigsBuilder);
}
}

View File

@ -0,0 +1,31 @@
package org.enso.interpreter.instrument;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import org.enso.interpreter.runtime.state.ExecutionEnvironment;
public final class ExpressionExecutionState {
private final Map<UUID, ExecutionEnvironment> expressionConfigs;
public ExpressionExecutionState() {
this.expressionConfigs = new HashMap<>();
}
public ExpressionExecutionState(Map<UUID, ExecutionEnvironment> expressionConfigs) {
this.expressionConfigs = expressionConfigs;
}
public void setExpressionConfigs(Map<UUID, ExecutionEnvironment> expressionConfigs) {
this.expressionConfigs.putAll(expressionConfigs);
}
public void setExpressionExecuted(UUID expressionId) {
expressionConfigs.remove(expressionId);
}
public ExecutionEnvironment getExecutionEnvironment(UUID expressionId) {
return expressionConfigs.get(expressionId);
}
}

View File

@ -48,7 +48,7 @@ public class SetExecutionEnvironmentCommand extends AsynchronousCommand {
this.getClass(),
() -> {
var oldEnvironmentName =
ctx.executionService().getContext().getExecutionEnvironment().getName();
ctx.executionService().getContext().getGlobalExecutionEnvironment().getName();
if (!oldEnvironmentName.equals(executionEnvironment.name())) {
ctx.jobControlPlane()
.abortJobs(
@ -58,10 +58,21 @@ public class SetExecutionEnvironmentCommand extends AsynchronousCommand {
this.getClass(),
() -> {
Stack<InstrumentFrame> stack = ctx.contextManager().getStack(contextId);
ctx.executionService()
.getContext()
.setExecutionEnvironment(
ExecutionEnvironment.forName(executionEnvironment.name()));
ctx.state()
.executionHooks()
.add(
() ->
ctx.locking()
.withWriteCompilationLock(
this.getClass(),
() -> {
ctx.executionService()
.getContext()
.setExecutionEnvironment(
ExecutionEnvironment.forName(
executionEnvironment.name()));
return null;
}));
CacheInvalidation.invalidateAll(stack);
ctx.jobProcessor().run(ExecuteJob.apply(contextId, stack.toList()));
reply(new Runtime$Api$SetExecutionEnvironmentResponse(contextId), ctx);

View File

@ -5,6 +5,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.function.Consumer;
import org.enso.interpreter.instrument.ExpressionExecutionState;
import org.enso.interpreter.instrument.MethodCallsCache;
import org.enso.interpreter.instrument.OneshotExpression;
import org.enso.interpreter.instrument.RuntimeCache;
@ -31,6 +32,7 @@ final class ExecutionCallbacks implements IdExecutionService.Callbacks {
private final MethodCallsCache methodCallsCache;
private final UpdatesSynchronizationState syncState;
private final Map<UUID, FunctionCallInfo> calls = new HashMap<>();
private final ExpressionExecutionState expressionExecutionState;
private final Consumer<ExpressionValue> onCachedCallback;
private final Consumer<ExpressionValue> onComputedCallback;
private final Consumer<ExpressionCall> functionCallCallback;
@ -39,13 +41,16 @@ final class ExecutionCallbacks implements IdExecutionService.Callbacks {
/**
* Creates callbacks instance.
*
* @param visualizationHolder the holder of all visualizations attached to an execution context.
* @param nextExecutionItem the next item scheduled for execution.
* @param cache the precomputed expression values.
* @param methodCallsCache the storage tracking the executed updateCachedResult calls.
* @param syncState the synchronization state of runtime updates.
* @param nextExecutionItem the next item scheduled for execution.
* @param functionCallCallback the consumer of function call events.
* @param expressionExecutionState the execution state for each expression.
* @param onComputedCallback the consumer of the computed value events.
* @param onCachedCallback the consumer of the cached value events.
* @param functionCallCallback the consumer of function call events.
* @param onExecutedVisualizationCallback the consumer of an executed visualization result.
*/
ExecutionCallbacks(
VisualizationHolder visualizationHolder,
@ -53,6 +58,7 @@ final class ExecutionCallbacks implements IdExecutionService.Callbacks {
RuntimeCache cache,
MethodCallsCache methodCallsCache,
UpdatesSynchronizationState syncState,
ExpressionExecutionState expressionExecutionState,
Consumer<ExpressionValue> onCachedCallback,
Consumer<ExpressionValue> onComputedCallback,
Consumer<ExpressionCall> functionCallCallback,
@ -62,6 +68,7 @@ final class ExecutionCallbacks implements IdExecutionService.Callbacks {
this.cache = cache;
this.methodCallsCache = methodCallsCache;
this.syncState = syncState;
this.expressionExecutionState = expressionExecutionState;
this.onCachedCallback = onCachedCallback;
this.onComputedCallback = onComputedCallback;
this.functionCallCallback = functionCallCallback;
@ -141,6 +148,12 @@ final class ExecutionCallbacks implements IdExecutionService.Callbacks {
return null;
}
@Override
@CompilerDirectives.TruffleBoundary
public Object getExecutionEnvironment(IdExecutionService.Info info) {
return expressionExecutionState.getExecutionEnvironment(info.getId());
}
@CompilerDirectives.TruffleBoundary
private void callOnComputedCallback(ExpressionValue expressionValue) {
onComputedCallback.accept(expressionValue);

View File

@ -29,6 +29,7 @@ import org.enso.common.LanguageInfo;
import org.enso.common.MethodNames;
import org.enso.compiler.suggestions.SimpleUpdate;
import org.enso.interpreter.instrument.Endpoint;
import org.enso.interpreter.instrument.ExpressionExecutionState;
import org.enso.interpreter.instrument.MethodCallsCache;
import org.enso.interpreter.instrument.RuntimeCache;
import org.enso.interpreter.instrument.UpdatesSynchronizationState;
@ -157,9 +158,11 @@ public final class ExecutionService {
* @param methodCallsCache the storage tracking the executed method calls.
* @param syncState the synchronization state of runtime updates.
* @param nextExecutionItem the next item scheduled for execution.
* @param expressionExecutionState the execution state for each expression.
* @param funCallCallback the consumer for function call events.
* @param onComputedCallback the consumer of the computed value events.
* @param onCachedCallback the consumer of the cached value events.
* @param onExecutedVisualizationCallback the consumer of an executed visualization result.
*/
public void execute(
VisualizationHolder visualizationHolder,
@ -169,6 +172,7 @@ public final class ExecutionService {
MethodCallsCache methodCallsCache,
UpdatesSynchronizationState syncState,
UUID nextExecutionItem,
ExpressionExecutionState expressionExecutionState,
Consumer<ExecutionService.ExpressionCall> funCallCallback,
Consumer<ExecutionService.ExpressionValue> onComputedCallback,
Consumer<ExecutionService.ExpressionValue> onCachedCallback,
@ -188,6 +192,7 @@ public final class ExecutionService {
cache,
methodCallsCache,
syncState,
expressionExecutionState,
onCachedCallback,
onComputedCallback,
funCallCallback,
@ -220,9 +225,11 @@ public final class ExecutionService {
* @param methodCallsCache the storage tracking the executed method calls.
* @param syncState the synchronization state of runtime updates.
* @param nextExecutionItem the next item scheduled for execution.
* @param expressionExecutionState the execution state for each expression.
* @param funCallCallback the consumer for function call events.
* @param onComputedCallback the consumer of the computed value events.
* @param onCachedCallback the consumer of the cached value events.
* @param onExecutedVisualizationCallback the consumer of an executed visualization result.
*/
public void execute(
String moduleName,
@ -233,6 +240,7 @@ public final class ExecutionService {
MethodCallsCache methodCallsCache,
UpdatesSynchronizationState syncState,
UUID nextExecutionItem,
ExpressionExecutionState expressionExecutionState,
Consumer<ExecutionService.ExpressionCall> funCallCallback,
Consumer<ExecutionService.ExpressionValue> onComputedCallback,
Consumer<ExecutionService.ExpressionValue> onCachedCallback,
@ -255,6 +263,7 @@ public final class ExecutionService {
methodCallsCache,
syncState,
nextExecutionItem,
expressionExecutionState,
funCallCallback,
onComputedCallback,
onCachedCallback,
@ -360,6 +369,7 @@ public final class ExecutionService {
Consumer<ExpressionValue> onCachedCallback =
(value) -> context.getLogger().finest("_ON_CACHED_VALUE " + value.getExpressionId());
Consumer<ExecutedVisualization> onExecutedVisualizationCallback = (value) -> {};
ExpressionExecutionState expressionExecutionState = new ExpressionExecutionState();
var callbacks =
new ExecutionCallbacks(
@ -368,6 +378,7 @@ public final class ExecutionService {
cache,
methodCallsCache,
syncState,
expressionExecutionState,
onCachedCallback,
onComputedCallback,
funCallCallback,

View File

@ -1,13 +1,20 @@
package org.enso.interpreter.instrument.command
import org.enso.interpreter.instrument.{CacheInvalidation, InstrumentFrame}
import org.enso.compiler.core.Implicits.AsMetadata
import org.enso.compiler.pass.analyse.DataflowAnalysis
import org.enso.compiler.refactoring.IRUtils
import org.enso.interpreter.instrument.command.RecomputeContextCmd.InvalidateExpressions
import org.enso.interpreter.instrument.{
CacheInvalidation,
ExecutionConfig,
InstrumentFrame
}
import org.enso.interpreter.instrument.execution.RuntimeContext
import org.enso.interpreter.instrument.job.{EnsureCompiledJob, ExecuteJob}
import org.enso.polyglot.runtime.Runtime.Api
import org.enso.polyglot.runtime.Runtime.Api.RequestId
import scala.concurrent.{ExecutionContext, Future}
import scala.jdk.CollectionConverters._
/** A command that forces a recomputation of the current position.
*
@ -41,56 +48,19 @@ class RecomputeContextCmd(
reply(Api.EmptyStackError(request.contextId))
false
} else {
val cacheInvalidationCommands = request.expressions.toSeq
.map(CacheInvalidation.Command(_))
.map(CacheInvalidation(CacheInvalidation.StackSelector.Top, _))
CacheInvalidation.runAll(
stack,
cacheInvalidationCommands
ctx.state.executionHooks.add(
InvalidateExpressions(
request.contextId,
request.expressions,
request.expressionConfigs
)
)
sendPendingUpdates(stack)
reply(Api.RecomputeContextResponse(request.contextId))
true
}
}
}
private def sendPendingUpdates(
stack: Iterable[InstrumentFrame]
)(implicit ctx: RuntimeContext): Unit = {
val invalidatedExpressions =
request.expressions
.map {
case expressions: Api.InvalidatedExpressions.Expressions =>
expressions.value.toSet
case _: Api.InvalidatedExpressions.All =>
stack.headOption
.map { frame =>
frame.cache.getWeights.keySet().asScala.toSet
}
.getOrElse(Set())
}
.getOrElse(Set())
if (invalidatedExpressions.nonEmpty) {
val updates = invalidatedExpressions.collect {
case expressionId if expressionId ne null =>
Api.ExpressionUpdate(
expressionId,
None,
None,
Vector.empty,
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
}
ctx.endpoint.sendToClient(
Api.Response(Api.ExpressionUpdates(request.contextId, updates))
)
}
}
private def doesContextExist(implicit ctx: RuntimeContext): Boolean = {
ctx.contextManager.contains(request.contextId)
}
@ -109,6 +79,13 @@ class RecomputeContextCmd(
): Future[Unit] = {
if (isStackNonEmpty) {
val stack = ctx.contextManager.getStack(request.contextId)
val executionConfig =
ExecutionConfig.create(
request.executionEnvironment,
request.expressionConfigs
)
ctx.state.expressionExecutionState
.setExpressionConfigs(executionConfig.expressionConfigs)
for {
_ <- ctx.jobProcessor.run(EnsureCompiledJob(stack))
_ <- ctx.jobProcessor.run(
@ -125,3 +102,114 @@ class RecomputeContextCmd(
}
}
object RecomputeContextCmd {
/** Invalidate caches for the request. */
sealed private case class InvalidateExpressions(
contextId: Api.ContextId,
expressions: Option[Api.InvalidatedExpressions],
expressionConfigs: Seq[Api.ExpressionConfig]
)(implicit ctx: RuntimeContext)
extends Runnable {
override def run(): Unit = {
val stack = ctx.contextManager.getStack(contextId)
val invalidationCommands =
ctx.locking.withWriteCompilationLock(
classOf[RecomputeContextCmd],
() => {
val expressionsInvalidationCommands = expressions.toSeq
.map(CacheInvalidation.Command(_))
.map(CacheInvalidation(CacheInvalidation.StackSelector.All, _))
val expressionConfigsDependentInvalidationCommands =
expressionConfigs
.map(_.expressionId)
.flatMap(RecomputeContextCmd.invalidateDependent)
val allInvalidationCommands =
expressionsInvalidationCommands ++ expressionConfigsDependentInvalidationCommands
CacheInvalidation.runAll(stack, allInvalidationCommands)
allInvalidationCommands
}
)
sendPendingUpdates(stack, contextId, invalidationCommands)
}
}
/** Invalidate dependent nodes of the provided expression.
*
* @param expressionId the expression id
* @return commands to invalidate dependent nodes of the provided expression
*/
private def invalidateDependent(
expressionId: Api.ExpressionId
)(implicit ctx: RuntimeContext): Seq[CacheInvalidation] = {
val builder = Vector.newBuilder[CacheInvalidation]
ctx.executionService.getContext
.findModuleByExpressionId(expressionId)
.ifPresent { module =>
module.getIr
.getMetadata(DataflowAnalysis)
.foreach { metadata =>
val dependents =
IRUtils
.findByExternalId(module.getIr, expressionId)
.map { ir =>
DataflowAnalysis.DependencyInfo.Type
.Static(ir.getId, ir.getExternalId)
}
.flatMap { expressionKey =>
metadata.dependents.getExternal(expressionKey)
}
.fold(Set(expressionId))(_ + expressionId)
builder += CacheInvalidation(
CacheInvalidation.StackSelector.All,
CacheInvalidation.Command.InvalidateKeys(dependents)
)
}
}
builder.result()
}
private def sendPendingUpdates(
stack: Iterable[InstrumentFrame],
contextId: Api.ContextId,
cacheInvalidations: Seq[CacheInvalidation]
)(implicit ctx: RuntimeContext): Unit = {
val builder = Set.newBuilder[Api.ExpressionId]
cacheInvalidations.map(_.command).foreach {
case CacheInvalidation.Command.InvalidateAll =>
stack.headOption
.map { frame =>
frame.cache.getWeights.keySet().forEach(builder.addOne)
}
case CacheInvalidation.Command.InvalidateKeys(expressionIds) =>
builder ++= expressionIds
case _ =>
}
val invalidatedExpressions = builder.result()
if (invalidatedExpressions.nonEmpty) {
val updates = invalidatedExpressions.collect {
case expressionId if expressionId ne null =>
Api.ExpressionUpdate(
expressionId,
None,
None,
Vector.empty,
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
}
ctx.endpoint.sendToClient(
Api.Response(Api.ExpressionUpdates(contextId, updates))
)
}
}
}

View File

@ -1,5 +1,7 @@
package org.enso.interpreter.instrument.execution
import org.enso.interpreter.instrument.ExpressionExecutionState
/** The state of the runtime */
final class ExecutionState {
@ -8,5 +10,7 @@ final class ExecutionState {
val executionHooks: ExecutionHooks = new RuntimeExecutionHooks
val expressionExecutionState = new ExpressionExecutionState()
val suggestions: ModuleIndexing = ModuleIndexing.createInstance()
}

View File

@ -70,7 +70,9 @@ class ExecuteJob(
() => {
val context = ctx.executionService.getContext
val originalExecutionEnvironment =
executionEnvironment.map(_ => context.getExecutionEnvironment)
executionEnvironment.map(_ =>
context.getGlobalExecutionEnvironment
)
executionEnvironment.foreach(env =>
context.setExecutionEnvironment(
ExecutionEnvironment.forName(env.name)

View File

@ -49,6 +49,7 @@ import java.io.File
import java.util.UUID
import java.util.function.Consumer
import java.util.logging.Level
import scala.jdk.OptionConverters.RichOptional
import scala.util.Try
@ -57,7 +58,7 @@ import scala.util.Try
*/
object ProgramExecutionSupport {
/** Runs an Enso program.
/** Runs the program.
*
* @param contextId an identifier of an execution context
* @param executionFrame an execution frame
@ -142,6 +143,7 @@ object ProgramExecutionSupport {
methodCallsCache,
syncState,
callStack.headOption.map(_.expressionId).orNull,
ctx.state.expressionExecutionState,
callablesCallback,
onComputedValueCallback,
onCachedValueCallback,
@ -183,6 +185,7 @@ object ProgramExecutionSupport {
methodCallsCache,
syncState,
callStack.headOption.map(_.expressionId).orNull,
ctx.state.expressionExecutionState,
callablesCallback,
onComputedValueCallback,
onCachedValueCallback,
@ -226,7 +229,7 @@ object ProgramExecutionSupport {
}
}
/** Runs an Enso program.
/** Runs the program.
*
* @param contextId an identifier of an execution context
* @param stack a call stack
@ -271,7 +274,9 @@ object ProgramExecutionSupport {
Some(Api.ExecutionResult.Failure("Execution stack is empty.", None))
)
_ <-
Try(executeProgram(contextId, stackItem, localCalls)).toEither.left
Try(
executeProgram(contextId, stackItem, localCalls)
).toEither.left
.map(onExecutionError(stackItem.item, _))
} yield ()
logger.log(Level.FINEST, s"Execution finished: $executionResult")
@ -514,6 +519,7 @@ object ProgramExecutionSupport {
}
syncState.setExpressionSync(expressionId)
ctx.state.expressionExecutionState.setExpressionExecuted(expressionId)
if (methodCall.isDefined) {
syncState.setMethodPointerSync(expressionId)
}

View File

@ -38,6 +38,7 @@ import org.enso.interpreter.runtime.error.DataflowError;
import org.enso.interpreter.runtime.error.PanicException;
import org.enso.interpreter.runtime.error.PanicSentinel;
import org.enso.interpreter.runtime.instrument.Timer;
import org.enso.interpreter.runtime.state.ExecutionEnvironment;
import org.enso.interpreter.runtime.state.State;
import org.enso.interpreter.runtime.tag.AvoidIdInstrumentationTag;
import org.enso.interpreter.runtime.tag.IdentifiedTag;
@ -182,6 +183,7 @@ public class IdExecutionInstrument extends TruffleInstrument implements IdExecut
private final EventContext context;
private long nanoTimeElapsed = 0;
private ExecutionEnvironment originalExecutionEnvironment = null;
/**
* Creates a new event node.
@ -209,6 +211,8 @@ public class IdExecutionInstrument extends TruffleInstrument implements IdExecut
if (result != null) {
throw context.createUnwind(result);
}
setExecutionEnvironment(info);
nanoTimeElapsed = timer.getTime();
}
@ -250,10 +254,13 @@ public class IdExecutionInstrument extends TruffleInstrument implements IdExecut
frame == null ? null : frame.materialize(),
node);
callbacks.updateCachedResult(info);
resetExecutionEnvironment();
if (info.isPanic()) {
throw context.createUnwind(result);
}
} else if (node instanceof ExpressionNode) {
resetExecutionEnvironment();
}
}
@ -312,6 +319,23 @@ public class IdExecutionInstrument extends TruffleInstrument implements IdExecut
});
return result == null;
}
private void setExecutionEnvironment(IdExecutionService.Info info) {
ExecutionEnvironment nodeEnvironment =
(ExecutionEnvironment) callbacks.getExecutionEnvironment(info);
if (nodeEnvironment != null && originalExecutionEnvironment == null) {
EnsoContext context = EnsoContext.get(this);
originalExecutionEnvironment = context.getGlobalExecutionEnvironment();
context.setExecutionEnvironment(nodeEnvironment);
}
}
private void resetExecutionEnvironment() {
if (originalExecutionEnvironment != null) {
EnsoContext.get(this).setExecutionEnvironment(originalExecutionEnvironment);
originalExecutionEnvironment = null;
}
}
}
}

View File

@ -12,6 +12,7 @@ import org.enso.polyglot.runtime.Runtime.Api.{
MethodPointer
}
import org.enso.text.{ContentVersion, Sha3_224VersionCalculator}
import org.enso.text.editing.model
import org.graalvm.polyglot.Context
import org.scalatest.BeforeAndAfterEach
import org.scalatest.flatspec.AnyFlatSpec
@ -262,6 +263,153 @@ class RuntimeAsyncCommandsTest
diagnostic.stack should not be empty
}
it should "recompute expression in context after interruption" in {
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val metadata = new Metadata
val idOut = metadata.addItem(179, 17, "aa")
val code =
"""from Standard.Base import all
|from Standard.Base.Runtime.Context import Input, Output
|polyglot java import java.lang.Thread
|
|main =
| IO.println "started"
| loop 50
| out = Output.is_enabled
| IO.println out
|
|loop n=0 s=0 =
| if (s > n) then s else
| Thread.sleep 100
| loop n s+1
|""".stripMargin.linesIterator.mkString("\n")
val contents = metadata.appendToCode(code)
val mainFile = context.writeMain(contents)
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Open the new file
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, contents))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(
3
) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(
contextId,
idOut,
ConstantsGen.BOOLEAN,
methodCall = Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Runtime",
"Standard.Base.Runtime.Context",
"is_enabled"
),
Vector(1)
)
)
),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("started", "False")
// recompute
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(
contextId,
None,
None,
Seq(
Api.ExpressionConfig(idOut, Some(Api.ExecutionEnvironment.Live()))
)
)
)
)
val responses = context.receiveNIgnorePendingExpressionUpdates(1)
responses should contain(
Api.Response(requestId, Api.RecomputeContextResponse(contextId))
)
// wait for program to start and interrupt
var isProgramStarted = false
var iteration = 0
while (!isProgramStarted && iteration < 100) {
val out = context.consumeOut
Thread.sleep(200)
isProgramStarted = out == List("started")
iteration += 1
}
if (!isProgramStarted) {
fail("Program start timed out")
}
context.consumeOut shouldEqual List()
// trigger re-computation
context.send(
Api.Request(
Api.EditFileNotification(
mainFile,
Seq(
model.TextEdit(
model.Range(model.Position(10, 7), model.Position(10, 8)),
"0"
)
),
execute = true,
idMap = None
)
)
)
val responses1 = context.receiveNIgnorePendingExpressionUpdates(3)
responses1 should contain allOf (
TestMessages.update(
contextId,
idOut,
ConstantsGen.BOOLEAN,
fromCache = false,
typeChanged = false,
methodCall = Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Runtime",
"Standard.Base.Runtime.Context",
"is_enabled"
),
Vector(1)
)
)
),
context.executionComplete(contextId)
)
context.consumeOut should contain("True")
}
it should "interrupt running execution context without sending Panic in expression updates" in {
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
@ -330,7 +478,10 @@ class RuntimeAsyncCommandsTest
// recompute/interrupt
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
val responses = context.receiveNIgnoreStdLib(
3
@ -486,7 +637,8 @@ class RuntimeAsyncCommandsTest
Api.RecomputeContextRequest(
contextId,
Some(InvalidatedExpressions.Expressions(Vector(idOp1, idOp2))),
None
None,
Seq()
)
)
)

View File

@ -196,7 +196,7 @@ class RuntimeExecutionEnvironmentTest
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List()
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
@ -232,7 +232,7 @@ class RuntimeExecutionEnvironmentTest
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Live()
.name
}
@ -300,7 +300,7 @@ class RuntimeExecutionEnvironmentTest
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List()
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
@ -318,7 +318,7 @@ class RuntimeExecutionEnvironmentTest
context.receiveNIgnoreStdLib(1) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.SetExecutionEnvironmentResponse(contextId))
)
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
@ -353,7 +353,7 @@ class RuntimeExecutionEnvironmentTest
),
context.executionComplete(contextId)
)
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Live()
.name
}

View File

@ -33,7 +33,8 @@ class RuntimeServerTest
var context: TestContext = _
class TestContext(packageName: String)
extends InstrumentTestContext(packageName) {
extends InstrumentTestContext(packageName)
with RuntimeServerTest.TestMain {
val out: ByteArrayOutputStream = new ByteArrayOutputStream()
val logOut: ByteArrayOutputStream = new ByteArrayOutputStream()
@ -113,286 +114,6 @@ class RuntimeServerTest
def executionComplete(contextId: UUID): Api.Response =
Api.Response(Api.ExecutionComplete(contextId))
// === The Tests ==========================================================
object Main {
val metadata = new Metadata
val idMainX = metadata.addItem(63, 1, "aa1")
val idMainY = metadata.addItem(73, 7, "aa2")
val idMainZ = metadata.addItem(89, 5, "aa3")
val idFooY = metadata.addItem(133, 8, "ff2")
val idFooZ = metadata.addItem(150, 5, "ff3")
def code =
metadata.appendToCode(
"""
|from Standard.Base.Data.Numbers import all
|
|main =
| x = 6
| y = x.foo 5
| z = y + 5
| z
|
|Number.foo self = x ->
| y = self + 3
| z = y * x
| z
|""".stripMargin.linesIterator.mkString("\n")
)
object Update {
def mainX(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainX,
Some(ConstantsGen.INTEGER),
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def pendingZ(): Api.ExpressionUpdate =
Api.ExpressionUpdate(
Main.idFooZ,
None,
None,
Vector(),
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
def pendingY(): Api.ExpressionUpdate =
Api.ExpressionUpdate(
Main.idFooY,
None,
None,
Vector(),
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
def mainY(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
ConstantsGen.NUMBER,
"foo"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def mainZ(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"+"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def fooY(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idFooY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"+"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def fooZ(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idFooZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"*"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
}
}
object Main2 {
val metadata = new Metadata
val idMainY = metadata.addItem(178, 5)
val idMainZ = metadata.addItem(192, 5)
val code = metadata.appendToCode(
"""from Standard.Base import all
|
|foo = arg ->
| IO.println "I'm expensive!"
| arg + 5
|
|bar = arg ->
| IO.println "I'm more expensive!"
| arg * 5
|
|main =
| x = 10
| y = foo x
| z = bar y
| z
|""".stripMargin.linesIterator.mkString("\n")
)
object Update {
def mainY(contextId: UUID) =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
idMainY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
"Enso_Test.Test.Main",
"foo"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
true,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def mainZ(contextId: UUID) =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
idMainZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
"Enso_Test.Test.Main",
"bar"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
true,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
}
}
}
override protected def beforeEach(): Unit = {
@ -5011,251 +4732,6 @@ class RuntimeServerTest
context.consumeOut shouldEqual List()
}
it should "recompute expressions without invalidation" in {
val contents = context.Main.code
val mainFile = context.writeMain(contents)
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Set sources for the module
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, contents))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
// recompute
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
)
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
context.executionComplete(contextId)
)
}
it should "recompute expressions invalidating all" in {
val contents = context.Main.code
val mainFile = context.writeMain(contents)
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Set sources for the module
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, contents))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
// recompute
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(
contextId,
Some(Api.InvalidatedExpressions.All()),
None
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
TestMessages.pending(
contextId,
context.Main.idMainX,
context.Main.idMainY,
context.Main.idMainZ,
context.Main.idFooY,
context.Main.idFooZ
),
context.Main.Update.mainX(contextId, typeChanged = false),
context.Main.Update.mainY(contextId, typeChanged = false),
context.Main.Update.mainZ(contextId, typeChanged = false),
context.executionComplete(contextId)
)
}
it should "recompute expressions invalidating some" in {
val contents = context.Main.code
val mainFile = context.writeMain(contents)
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Set sources for the module
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, contents))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
context.receiveNone shouldEqual None
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
// recompute
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(
contextId,
Some(
Api.InvalidatedExpressions.Expressions(Vector(context.Main.idMainZ))
),
None
)
)
)
context.receiveN(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
TestMessages.pending(contextId, context.Main.idMainZ),
context.Main.Update.mainZ(contextId, typeChanged = false),
context.executionComplete(contextId)
)
}
it should "recompute expressions changing an execution environment" in {
val contents = context.Main.code
val mainFile = context.writeMain(contents)
val moduleName = "Enso_Test.Test.Main"
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
// create context
context.send(Api.Request(requestId, Api.CreateContextRequest(contextId)))
context.receive shouldEqual Some(
Api.Response(requestId, Api.CreateContextResponse(contextId))
)
// Set sources for the module
context.send(
Api.Request(requestId, Api.OpenFileRequest(mainFile, contents))
)
context.receive shouldEqual Some(
Api.Response(Some(requestId), Api.OpenFileResponse)
)
// push main
val item1 = Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, moduleName, "main"),
None,
Vector()
)
context.send(
Api.Request(requestId, Api.PushContextRequest(contextId, item1))
)
context.receiveNIgnoreStdLib(5) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
context.Main.Update.mainX(contextId),
context.Main.Update.mainY(contextId),
context.Main.Update.mainZ(contextId),
context.executionComplete(contextId)
)
// recompute
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(
contextId,
Some(Api.InvalidatedExpressions.All()),
Some(Api.ExecutionEnvironment.Live())
)
)
)
context.receiveN(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
TestMessages.pending(
contextId,
context.Main.idMainX,
context.Main.idMainY,
context.Main.idMainZ,
context.Main.idFooY,
context.Main.idFooZ
),
context.Main.Update.mainX(contextId, typeChanged = false),
context.Main.Update.mainY(contextId, typeChanged = false),
context.Main.Update.mainZ(contextId, typeChanged = false),
context.executionComplete(contextId)
)
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
}
it should "return error when module not found" in {
val contents = context.Main.code
val mainFile = context.writeMain(context.Main.code)
@ -6667,7 +6143,10 @@ class RuntimeServerTest
// recompute
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
@ -6757,7 +6236,10 @@ class RuntimeServerTest
// recompute existing stack
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
context.receiveN(2) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.RecomputeContextResponse(contextId)),
@ -6771,7 +6253,8 @@ class RuntimeServerTest
Api.RecomputeContextRequest(
contextId,
Some(Api.InvalidatedExpressions.All()),
None
None,
Seq()
)
)
)
@ -7468,7 +6951,7 @@ class RuntimeServerTest
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Design()
.name
@ -7489,7 +6972,7 @@ class RuntimeServerTest
context.executionComplete(contextId)
)
context.consumeOut shouldEqual List("Hello World!")
context.languageContext.getExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
context.languageContext.getGlobalExecutionEnvironment.getName shouldEqual Api.ExecutionEnvironment
.Live()
.name
}
@ -7724,3 +7207,284 @@ class RuntimeServerTest
}
}
object RuntimeServerTest {
trait TestMain {
object Main {
val metadata = new Metadata
val idMainX = metadata.addItem(63, 1, "aa1")
val idMainY = metadata.addItem(73, 7, "aa2")
val idMainZ = metadata.addItem(89, 5, "aa3")
val idFooY = metadata.addItem(133, 8, "ff2")
val idFooZ = metadata.addItem(150, 5, "ff3")
def code =
metadata.appendToCode(
"""
|from Standard.Base.Data.Numbers import all
|
|main =
| x = 6
| y = x.foo 5
| z = y + 5
| z
|
|Number.foo self = x ->
| y = self + 3
| z = y * x
| z
|""".stripMargin.linesIterator.mkString("\n")
)
object Update {
def mainX(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainX,
Some(ConstantsGen.INTEGER),
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def pendingZ(): Api.ExpressionUpdate =
Api.ExpressionUpdate(
Main.idFooZ,
None,
None,
Vector(),
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
def pendingY(): Api.ExpressionUpdate =
Api.ExpressionUpdate(
Main.idFooY,
None,
None,
Vector(),
true,
false,
Api.ExpressionUpdate.Payload.Pending(None, None)
)
def mainY(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
ConstantsGen.NUMBER,
"foo"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def mainZ(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idMainZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"+"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def fooY(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idFooY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"+"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def fooZ(
contextId: UUID,
fromCache: Boolean = false,
typeChanged: Boolean = true
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Main.idFooZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Standard.Base.Data.Numbers",
ConstantsGen.INTEGER,
"*"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
typeChanged,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
}
}
object Main2 {
val metadata = new Metadata
val idMainY = metadata.addItem(178, 5)
val idMainZ = metadata.addItem(192, 5)
val code = metadata.appendToCode(
"""from Standard.Base import all
|
|foo = arg ->
| IO.println "I'm expensive!"
| arg + 5
|
|bar = arg ->
| IO.println "I'm more expensive!"
| arg * 5
|
|main =
| x = 10
| y = foo x
| z = bar y
| z
|""".stripMargin.linesIterator.mkString("\n")
)
object Update {
def mainY(contextId: UUID) =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
idMainY,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
"Enso_Test.Test.Main",
"foo"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
true,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
def mainZ(contextId: UUID) =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
idMainZ,
Some(ConstantsGen.INTEGER),
Some(
Api.MethodCall(
Api.MethodPointer(
"Enso_Test.Test.Main",
"Enso_Test.Test.Main",
"bar"
)
)
),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
true,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
}
}
}
}

View File

@ -1414,7 +1414,10 @@ class RuntimeSuggestionUpdatesTest
// recompute
context.send(
Api.Request(requestId, Api.RecomputeContextRequest(contextId, None, None))
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
val updates2 = context.receiveNIgnoreExpressionUpdates(3)
updates2.length shouldEqual 3

View File

@ -423,7 +423,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
@ -551,7 +551,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
context.receiveNIgnoreExpressionUpdates(2) should contain allOf (
@ -570,7 +570,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
Vector(context.Main.idMainX)
)
),
None
None,
Seq()
)
)
)
@ -1577,7 +1578,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
context.receiveNIgnoreExpressionUpdates(
@ -1598,7 +1599,8 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
Vector(context.Main.idMainX)
)
),
None
None,
Seq()
)
)
)
@ -2905,7 +2907,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
@ -3079,7 +3081,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
@ -3218,7 +3220,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)
@ -3402,7 +3404,7 @@ class RuntimeVisualizationsTest extends AnyFlatSpec with Matchers {
context.send(
Api.Request(
requestId,
Api.RecomputeContextRequest(contextId, None, None)
Api.RecomputeContextRequest(contextId, None, None, Seq())
)
)

View File

@ -1,6 +1,7 @@
package org.enso.interpreter;
import com.oracle.truffle.api.CallTarget;
import com.oracle.truffle.api.ContextThreadLocal;
import com.oracle.truffle.api.Option;
import com.oracle.truffle.api.TruffleLanguage;
import com.oracle.truffle.api.TruffleLogger;
@ -87,6 +88,9 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
private static final LanguageReference<EnsoLanguage> REFERENCE =
LanguageReference.create(EnsoLanguage.class);
private final ContextThreadLocal<ExecutionEnvironment[]> executionEnvironment =
locals.createContextThreadLocal((ctx, thread) -> new ExecutionEnvironment[1]);
public static EnsoLanguage get(Node node) {
return REFERENCE.get(node);
}
@ -362,6 +366,7 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
}
/** Conversions of primitive values */
@Override
protected Object getLanguageView(EnsoContext context, Object value) {
if (value instanceof Boolean b) {
var bool = context.getBuiltins().bool();
@ -370,4 +375,12 @@ public final class EnsoLanguage extends TruffleLanguage<EnsoContext> {
}
return null;
}
public ExecutionEnvironment getExecutionEnvironment() {
return executionEnvironment.get()[0];
}
public void setExecutionEnvironment(ExecutionEnvironment executionEnvironment) {
this.executionEnvironment.get()[0] = executionEnvironment;
}
}

View File

@ -120,4 +120,9 @@ final class Instrumentor implements EnsoObject, IdExecutionService.Callbacks {
}
return null;
}
@Override
public Object getExecutionEnvironment(IdExecutionService.Info info) {
return null;
}
}

View File

@ -108,7 +108,7 @@ public final class EnsoContext {
private final AtomicLong clock = new AtomicLong();
private final Shape rootStateShape = Shape.newBuilder().layout(State.Container.class).build();
private ExecutionEnvironment executionEnvironment;
private ExecutionEnvironment globalExecutionEnvironment;
private final int warningsLimit;
@ -144,7 +144,7 @@ public final class EnsoContext {
getOption(RuntimeOptions.DISABLE_IR_CACHES_KEY) || isParallelismEnabled;
this.isPrivateCheckDisabled = getOption(RuntimeOptions.DISABLE_PRIVATE_CHECK_KEY);
this.isStaticTypeAnalysisEnabled = getOption(RuntimeOptions.ENABLE_STATIC_ANALYSIS_KEY);
this.executionEnvironment = getOption(EnsoLanguage.EXECUTION_ENVIRONMENT);
this.globalExecutionEnvironment = getOption(EnsoLanguage.EXECUTION_ENVIRONMENT);
this.assertionsEnabled = shouldAssertionsBeEnabled();
this.shouldWaitForPendingSerializationJobs =
getOption(RuntimeOptions.WAIT_FOR_PENDING_SERIALIZATION_JOBS_KEY);
@ -869,13 +869,19 @@ public final class EnsoContext {
return clock.getAndIncrement();
}
public ExecutionEnvironment getGlobalExecutionEnvironment() {
return globalExecutionEnvironment;
}
public ExecutionEnvironment getExecutionEnvironment() {
return executionEnvironment;
ExecutionEnvironment env = language.getExecutionEnvironment();
return env == null ? getGlobalExecutionEnvironment() : env;
}
/** Set the runtime execution environment of this context. */
public void setExecutionEnvironment(ExecutionEnvironment executionEnvironment) {
this.executionEnvironment = executionEnvironment;
this.globalExecutionEnvironment = executionEnvironment;
language.setExecutionEnvironment(executionEnvironment);
}
/**
@ -883,11 +889,12 @@ public final class EnsoContext {
*
* @param context the execution context
* @param environmentName the execution environment name
* @return the execution environment version before modification
*/
public ExecutionEnvironment enableExecutionEnvironment(Atom context, String environmentName) {
ExecutionEnvironment original = executionEnvironment;
if (executionEnvironment.getName().equals(environmentName)) {
executionEnvironment = executionEnvironment.withContextEnabled(context);
ExecutionEnvironment original = globalExecutionEnvironment;
if (original.getName().equals(environmentName)) {
setExecutionEnvironment(original.withContextEnabled(context));
}
return original;
}
@ -897,11 +904,12 @@ public final class EnsoContext {
*
* @param context the execution context
* @param environmentName the execution environment name
* @return the execution environment version before modification
*/
public ExecutionEnvironment disableExecutionEnvironment(Atom context, String environmentName) {
ExecutionEnvironment original = executionEnvironment;
if (executionEnvironment.getName().equals(environmentName)) {
executionEnvironment = executionEnvironment.withContextDisabled(context);
ExecutionEnvironment original = globalExecutionEnvironment;
if (original.getName().equals(environmentName)) {
setExecutionEnvironment(original.withContextDisabled(context));
}
return original;
}

View File

@ -105,7 +105,7 @@ public class ExecutionEnvironment {
case DESIGN_ENVIRONMENT_NAME:
return DESIGN;
default:
throw new RuntimeException("Unsupported Execution Environment `" + name + "`");
throw new IllegalArgumentException("Unsupported Execution Environment `" + name + "`");
}
}
}

View File

@ -13,6 +13,7 @@ export project.Data.Numbers.Number
export project.Data.Numbers.Integer
export project.Data.Vector.Vector
export project.Function.Function
export project.Nothing.Nothing
export project.Polyglot.Java
export project.Polyglot.Polyglot
export project.Runtime

View File

@ -23,13 +23,13 @@ type Context
Output -> "Output"
if_enabled : Any -> Text -> Text -> Boolean -> Any ! Forbidden_Operation
if_enabled self ~action environment="design" disabled_message="The "+self.name+" context is disabled." panic=True =
if_enabled self ~action environment=Runtime.current_execution_environment disabled_message="The "+self.name+" context is disabled." panic=True =
if self.is_enabled environment then action else
error = Forbidden_Operation.Error disabled_message
if panic then Panic.throw error else Error.throw error
is_enabled : Text -> Boolean
is_enabled self environment="design" =
is_enabled self environment=Runtime.current_execution_environment =
self.is_enabled_builtin environment
## PRIVATE