Handle PanicSentinel in Interpreter (#1436)

PR adds the ability to handle runtime exceptions 
in the interpreter and continue the execution 
passing the PanicSentinel value.
This commit is contained in:
Dmitry Bushev 2021-02-02 12:02:00 +03:00 committed by GitHub
parent f2775176f7
commit f0115587b0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 705 additions and 982 deletions

View File

@ -319,7 +319,7 @@ interface ExpressionUpdate {
An information about the computed value. An information about the computed value.
```typescript ```typescript
type ExpressionUpdatePayload = Value | DatafalowError | RuntimeError | Poisoned; type ExpressionUpdatePayload = Value | DatafalowError | Panic;
/** /**
* An empty payload. Indicates that the expression was computed to a value. * An empty payload. Indicates that the expression was computed to a value.
@ -339,7 +339,7 @@ interface DataflowError {
/** /**
* Indicates that the expression failed with the runtime exception. * Indicates that the expression failed with the runtime exception.
*/ */
interface RuntimeError { interface Panic {
/** /**
* The error message. * The error message.
*/ */
@ -350,17 +350,6 @@ interface RuntimeError {
*/ */
trace: ExpressionId[]; trace: ExpressionId[];
} }
/**
* Indicates that the expression was not computed due to a dependency,
* that failed with the runtime exception.
*/
interface Poisoned {
/**
* The list of expressions leading to the root expression that failed.
*/
trace: ExpressionId[];
}
``` ```
### `VisualisationConfiguration` ### `VisualisationConfiguration`

View File

@ -200,12 +200,9 @@ final class ContextEventsListener(
case Api.ExpressionUpdate.Payload.DataflowError(trace) => case Api.ExpressionUpdate.Payload.DataflowError(trace) =>
ContextRegistryProtocol.ExpressionUpdate.Payload.DataflowError(trace) ContextRegistryProtocol.ExpressionUpdate.Payload.DataflowError(trace)
case Api.ExpressionUpdate.Payload.RuntimeError(message, trace) => case Api.ExpressionUpdate.Payload.Panic(message, trace) =>
ContextRegistryProtocol.ExpressionUpdate.Payload ContextRegistryProtocol.ExpressionUpdate.Payload
.RuntimeError(message, trace) .Panic(message, trace)
case Api.ExpressionUpdate.Payload.Poisoned(trace) =>
ContextRegistryProtocol.ExpressionUpdate.Payload.Poisoned(trace)
} }
/** Convert the runtime profiling info to the context registry protocol /** Convert the runtime profiling info to the context registry protocol

View File

@ -143,18 +143,11 @@ object ContextRegistryProtocol {
* @param message the error message * @param message the error message
* @param trace the stack trace * @param trace the stack trace
*/ */
case class RuntimeError( case class Panic(
message: String, message: String,
trace: Seq[UUID] trace: Seq[UUID]
) extends Payload ) extends Payload
/** Indicates that the expression was not computed due to a dependency,
* that failed with the runtime exception.
*
* @param trace the list of expressions leading to the root error.
*/
case class Poisoned(trace: Seq[UUID]) extends Payload
private object CodecField { private object CodecField {
val Type = "type" val Type = "type"
@ -166,9 +159,8 @@ object ContextRegistryProtocol {
val DataflowError = "DataflowError" val DataflowError = "DataflowError"
val RuntimeError = "RuntimeError" val Panic = "Panic"
val Poisoned = "Poisoned"
} }
implicit val encoder: Encoder[Payload] = implicit val encoder: Encoder[Payload] =
@ -183,18 +175,11 @@ object ContextRegistryProtocol {
Json.obj(CodecField.Type -> PayloadType.DataflowError.asJson) Json.obj(CodecField.Type -> PayloadType.DataflowError.asJson)
) )
case m: Payload.RuntimeError => case m: Payload.Panic =>
Encoder[Payload.RuntimeError] Encoder[Payload.Panic]
.apply(m) .apply(m)
.deepMerge( .deepMerge(
Json.obj(CodecField.Type -> PayloadType.RuntimeError.asJson) Json.obj(CodecField.Type -> PayloadType.Panic.asJson)
)
case m: Payload.Poisoned =>
Encoder[Payload.Poisoned]
.apply(m)
.deepMerge(
Json.obj(CodecField.Type -> PayloadType.Poisoned.asJson)
) )
} }
@ -207,11 +192,8 @@ object ContextRegistryProtocol {
case PayloadType.DataflowError => case PayloadType.DataflowError =>
Decoder[Payload.DataflowError].tryDecode(cursor) Decoder[Payload.DataflowError].tryDecode(cursor)
case PayloadType.RuntimeError => case PayloadType.Panic =>
Decoder[Payload.RuntimeError].tryDecode(cursor) Decoder[Payload.Panic].tryDecode(cursor)
case PayloadType.Poisoned =>
Decoder[Payload.Poisoned].tryDecode(cursor)
} }
} }
} }

View File

@ -90,7 +90,7 @@ final class SuggestionsHandler(
override def preStart(): Unit = { override def preStart(): Unit = {
context.system.eventStream context.system.eventStream
.subscribe(self, classOf[Api.ExpressionValuesComputed]) .subscribe(self, classOf[Api.ExpressionUpdates])
context.system.eventStream context.system.eventStream
.subscribe(self, classOf[Api.SuggestionsDatabaseModuleUpdateNotification]) .subscribe(self, classOf[Api.SuggestionsDatabaseModuleUpdateNotification])
context.system.eventStream.subscribe(self, classOf[ProjectNameChangedEvent]) context.system.eventStream.subscribe(self, classOf[ProjectNameChangedEvent])
@ -167,11 +167,11 @@ final class SuggestionsHandler(
) )
} }
case Api.ExpressionValuesComputed(_, updates) => case Api.ExpressionUpdates(_, updates) =>
log.debug( log.debug(
s"ExpressionValuesComputed ${updates.map(u => (u.expressionId, u.expressionType))}" s"ExpressionValuesComputed ${updates.map(u => (u.expressionId, u.expressionType))}"
) )
val types = updates val types = updates.toSeq
.flatMap(update => update.expressionType.map(update.expressionId -> _)) .flatMap(update => update.expressionType.map(update.expressionId -> _))
suggestionsRepo suggestionsRepo
.updateAll(types) .updateAll(types)

View File

@ -181,7 +181,7 @@ class ContextEventsListenerSpec
None, None,
Vector(), Vector(),
false, false,
Api.ExpressionUpdate.Payload.RuntimeError("Method failure", Seq()) Api.ExpressionUpdate.Payload.Panic("Method failure", Seq())
) )
) )
) )
@ -199,47 +199,7 @@ class ContextEventsListenerSpec
Vector(), Vector(),
false, false,
ContextRegistryProtocol.ExpressionUpdate.Payload ContextRegistryProtocol.ExpressionUpdate.Payload
.RuntimeError("Method failure", Seq()) .Panic("Method failure", Seq())
)
),
None
)
)
)
}
"send poisoning error updates" taggedAs Retry in withDb {
(clientId, contextId, _, router, listener) =>
listener ! Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
Suggestions.local.externalId.get,
None,
None,
Vector(),
false,
Api.ExpressionUpdate.Payload.Poisoned(
Seq(Suggestions.method.externalId.get)
)
)
)
)
router.expectMsg(
DeliverToJsonController(
clientId,
ContextRegistryProtocol.ExpressionUpdatesNotification(
contextId,
Vector(
ContextRegistryProtocol.ExpressionUpdate(
Suggestions.local.externalId.get,
None,
None,
Vector(),
false,
ContextRegistryProtocol.ExpressionUpdate.Payload
.Poisoned(Seq(Suggestions.method.externalId.get))
) )
), ),
None None

View File

@ -105,10 +105,6 @@ object Runtime {
value = classOf[Api.VisualisationModified], value = classOf[Api.VisualisationModified],
name = "visualisationModified" name = "visualisationModified"
), ),
new JsonSubTypes.Type(
value = classOf[Api.ExpressionValuesComputed],
name = "expressionValuesComputed"
),
new JsonSubTypes.Type( new JsonSubTypes.Type(
value = classOf[Api.ExpressionUpdates], value = classOf[Api.ExpressionUpdates],
name = "expressionUpdates" name = "expressionUpdates"
@ -251,24 +247,6 @@ object Runtime {
case class LocalCall(expressionId: ExpressionId) extends StackItem case class LocalCall(expressionId: ExpressionId) extends StackItem
} }
/** An update containing information about expression.
*
* @param expressionId expression id.
* @param expressionType the type of expression.
* @param methodCall the pointer to a method definition.
* @param profilingInfo profiling information about the execution of this
* expression
* @param fromCache whether or not the value for this expression came from
* the cache
*/
case class ExpressionValueUpdate(
expressionId: ExpressionId,
expressionType: Option[String],
methodCall: Option[MethodPointer],
profilingInfo: Vector[ProfilingInfo],
fromCache: Boolean
)
/** An update about the computed expression. /** An update about the computed expression.
* *
* @param expressionId the expression id * @param expressionId the expression id
@ -303,12 +281,8 @@ object Runtime {
name = "expressionUpdatePayloadDataflowError" name = "expressionUpdatePayloadDataflowError"
), ),
new JsonSubTypes.Type( new JsonSubTypes.Type(
value = classOf[Payload.RuntimeError], value = classOf[Payload.Panic],
name = "expressionUpdatePayloadRuntimeError" name = "expressionUpdatePayloadPanic"
),
new JsonSubTypes.Type(
value = classOf[Payload.Poisoned],
name = "expressionUpdatePayloadPoisoned"
) )
) )
) )
@ -331,17 +305,11 @@ object Runtime {
* @param message the error message * @param message the error message
* @param trace the stack trace * @param trace the stack trace
*/ */
case class RuntimeError( case class Panic(
message: String, message: String,
trace: Seq[ExpressionId] trace: Seq[ExpressionId]
) extends Payload ) extends Payload
/** Indicates that the expression was not computed due to a dependency,
* that failed with the runtime exception.
*
* @param trace the list of expressions leading to the root error.
*/
case class Poisoned(trace: Seq[ExpressionId]) extends Payload
} }
} }
@ -397,17 +365,6 @@ object Runtime {
extends InvalidatedExpressions extends InvalidatedExpressions
} }
// TODO: [DB] Remove when IDE implements new updates API
/** A notification about updated expressions of the context.
*
* @param contextId the context's id.
* @param updates a list of updates.
*/
case class ExpressionValuesComputed(
contextId: ContextId,
updates: Vector[ExpressionValueUpdate]
) extends ApiNotification
/** A notification about updated expressions of the context. /** A notification about updated expressions of the context.
* *
* @param contextId the context's id. * @param contextId the context's id.

View File

@ -10,7 +10,6 @@ import com.oracle.truffle.api.interop.InteropException;
import com.oracle.truffle.api.interop.InteropLibrary; import com.oracle.truffle.api.interop.InteropLibrary;
import com.oracle.truffle.api.nodes.Node; import com.oracle.truffle.api.nodes.Node;
import com.oracle.truffle.api.nodes.RootNode; import com.oracle.truffle.api.nodes.RootNode;
import java.util.Arrays;
import org.enso.interpreter.instrument.execution.Timer; import org.enso.interpreter.instrument.execution.Timer;
import org.enso.interpreter.instrument.profiling.ExecutionTime; import org.enso.interpreter.instrument.profiling.ExecutionTime;
import org.enso.interpreter.instrument.profiling.ProfilingInfo; import org.enso.interpreter.instrument.profiling.ProfilingInfo;
@ -19,14 +18,13 @@ import org.enso.interpreter.node.ExpressionNode;
import org.enso.interpreter.node.MethodRootNode; import org.enso.interpreter.node.MethodRootNode;
import org.enso.interpreter.node.callable.FunctionCallInstrumentationNode; import org.enso.interpreter.node.callable.FunctionCallInstrumentationNode;
import org.enso.interpreter.runtime.control.TailCallException; import org.enso.interpreter.runtime.control.TailCallException;
import org.enso.interpreter.runtime.error.PanicException;
import org.enso.interpreter.runtime.error.PanicSentinel;
import org.enso.interpreter.runtime.tag.IdentifiedTag; import org.enso.interpreter.runtime.tag.IdentifiedTag;
import org.enso.interpreter.runtime.type.Types; import org.enso.interpreter.runtime.type.Types;
import org.enso.pkg.QualifiedName; import org.enso.pkg.QualifiedName;
import java.util.HashMap; import java.util.*;
import java.util.Map;
import java.util.Objects;
import java.util.UUID;
import java.util.function.Consumer; import java.util.function.Consumer;
/** An instrument for getting values from AST-identified expressions. */ /** An instrument for getting values from AST-identified expressions. */
@ -51,7 +49,8 @@ public class IdExecutionInstrument extends TruffleInstrument {
this.env = env; this.env = env;
} }
/** Override the default nanosecond timer with the specified {@code timer}. /**
* Override the default nanosecond timer with the specified {@code timer}.
* *
* @param timer the timer to override with * @param timer the timer to override with
*/ */
@ -270,7 +269,7 @@ public class IdExecutionInstrument extends TruffleInstrument {
private final Consumer<ExpressionCall> functionCallCallback; private final Consumer<ExpressionCall> functionCallCallback;
private final Consumer<ExpressionValue> onComputedCallback; private final Consumer<ExpressionValue> onComputedCallback;
private final Consumer<ExpressionValue> onCachedCallback; private final Consumer<ExpressionValue> onCachedCallback;
private final Consumer<Throwable> onExceptionalCallback; private final Consumer<Exception> onExceptionalCallback;
private final RuntimeCache cache; private final RuntimeCache cache;
private final MethodCallsCache callsCache; private final MethodCallsCache callsCache;
private final UUID nextExecutionItem; private final UUID nextExecutionItem;
@ -299,7 +298,7 @@ public class IdExecutionInstrument extends TruffleInstrument {
Consumer<ExpressionCall> functionCallCallback, Consumer<ExpressionCall> functionCallCallback,
Consumer<ExpressionValue> onComputedCallback, Consumer<ExpressionValue> onComputedCallback,
Consumer<ExpressionValue> onCachedCallback, Consumer<ExpressionValue> onCachedCallback,
Consumer<Throwable> onExceptionalCallback, Consumer<Exception> onExceptionalCallback,
Timer timer) { Timer timer) {
this.entryCallTarget = entryCallTarget; this.entryCallTarget = entryCallTarget;
this.cache = cache; this.cache = cache;
@ -322,15 +321,7 @@ public class IdExecutionInstrument extends TruffleInstrument {
if (!isTopFrame(entryCallTarget)) { if (!isTopFrame(entryCallTarget)) {
return; return;
} }
UUID nodeId = getNodeId(context.getInstrumentedNode());
Node node = context.getInstrumentedNode();
UUID nodeId = null;
if (node instanceof ExpressionNode) {
nodeId = ((ExpressionNode) node).getId();
} else if (node instanceof FunctionCallInstrumentationNode) {
nodeId = ((FunctionCallInstrumentationNode) node).getId();
}
// Add a flag to say it was cached. // Add a flag to say it was cached.
// An array of `ProfilingInfo` in the value update. // An array of `ProfilingInfo` in the value update.
@ -396,6 +387,9 @@ public class IdExecutionInstrument extends TruffleInstrument {
onComputedCallback.accept( onComputedCallback.accept(
new ExpressionValue( new ExpressionValue(
nodeId, result, resultType, cachedType, call, cachedCall, profilingInfo, false)); nodeId, result, resultType, cachedType, call, cachedCall, profilingInfo, false));
if (result instanceof PanicSentinel) {
throw context.createUnwind(result);
}
} }
} }
@ -414,6 +408,11 @@ public class IdExecutionInstrument extends TruffleInstrument {
} catch (InteropException e) { } catch (InteropException e) {
onExceptionalCallback.accept(e); onExceptionalCallback.accept(e);
} }
} else if (exception instanceof PanicException) {
PanicException panicException = (PanicException) exception;
onReturnValue(context, frame, new PanicSentinel(panicException, context.getInstrumentedNode()));
} else if (exception instanceof PanicSentinel) {
onReturnValue(context, frame, exception);
} }
} }
@ -446,6 +445,16 @@ public class IdExecutionInstrument extends TruffleInstrument {
}); });
return result == null; return result == null;
} }
private UUID getNodeId(Node node) {
if (node instanceof ExpressionNode) {
return ((ExpressionNode) node).getId();
}
if (node instanceof FunctionCallInstrumentationNode) {
return ((FunctionCallInstrumentationNode) node).getId();
}
return null;
}
} }
/** /**
@ -473,7 +482,7 @@ public class IdExecutionInstrument extends TruffleInstrument {
Consumer<IdExecutionInstrument.ExpressionCall> functionCallCallback, Consumer<IdExecutionInstrument.ExpressionCall> functionCallCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback,
Consumer<Throwable> onExceptionalCallback) { Consumer<Exception> onExceptionalCallback) {
SourceSectionFilter filter = SourceSectionFilter filter =
SourceSectionFilter.newBuilder() SourceSectionFilter.newBuilder()
.tagIs(StandardTags.ExpressionTag.class, StandardTags.CallTag.class) .tagIs(StandardTags.ExpressionTag.class, StandardTags.CallTag.class)

View File

@ -12,6 +12,7 @@ public class Constants {
public static final String INTEGER = "Builtins.Main.Integer"; public static final String INTEGER = "Builtins.Main.Integer";
public static final String DECIMAL = "Builtins.Main.Decimal"; public static final String DECIMAL = "Builtins.Main.Decimal";
public static final String NOTHING = "Builtins.Main.Nothing"; public static final String NOTHING = "Builtins.Main.Nothing";
public static final String PANIC = "Builtins.Main.Panic";
public static final String REF = "Builtins.Main.Ref"; public static final String REF = "Builtins.Main.Ref";
public static final String TEXT = "Builtins.Main.Text"; public static final String TEXT = "Builtins.Main.Text";
public static final String THUNK = "Builtins.Main.Thunk"; public static final String THUNK = "Builtins.Main.Thunk";

View File

@ -122,6 +122,8 @@ public class Types {
return Constants.ARRAY; return Constants.ARRAY;
} else if (TypesGen.isRef(value)) { } else if (TypesGen.isRef(value)) {
return Constants.REF; return Constants.REF;
} else if (TypesGen.isPanicSentinel(value)) {
return Constants.PANIC;
} else { } else {
return null; return null;
} }

View File

@ -102,7 +102,7 @@ public class ExecutionService {
Consumer<IdExecutionInstrument.ExpressionCall> funCallCallback, Consumer<IdExecutionInstrument.ExpressionCall> funCallCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback,
Consumer<Throwable> onExceptionalCallback) Consumer<Exception> onExceptionalCallback)
throws ArityException, SourceNotFoundException, UnsupportedMessageException, throws ArityException, SourceNotFoundException, UnsupportedMessageException,
UnsupportedTypeException { UnsupportedTypeException {
SourceSection src = call.getFunction().getSourceSection(); SourceSection src = call.getFunction().getSourceSection();
@ -153,7 +153,7 @@ public class ExecutionService {
Consumer<IdExecutionInstrument.ExpressionCall> funCallCallback, Consumer<IdExecutionInstrument.ExpressionCall> funCallCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onComputedCallback,
Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback, Consumer<IdExecutionInstrument.ExpressionValue> onCachedCallback,
Consumer<Throwable> onExceptionalCallback) Consumer<Exception> onExceptionalCallback)
throws ArityException, ConstructorNotFoundException, MethodNotFoundException, throws ArityException, ConstructorNotFoundException, MethodNotFoundException,
ModuleNotFoundException, UnsupportedMessageException, UnsupportedTypeException { ModuleNotFoundException, UnsupportedMessageException, UnsupportedTypeException {
Module module = Module module =

View File

@ -1,82 +1,56 @@
package org.enso.interpreter.instrument.execution package org.enso.interpreter.instrument.execution
import com.oracle.truffle.api.TruffleException import java.io.File
import com.oracle.truffle.api.source.SourceSection
import org.enso.compiler.pass.analyse.DataflowAnalysis import com.oracle.truffle.api.{
import org.enso.polyglot.LanguageInfo TruffleException,
TruffleStackTrace,
TruffleStackTraceElement
}
import org.enso.polyglot.runtime.Runtime.Api import org.enso.polyglot.runtime.Runtime.Api
import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters.RichOptional
/** Methods for handling exceptions in the interpreter. */ /** Methods for handling exceptions in the interpreter. */
object ErrorResolver { object ErrorResolver {
/** Create expression updates about the failed expression and expressions /** Create a stack trace of a guest language from a java exception.
* that were not executed (poisoned) due to the failed expression.
* *
* @param error the runtime exception * @param throwable the exception
* @param ctx the runtime context * @param ctx the runtime context
* @return the list of updates about the expressions not executed * @return a runtime API representation of a stack trace
*/ */
def createUpdates(error: Throwable)(implicit def getStackTrace(
ctx: RuntimeContext throwable: Throwable
): Set[Api.ExpressionUpdate] = { )(implicit ctx: RuntimeContext): Vector[Api.StackTraceElement] =
getErrorSource(error) match { TruffleStackTrace
case Some(section) => .getStackTrace(throwable)
val moduleName = section.getSource.getName .asScala
val moduleOpt = ctx.executionService.getContext.findModule(moduleName) .map(toStackElement)
moduleOpt .toVector
.map { module =>
val meta = module.getIr.unsafeGetMetadata(
DataflowAnalysis,
"Empty dataflow analysis metadata during program execution."
)
LocationResolver
.getExpressionId(section, module)
.map { expressionId =>
val poisoned: Set[Api.ExpressionUpdate] = meta
.getExternal(toDataflowDependencyType(expressionId))
.getOrElse(Set())
.map { id =>
Api.ExpressionUpdate(
id,
None,
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
Api.ExpressionUpdate.Payload.Poisoned(Seq())
)
}
val failed =
Api.ExpressionUpdate(
expressionId.externalId,
None,
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
Api.ExpressionUpdate.Payload
.RuntimeError(error.getMessage, Seq())
)
poisoned + failed
}
.getOrElse(Set())
}
.orElse(Set())
case None =>
Set()
}
}
/** Get the source location of the runtime exception. /** Convert from the truffle stack element to the runtime API representation.
* *
* @param error the runtime exception * @param element the trufle stack trace element
* @return the error location in the source file * @param ctx the runtime context
* @return the runtime API representation of the stack trace element
*/ */
private def getErrorSource(error: Throwable): Option[SourceSection] = private def toStackElement(
error match { element: TruffleStackTraceElement
case ex: TruffleException )(implicit ctx: RuntimeContext): Api.StackTraceElement = {
if getLanguage(ex).forall(_ == LanguageInfo.ID) => val node = element.getLocation
Option(ex.getSourceLocation) node.getEncapsulatingSourceSection match {
case _ => case null =>
None Api.StackTraceElement(node.getRootNode.getName, None, None, None)
case section =>
Api.StackTraceElement(
element.getTarget.getRootNode.getName,
findFileByModuleName(section.getSource.getName),
Some(LocationResolver.sectionToRange(section)),
LocationResolver.getExpressionId(section).map(_.externalId)
)
}
} }
/** Get the language produced the runtime exception. /** Get the language produced the runtime exception.
@ -84,19 +58,24 @@ object ErrorResolver {
* @param ex the runtime exception * @param ex the runtime exception
* @return the language of the source file produced the runtime exception * @return the language of the source file produced the runtime exception
*/ */
private def getLanguage(ex: TruffleException): Option[String] = def getLanguage(ex: TruffleException): Option[String] =
for { for {
location <- Option(ex.getSourceLocation) location <- Option(ex.getSourceLocation)
source <- Option(location.getSource) source <- Option(location.getSource)
} yield source.getLanguage } yield source.getLanguage
/** Convert this expression id to the dataflow dependency type. */ /** Find source file path by the module name.
private def toDataflowDependencyType( *
id: LocationResolver.ExpressionId * @param module the module name
): DataflowAnalysis.DependencyInfo.Type.Static = * @param ctx the runtime context
DataflowAnalysis.DependencyInfo.Type * @return the source file path
.Static(id.internalId, Some(id.externalId)) */
private def findFileByModuleName(
module: String
)(implicit ctx: RuntimeContext): Option[File] =
for {
module <- ctx.executionService.getContext.findModule(module).toScala
path <- Option(module.getPath)
} yield new File(path)
val DependencyFailed: String =
"Dependency failed."
} }

View File

@ -1,11 +1,12 @@
package org.enso.interpreter.instrument.job package org.enso.interpreter.instrument.job
import java.io.File
import java.util.function.Consumer
import java.util.logging.Level
import java.util.{Objects, UUID}
import cats.implicits._ import cats.implicits._
import com.oracle.truffle.api.{ import com.oracle.truffle.api.TruffleException
TruffleException,
TruffleStackTrace,
TruffleStackTraceElement
}
import org.enso.interpreter.instrument.IdExecutionInstrument.{ import org.enso.interpreter.instrument.IdExecutionInstrument.{
ExpressionCall, ExpressionCall,
ExpressionValue ExpressionValue
@ -36,12 +37,8 @@ import org.enso.interpreter.service.error.{
import org.enso.polyglot.LanguageInfo import org.enso.polyglot.LanguageInfo
import org.enso.polyglot.runtime.Runtime.Api import org.enso.polyglot.runtime.Runtime.Api
import org.enso.polyglot.runtime.Runtime.Api.ContextId import org.enso.polyglot.runtime.Runtime.Api.ContextId
import org.enso.interpreter.runtime.error.PanicSentinel
import java.io.File
import java.util.function.Consumer
import java.util.logging.Level
import java.util.{Objects, UUID}
import scala.jdk.CollectionConverters._
import scala.jdk.OptionConverters._ import scala.jdk.OptionConverters._
/** Provides support for executing Enso code. Adds convenient methods to /** Provides support for executing Enso code. Adds convenient methods to
@ -65,7 +62,7 @@ trait ProgramExecutionSupport {
onCachedMethodCallCallback: Consumer[ExpressionValue], onCachedMethodCallCallback: Consumer[ExpressionValue],
onComputedCallback: Consumer[ExpressionValue], onComputedCallback: Consumer[ExpressionValue],
onCachedCallback: Consumer[ExpressionValue], onCachedCallback: Consumer[ExpressionValue],
onExceptionalCallback: Consumer[Throwable] onExceptionalCallback: Consumer[Exception]
)(implicit ctx: RuntimeContext): Unit = { )(implicit ctx: RuntimeContext): Unit = {
val methodCallsCache = new MethodCallsCache val methodCallsCache = new MethodCallsCache
var enterables = Map[UUID, FunctionCall]() var enterables = Map[UUID, FunctionCall]()
@ -196,7 +193,7 @@ trait ProgramExecutionSupport {
fireVisualisationUpdates(contextId, value) fireVisualisationUpdates(contextId, value)
} }
val onExceptionalCallback: Consumer[Throwable] = { value => val onExceptionalCallback: Consumer[Exception] = { value =>
logger.log(Level.FINEST, s"ON_ERROR $value") logger.log(Level.FINEST, s"ON_ERROR $value")
sendErrorUpdate(contextId, value) sendErrorUpdate(contextId, value)
} }
@ -219,7 +216,7 @@ trait ProgramExecutionSupport {
onExceptionalCallback onExceptionalCallback
) )
) )
.leftMap(onExecutionError(contextId, stackItem.item, _)) .leftMap(onExecutionError(stackItem.item, _))
} yield () } yield ()
logger.log(Level.FINEST, s"Execution finished: $executionResult") logger.log(Level.FINEST, s"Execution finished: $executionResult")
executionResult.fold(Some(_), _ => None) executionResult.fold(Some(_), _ => None)
@ -227,13 +224,11 @@ trait ProgramExecutionSupport {
/** Execution error handler. /** Execution error handler.
* *
* @param contextId an identifier of an execution context
* @param item the stack item being executed * @param item the stack item being executed
* @param error the execution error * @param error the execution error
* @return the error message * @return the error message
*/ */
private def onExecutionError( private def onExecutionError(
contextId: ContextId,
item: ExecutionItem, item: ExecutionItem,
error: Throwable error: Throwable
)(implicit ctx: RuntimeContext): Api.ExecutionResult = { )(implicit ctx: RuntimeContext): Api.ExecutionResult = {
@ -250,7 +245,6 @@ trait ProgramExecutionSupport {
ctx.executionService.getLogger ctx.executionService.getLogger
.log(Level.FINEST, s"Error executing a function $itemName.", error) .log(Level.FINEST, s"Error executing a function $itemName.", error)
} }
sendExpressionUpdates(contextId, ErrorResolver.createUpdates(error))
executionUpdate.getOrElse( executionUpdate.getOrElse(
Api.ExecutionResult Api.ExecutionResult
.Failure(s"Error in function $itemName.", None) .Failure(s"Error in function $itemName.", None)
@ -266,14 +260,9 @@ trait ProgramExecutionSupport {
private def getExecutionOutcome( private def getExecutionOutcome(
t: Throwable t: Throwable
)(implicit ctx: RuntimeContext): Option[Api.ExecutionResult] = { )(implicit ctx: RuntimeContext): Option[Api.ExecutionResult] = {
def getLanguage(ex: TruffleException): Option[String] =
for {
location <- Option(ex.getSourceLocation)
source <- Option(location.getSource)
} yield source.getLanguage
t match { t match {
case ex: TruffleException case ex: TruffleException
if getLanguage(ex).forall(_ == LanguageInfo.ID) => if ErrorResolver.getLanguage(ex).forall(_ == LanguageInfo.ID) =>
val section = Option(ex.getSourceLocation) val section = Option(ex.getSourceLocation)
Some( Some(
Api.ExecutionResult.Diagnostic.error( Api.ExecutionResult.Diagnostic.error(
@ -283,7 +272,7 @@ trait ProgramExecutionSupport {
section section
.flatMap(LocationResolver.getExpressionId(_)) .flatMap(LocationResolver.getExpressionId(_))
.map(_.externalId), .map(_.externalId),
getStackTrace(ex) ErrorResolver.getStackTrace(ex)
) )
) )
@ -311,45 +300,7 @@ trait ProgramExecutionSupport {
} }
} }
/** Create a stack trace of a guest language from a java exception. private def sendErrorUpdate(contextId: ContextId, error: Exception)(implicit
*
* @param throwable the exception
* @param ctx the runtime context
* @return a runtime API representation of a stack trace
*/
private def getStackTrace(
throwable: Throwable
)(implicit ctx: RuntimeContext): Vector[Api.StackTraceElement] =
TruffleStackTrace
.getStackTrace(throwable)
.asScala
.map(toStackElement)
.toVector
/** Convert from the truffle stack element to the runtime API representation.
*
* @param element the trufle stack trace element
* @param ctx the runtime context
* @return the runtime API representation of the stack trace element
*/
private def toStackElement(
element: TruffleStackTraceElement
)(implicit ctx: RuntimeContext): Api.StackTraceElement = {
val node = element.getLocation
node.getEncapsulatingSourceSection match {
case null =>
Api.StackTraceElement(node.getRootNode.getName, None, None, None)
case section =>
Api.StackTraceElement(
element.getTarget.getRootNode.getName,
findFileByModuleName(section.getSource.getName),
Some(LocationResolver.sectionToRange(section)),
LocationResolver.getExpressionId(section).map(_.externalId)
)
}
}
private def sendErrorUpdate(contextId: ContextId, error: Throwable)(implicit
ctx: RuntimeContext ctx: RuntimeContext
): Unit = { ): Unit = {
ctx.endpoint.sendToClient( ctx.endpoint.sendToClient(
@ -362,19 +313,6 @@ trait ProgramExecutionSupport {
) )
} }
private def sendExpressionUpdates(
contextId: ContextId,
updates: Set[Api.ExpressionUpdate]
)(implicit ctx: RuntimeContext): Unit = {
if (updates.nonEmpty) {
ctx.endpoint.sendToClient(
Api.Response(
Api.ExpressionUpdates(contextId, updates)
)
)
}
}
private def sendValueUpdate( private def sendValueUpdate(
contextId: ContextId, contextId: ContextId,
value: ExpressionValue, value: ExpressionValue,
@ -386,25 +324,16 @@ trait ProgramExecutionSupport {
!Objects.equals(value.getCallInfo, value.getCachedCallInfo) || !Objects.equals(value.getCallInfo, value.getCachedCallInfo) ||
!Objects.equals(value.getType, value.getCachedType) !Objects.equals(value.getType, value.getCachedType)
) { ) {
// TODO: [DB] Remove when IDE implements new updates API val payload = value.getValue match {
ctx.endpoint.sendToClient( case sentinel: PanicSentinel =>
Api.Response( Api.ExpressionUpdate.Payload
Api.ExpressionValuesComputed( .Panic(
contextId, sentinel.getMessage,
Vector( ErrorResolver.getStackTrace(sentinel).flatMap(_.expressionId)
Api.ExpressionValueUpdate(
value.getExpressionId,
Option(value.getType),
methodPointer,
value.getProfilingInfo.map { case e: ExecutionTime =>
Api.ProfilingInfo.ExecutionTime(e.getNanoTimeElapsed)
}.toVector,
value.wasCached()
)
)
)
)
) )
case _ =>
Api.ExpressionUpdate.Payload.Value()
}
ctx.endpoint.sendToClient( ctx.endpoint.sendToClient(
Api.Response( Api.Response(
Api.ExpressionUpdates( Api.ExpressionUpdates(
@ -418,7 +347,7 @@ trait ProgramExecutionSupport {
Api.ProfilingInfo.ExecutionTime(e.getNanoTimeElapsed) Api.ProfilingInfo.ExecutionTime(e.getNanoTimeElapsed)
}.toVector, }.toVector,
value.wasCached(), value.wasCached(),
Api.ExpressionUpdate.Payload.Value() payload
) )
) )
) )

View File

@ -1,5 +1,13 @@
package org.enso.interpreter.test.instrument package org.enso.interpreter.test.instrument
import java.io.{ByteArrayOutputStream, File}
import java.nio.ByteBuffer
import java.nio.file.Files
import java.util.UUID
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
import org.enso.interpreter.instrument.execution.Timer
import org.enso.interpreter.runtime.`type`.Constants
import org.enso.interpreter.test.Metadata import org.enso.interpreter.test.Metadata
import org.enso.pkg.{Package, PackageManager} import org.enso.pkg.{Package, PackageManager}
import org.enso.polyglot._ import org.enso.polyglot._
@ -11,11 +19,6 @@ import org.graalvm.polyglot.io.MessageEndpoint
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import java.io.{ByteArrayOutputStream, File}
import java.nio.ByteBuffer
import java.nio.file.Files
import java.util.UUID
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
@scala.annotation.nowarn("msg=multiarg infix syntax") @scala.annotation.nowarn("msg=multiarg infix syntax")
class ExpressionErrorsTest class ExpressionErrorsTest
@ -23,6 +26,14 @@ class ExpressionErrorsTest
with Matchers with Matchers
with BeforeAndAfterEach { with BeforeAndAfterEach {
// === Test Timer ===========================================================
class TestTimer extends Timer {
override def getTime(): Long = 0
}
// === Test Utilities =======================================================
var context: TestContext = _ var context: TestContext = _
class TestContext(packageName: String) { class TestContext(packageName: String) {
@ -66,6 +77,13 @@ class ExpressionErrorsTest
) )
executionContext.context.initialize(LanguageInfo.ID) executionContext.context.initialize(LanguageInfo.ID)
val languageContext = executionContext.context
.getBindings(LanguageInfo.ID)
.invokeMember(MethodNames.TopScope.LEAK_CONTEXT)
.asHostObject[org.enso.interpreter.runtime.Context]
languageContext.getLanguage.getIdExecutionInstrument
.overrideTimer(new TestTimer)
def writeMain(contents: String): File = def writeMain(contents: String): File =
Files.write(pkg.mainFile.toPath, contents.getBytes).toFile Files.write(pkg.mainFile.toPath, contents.getBytes).toFile
@ -103,34 +121,49 @@ class ExpressionErrorsTest
object Update { object Update {
def error( def panic(
contextId: UUID,
expressionId: UUID, expressionId: UUID,
payload: Api.ExpressionUpdate.Payload payload: Api.ExpressionUpdate.Payload
): Api.ExpressionUpdate = ): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate( Api.ExpressionUpdate(
expressionId, expressionId,
None, Some(Constants.PANIC),
None, None,
Vector(Api.ProfilingInfo.ExecutionTime(0)), Vector(Api.ProfilingInfo.ExecutionTime(0)),
false, false,
payload payload
) )
)
def runtimeError( )
expressionId: UUID, )
message: String
): Api.ExpressionUpdate = def panic(
Api.ExpressionUpdate( contextId: UUID,
expressionId, expressionId: UUID,
None, methodPointer: Api.MethodPointer,
None, payload: Api.ExpressionUpdate.Payload
Vector(Api.ProfilingInfo.ExecutionTime(0)), ): Api.Response =
false, Api.Response(
Api.ExpressionUpdate.Payload.RuntimeError(message, Seq()) Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
expressionId,
Some(Constants.PANIC),
Some(methodPointer),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
payload
)
)
)
) )
def poisonedError(expressionId: UUID): Api.ExpressionUpdate =
error(expressionId, Api.ExpressionUpdate.Payload.Poisoned(Seq()))
} }
def contentsVersion(content: String): ContentVersion = def contentsVersion(content: String): ContentVersion =
@ -141,12 +174,11 @@ class ExpressionErrorsTest
val Some(Api.Response(_, Api.InitializedNotification())) = context.receive val Some(Api.Response(_, Api.InitializedNotification())) = context.receive
} }
it should "return dataflow errors in method body" in { it should "return panic sentinels in method body" in {
val contextId = UUID.randomUUID() val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID() val requestId = UUID.randomUUID()
val moduleName = "Test.Main" val moduleName = "Test.Main"
val metadata = new Metadata val metadata = new Metadata
@scala.annotation.unused
val fooBodyId = metadata.addItem(21, 5) val fooBodyId = metadata.addItem(21, 5)
val xId = metadata.addItem(35, 9) val xId = metadata.addItem(35, 9)
val yId = metadata.addItem(53, 8) val yId = metadata.addItem(53, 8)
@ -188,7 +220,7 @@ class ExpressionErrorsTest
) )
) )
) )
context.receive(5) should contain theSameElementsAs Seq( context.receive(7) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)), Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response( Api.Response(
Api.ExecutionUpdate( Api.ExecutionUpdate(
@ -198,50 +230,325 @@ class ExpressionErrorsTest
"Variable `undefined` is not defined.", "Variable `undefined` is not defined.",
Some(mainFile), Some(mainFile),
Some(model.Range(model.Position(2, 8), model.Position(2, 17))), Some(model.Range(model.Position(2, 8), model.Position(2, 17))),
Some(xId),
Vector()
)
)
)
),
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Update.runtimeError(
xId,
"Compile_Error Variable `undefined` is not defined."
),
Update.poisonedError(yId),
Update.poisonedError(mainResId)
)
)
),
Api.Response(
Api.ExecutionUpdate(
contextId,
Seq(
Api.ExecutionResult.Diagnostic.error(
"Compile_Error Variable `undefined` is not defined.",
Some(mainFile),
Some(model.Range(model.Position(2, 8), model.Position(2, 17))),
Some(xId),
Vector(
Api.StackTraceElement(
"Main.main",
Some(mainFile),
Some(
model.Range(model.Position(2, 8), model.Position(2, 17))
),
Some(xId) Some(xId)
) )
) )
) )
),
Update.panic(
contextId,
xId,
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `undefined` is not defined.",
Seq(xId)
) )
),
Update.panic(
contextId,
fooBodyId,
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `undefined` is not defined.",
Seq(xId)
)
),
Update.panic(
contextId,
yId,
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `undefined` is not defined.",
Seq(xId)
)
),
Update.panic(
contextId,
mainResId,
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `undefined` is not defined.",
Seq(xId)
) )
), ),
context.executionComplete(contextId) context.executionComplete(contextId)
) )
} }
it should "return panic sentinels in method calls" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Test.Main"
val metadata = new Metadata
val mainBodyId = metadata.addItem(28, 12)
val code =
"""foo a b = a + b + x
|
|main = here.foo 1 2
|""".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(Api.OpenFileNotification(mainFile, contents, true))
)
context.receiveNone shouldEqual None
// push main
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, "Test.Main", "main"),
None,
Vector()
)
)
)
)
context.receive(4) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
contextId,
Seq(
Api.ExecutionResult.Diagnostic.error(
"Variable `x` is not defined.",
Some(mainFile),
Some(model.Range(model.Position(0, 18), model.Position(0, 19))),
None
)
)
)
),
Update.panic(
contextId,
mainBodyId,
Api.MethodPointer("Test.Main", "Test.Main", "foo"),
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `x` is not defined.",
Seq(mainBodyId)
)
),
context.executionComplete(contextId)
)
}
it should "return dataflow errors in method body" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Test.Main"
val metadata = new Metadata
val fooBodyId = metadata.addItem(61, 5)
val xId = metadata.addItem(75, 19)
val yId = metadata.addItem(103, 8)
val mainResId = metadata.addItem(116, 7)
val code =
"""from Builtins import all
|
|type MyError
|
|main =
| foo a b = a + b
| x = Error.throw MyError
| y = foo x 42
| foo y 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(Api.OpenFileNotification(mainFile, contents, true))
)
context.receiveNone shouldEqual None
// push main
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, "Test.Main", "main"),
None,
Vector()
)
)
)
)
context.receive(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
TestMessages.update(contextId, xId, Constants.ERROR),
TestMessages.update(contextId, fooBodyId, Constants.ERROR),
TestMessages.update(contextId, yId, Constants.ERROR),
TestMessages.update(contextId, mainResId, Constants.ERROR),
context.executionComplete(contextId)
)
}
it should "return panic sentinels continuing execution" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Test.Main"
val metadata = new Metadata
val xId = metadata.addItem(41, 9)
val yId = metadata.addItem(59, 2)
val mainResId = metadata.addItem(66, 12)
val code =
"""from Builtins import all
|
|main =
| x = undefined
| y = 42
| IO.println y
|""".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(Api.OpenFileNotification(mainFile, contents, true))
)
context.receiveNone shouldEqual None
// push main
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, "Test.Main", "main"),
None,
Vector()
)
)
)
)
context.receive(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
contextId,
Seq(
Api.ExecutionResult.Diagnostic.warning(
"Unused variable x.",
Some(mainFile),
Some(model.Range(model.Position(3, 4), model.Position(3, 5)))
),
Api.ExecutionResult.Diagnostic.error(
"Variable `undefined` is not defined.",
Some(mainFile),
Some(model.Range(model.Position(3, 8), model.Position(3, 17))),
Some(xId)
)
)
)
),
Update.panic(
contextId,
xId,
Api.ExpressionUpdate.Payload.Panic(
"Compile_Error Variable `undefined` is not defined.",
Seq(xId)
)
),
TestMessages.update(contextId, yId, Constants.INTEGER),
TestMessages.update(contextId, mainResId, Constants.NOTHING),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual Seq("42")
}
it should "return dataflow errors continuing execution" in {
val contextId = UUID.randomUUID()
val requestId = UUID.randomUUID()
val moduleName = "Test.Main"
val metadata = new Metadata
val xId = metadata.addItem(55, 19)
val yId = metadata.addItem(83, 2)
val mainResId = metadata.addItem(90, 12)
val code =
"""from Builtins import all
|
|type MyError
|
|main =
| x = Error.throw MyError
| y = 42
| IO.println y
|""".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(Api.OpenFileNotification(mainFile, contents, true))
)
context.receiveNone shouldEqual None
// push main
context.send(
Api.Request(
requestId,
Api.PushContextRequest(
contextId,
Api.StackItem.ExplicitCall(
Api.MethodPointer(moduleName, "Test.Main", "main"),
None,
Vector()
)
)
)
)
context.receive(6) should contain theSameElementsAs Seq(
Api.Response(requestId, Api.PushContextResponse(contextId)),
Api.Response(
Api.ExecutionUpdate(
contextId,
Seq(
Api.ExecutionResult.Diagnostic.warning(
"Unused variable x.",
Some(mainFile),
Some(model.Range(model.Position(5, 4), model.Position(5, 5)))
)
)
)
),
TestMessages.update(contextId, xId, Constants.ERROR),
TestMessages.update(contextId, yId, Constants.INTEGER),
TestMessages.update(contextId, mainResId, Constants.NOTHING),
context.executionComplete(contextId)
)
context.consumeOut shouldEqual Seq("42")
}
} }

View File

@ -0,0 +1,112 @@
package org.enso.interpreter.test.instrument
import java.util.UUID
import org.enso.polyglot.runtime.Runtime.Api
/** Helper methods for creating test messages. */
object TestMessages {
/** Create an update response.
*
* @param contextId an identifier of the context
* @param expressionId an identifier of the expression
* @return the expression update response
*/
def update(
contextId: UUID,
expressionId: UUID
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
expressionId,
None,
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
/** Create an update response.
*
* @param contextId an identifier of the context
* @param expressionId an identifier of the expression
* @param expressionType a type of the expression
* @return the expression update response
*/
def update(
contextId: UUID,
expressionId: UUID,
expressionType: String
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
expressionId,
Some(expressionType),
None,
Vector(Api.ProfilingInfo.ExecutionTime(0)),
false,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
/** Create an update response.
*
* @param contextId an identifier of the context
* @param expressionId an identifier of the expression
* @param expressionType a type of the expression
* @param methodPointer a pointer to the method definition
* @return the expression update response
*/
def update(
contextId: UUID,
expressionId: UUID,
expressionType: String,
methodPointer: Api.MethodPointer
): Api.Response =
update(contextId, expressionId, expressionType, methodPointer, false)
/** Create an update response.
*
* @param contextId an identifier of the context
* @param expressionId an identifier of the expression
* @param expressionType a type of the expression
* @param methodPointer a pointer to the method definition
* @param fromCache whether or not the value for this expression came
* from the cache
* @return the expression update response
*/
def update(
contextId: UUID,
expressionId: UUID,
expressionType: String,
methodPointer: Api.MethodPointer,
fromCache: Boolean
): Api.Response =
Api.Response(
Api.ExpressionUpdates(
contextId,
Set(
Api.ExpressionUpdate(
expressionId,
Some(expressionType),
Some(methodPointer),
Vector(Api.ProfilingInfo.ExecutionTime(0)),
fromCache,
Api.ExpressionUpdate.Payload.Value()
)
)
)
)
}