[Speedy] create Scenario Machine (#15588)

This commit is contained in:
Remy 2022-12-16 09:20:42 +01:00 committed by GitHub
parent 6c25c7c8be
commit f1aca5e11c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
24 changed files with 311 additions and 286 deletions

View File

@ -140,7 +140,7 @@ class Context(val contextId: Context.ContextId, languageVersion: LanguageVersion
private val txSeeding =
crypto.Hash.hashPrivateKey(s"scenario-service")
private[this] def buildMachine(defn: SDefinition): Speedy.OffLedgerMachine =
private[this] def buildMachine(defn: SDefinition): Speedy.ScenarioMachine =
Speedy.Machine.fromScenarioSExpr(
PureCompiledPackages(allSignatures, defns, compilerConfig),
defn.body,

View File

@ -10,7 +10,7 @@ import com.daml.lf.data.Ref.{Identifier, PackageId, ParticipantId, Party}
import com.daml.lf.language.Ast._
import com.daml.lf.speedy.{InitialSeeding, Pretty, SError, SValue}
import com.daml.lf.speedy.SExpr.{SEApp, SExpr}
import com.daml.lf.speedy.Speedy.{Machine, OffLedgerMachine, OnLedgerMachine}
import com.daml.lf.speedy.Speedy.{Machine, PureMachine, UpdateMachine}
import com.daml.lf.speedy.SResult._
import com.daml.lf.transaction.{
Node,
@ -318,7 +318,7 @@ class Engine(val config: EngineConfig = Engine.StableConfig) {
submissionTime: Time.Timestamp,
seeding: speedy.InitialSeeding,
)(implicit loggingContext: LoggingContext): Result[(SubmittedTransaction, Tx.Metadata)] = {
val machine = OnLedgerMachine(
val machine = UpdateMachine(
compiledPackages = compiledPackages,
submissionTime = submissionTime,
initialSeeding = seeding,
@ -366,7 +366,7 @@ class Engine(val config: EngineConfig = Engine.StableConfig) {
// TODO SC remove 'return', notwithstanding a love of unhandled exceptions
@SuppressWarnings(Array("org.wartremover.warts.Return"))
private[engine] def interpretLoop(
machine: OnLedgerMachine,
machine: UpdateMachine,
time: Time.Timestamp,
): Result[(SubmittedTransaction, Tx.Metadata)] = {
def detailMsg = Some(
@ -418,7 +418,7 @@ class Engine(val config: EngineConfig = Engine.StableConfig) {
)
case err @ (_: SResultScenarioSubmit | _: SResultScenarioPassTime |
_: SResultScenarioGetParty) =>
_: SResultScenarioGetParty | _: SResultScenarioGetTime) =>
return ResultError(
Error.Interpretation.Internal(
NameOf.qualifiedNameOfCurrentFunc,
@ -430,7 +430,7 @@ class Engine(val config: EngineConfig = Engine.StableConfig) {
}
machine.finish match {
case Right(OnLedgerMachine.Result(tx, _, nodeSeeds, globalKeyMapping, disclosedContracts)) =>
case Right(UpdateMachine.Result(tx, _, nodeSeeds, globalKeyMapping, disclosedContracts)) =>
deps(tx).flatMap { deps =>
val meta = Tx.Metadata(
submissionSeed = None,
@ -522,13 +522,13 @@ class Engine(val config: EngineConfig = Engine.StableConfig) {
argument: Value,
interfaceId: Identifier,
)(implicit loggingContext: LoggingContext): Result[Versioned[Value]] = {
def interpret(machine: OffLedgerMachine): Result[SValue] = {
def interpret(machine: PureMachine): Result[SValue] = {
machine.run() match {
case SResultFinal(v) => ResultDone(v)
case SResultError(err) => handleError(err, None)
case err @ (_: SResultNeedPackage | _: SResultNeedContract | _: SResultNeedKey |
_: SResultNeedTime | _: SResultScenarioGetParty | _: SResultScenarioPassTime |
_: SResultScenarioSubmit) =>
_: SResultScenarioSubmit | _: SResultScenarioGetTime) =>
ResultError(
Error.Interpretation.Internal(
NameOf.qualifiedNameOfCurrentFunc,

View File

@ -59,7 +59,6 @@ da_scala_library(
"//daml-lf/transaction",
"//daml-lf/validation",
"//libs-scala/contextualized-logging",
"//libs-scala/nameof",
"//libs-scala/scala-utils",
],
)

View File

@ -8,7 +8,6 @@ package explore
import com.daml.lf.language.PackageInterface
import com.daml.lf.speedy.SExpr0._
import com.daml.lf.speedy.SValue._
import com.daml.lf.speedy.SResult._
import com.daml.lf.speedy.SBuiltin._
import com.daml.lf.speedy.Speedy._
import com.daml.logging.LoggingContext
@ -68,23 +67,17 @@ object PlaySpeedy {
Config(names)
}
def runMachine(name: String, machine: Machine, expected: Int): Unit = {
def runMachine(name: String, machine: PureMachine, expected: Int): Unit = {
println(s"example name: $name")
machine.run() match {
case SResultFinal(value) =>
println(s"final-value: $value")
value match {
case SInt64(got) =>
if (got != expected) {
throw new MachineProblem(s"Expected final integer to be $expected, but got $got")
}
case _ =>
throw new MachineProblem(s"Expected final-value to be an integer")
machine.runPure().toTry.get match {
case SInt64(got) =>
if (got != expected) {
throw MachineProblem(s"Expected final integer to be $expected, but got $got")
}
case res =>
throw new MachineProblem(s"Unexpected result from machine $res")
case _ =>
throw MachineProblem(s"Expected final-value to be an integer")
}
}

View File

@ -9,7 +9,6 @@ import com.daml.bazeltools.BazelRunfiles.{rlocation}
import com.daml.lf.archive.UniversalArchiveDecoder
import com.daml.lf.data.Ref.{DefinitionRef, Identifier, QualifiedName}
import com.daml.lf.speedy.SExpr._
import com.daml.lf.speedy.SResult._
import com.daml.lf.speedy.SValue._
import com.daml.lf.speedy.Speedy._
import com.daml.logging.LoggingContext
@ -85,7 +84,7 @@ object PlaySpeedy {
throw new MachineProblem(s"Unexpecteded result when compiling $x")
}
val machine: Machine = {
val machine: PureMachine = {
println(s"Setup machine for: ${config.funcName}(${config.argValue})")
val expr = {
val ref: DefinitionRef =
@ -100,14 +99,8 @@ object PlaySpeedy {
Machine.fromPureSExpr(compiledPackages, expr)
}
val result: SValue = {
println("Run...")
machine.run() match {
case SResultFinal(value) => value
case res => throw new MachineProblem(s"Unexpected result from machine $res")
}
}
println("Run...")
val result = machine.runPure().toTry.get
println(s"Final-value: $result")
}

View File

@ -24,7 +24,9 @@ private[speedy] object PhaseOne {
stacktracing: StackTraceMode,
)
private val SEGetTime = SEBuiltin(SBGetTime)
private val SUGetTime = SEBuiltin(SBUGetTime)
private val SSGetTime = SEBuiltin(SBSGetTime)
private val SBEToTextNumeric = SEAbs(1, SEBuiltin(SBToText))
@ -757,7 +759,7 @@ private[lf] final class PhaseOne(
}
}
case UpdateGetTime =>
Return(SEGetTime)
Return(SUGetTime)
case UpdateLookupByKey(RetrieveByKey(templateId, key)) =>
compileExp(env, key) { key =>
Return(t.LookupByKeyDefRef(templateId)(key))
@ -845,7 +847,7 @@ private[lf] final class PhaseOne(
case ScenarioMustFailAt(partyE, updateE, _retType @ _) =>
compileCommit(env, partyE, updateE, optLoc, mustFail = true)
case ScenarioGetTime =>
Return(SEGetTime)
Return(SSGetTime)
case ScenarioGetParty(e) =>
compileGetParty(env, e)
case ScenarioPass(relTime) =>
@ -911,7 +913,7 @@ private[lf] final class PhaseOne(
compileExp(env, body) { body =>
let(env, body) { (bodyPos, env) =>
unaryFunction(env) { (tokenPos, env) =>
Return(SBSPure(env.toSEVar(bodyPos), env.toSEVar(tokenPos)))
Return(SBPure(env.toSEVar(bodyPos), env.toSEVar(tokenPos)))
}
}
}

View File

@ -528,7 +528,7 @@ private[lf] object Pretty {
)
case SBUCreate => text("$create")
case SBFetchAny => text("$fetchAny")
case SBGetTime => text("$getTime")
case SBUGetTime | SBSGetTime => text("$getTime")
case _ => str(x)
}
case SEAppOnlyFunIsAtomic(fun, args) =>

View File

@ -226,7 +226,7 @@ private[speedy] sealed abstract class SBuiltinPure(arity: Int) extends SBuiltin(
}
}
private[speedy] sealed abstract class OnLedgerBuiltin(arity: Int)
private[speedy] sealed abstract class UpdateBuiltin(arity: Int)
extends SBuiltin(arity)
with Product {
@ -236,17 +236,38 @@ private[speedy] sealed abstract class OnLedgerBuiltin(arity: Int)
* @param machine the Speedy machine (machine state may be modified by the builtin)
* @return the builtin execution's resulting control value
*/
protected def executeWithLedger(
protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control
override private[speedy] final def execute(
args: util.ArrayList[SValue],
machine: Machine,
): Control = {
machine.asOnLedger(productPrefix)(executeWithLedger(args, _))
}
): Control =
machine.asUpdateMachine(productPrefix)(executeUpdate(args, _))
}
private[speedy] sealed abstract class ScenarioBuiltin(arity: Int)
extends SBuiltin(arity)
with Product {
/** On ledger builtins may reference the Speedy machine's ledger state.
*
* @param args arguments for executing the builtin
* @param machine the Speedy machine (machine state may be modified by the builtin)
* @return the builtin execution's resulting control value
*/
protected def executeScenario(
args: util.ArrayList[SValue],
machine: ScenarioMachine,
): Control
override private[speedy] final def execute(
args: util.ArrayList[SValue],
machine: Machine,
): Control =
machine.asScenarioMachine(productPrefix)(executeScenario(args, _))
}
private[lf] object SBuiltin {
@ -470,9 +491,9 @@ private[lf] object SBuiltin {
): Control = {
val coid = getSContractId(args, 0).coid
machine match {
case _: OffLedgerMachine =>
case _: PureMachine | _: ScenarioMachine =>
Control.Value(SOptional(Some(SText(coid))))
case _: OnLedgerMachine =>
case _: UpdateMachine =>
Control.Value(SValue.SValue.None)
}
}
@ -958,10 +979,10 @@ private[lf] object SBuiltin {
* -> CachedContract
* -> ContractId arg
*/
final case object SBUCreate extends OnLedgerBuiltin(1) {
override protected def executeWithLedger(
final case object SBUCreate extends UpdateBuiltin(1) {
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val cached = extractCachedContract(args.get(0))
val version = machine.tmplId2TxVersion(cached.templateId)
@ -1017,11 +1038,11 @@ private[lf] object SBuiltin {
choiceId: ChoiceName,
consuming: Boolean,
byKey: Boolean,
) extends OnLedgerBuiltin(4) {
) extends UpdateBuiltin(4) {
override protected def executeWithLedger(
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val coid = getSContractId(args, 1)
val cached =
@ -1123,10 +1144,10 @@ private[lf] object SBuiltin {
* -> Optional {key: key, maintainers: List Party} (template key, if present)
* -> a
*/
final case object SBFetchAny extends OnLedgerBuiltin(2) {
override protected def executeWithLedger(
final case object SBFetchAny extends UpdateBuiltin(2) {
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val coid = getSContractId(args, 0)
machine.getCachedContract(coid) match {
@ -1442,10 +1463,10 @@ private[lf] object SBuiltin {
final case class SBUInsertFetchNode(
templateId: TypeConName,
byKey: Boolean,
) extends OnLedgerBuiltin(1) {
override protected def executeWithLedger(
) extends UpdateBuiltin(1) {
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val coid = getSContractId(args, 0)
val cached =
@ -1480,10 +1501,10 @@ private[lf] object SBuiltin {
* -> Maybe (ContractId T)
* -> ()
*/
final case class SBUInsertLookupNode(templateId: TypeConName) extends OnLedgerBuiltin(2) {
override protected def executeWithLedger(
final case class SBUInsertLookupNode(templateId: TypeConName) extends UpdateBuiltin(2) {
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val keyWithMaintainers =
extractKeyWithMaintainers(NameOf.qualifiedNameOfCurrentFunc, args.get(0))
@ -1556,11 +1577,11 @@ private[lf] object SBuiltin {
private[speedy] sealed abstract class SBUKeyBuiltin(
operation: KeyOperation
) extends OnLedgerBuiltin(1) {
) extends UpdateBuiltin(1) {
final override def executeWithLedger(
final override def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val skey = args.get(0)
val keyWithMaintainers = extractKeyWithMaintainers(NameOf.qualifiedNameOfCurrentFunc, skey)
@ -1648,22 +1669,30 @@ private[lf] object SBuiltin {
extends SBUKeyBuiltin(new KeyOperation.Lookup(templateId))
/** $getTime :: Token -> Timestamp */
final case object SBGetTime extends SBuiltin(1) {
override private[speedy] def execute(
final case object SBUGetTime extends UpdateBuiltin(1) {
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: Machine,
machine: UpdateMachine,
): Control = {
checkToken(args, 0)
// $ugettime :: Token -> Timestamp
machine match {
case machine: OnLedgerMachine =>
machine.setDependsOnTime()
case _: OffLedgerMachine =>
}
machine.setDependsOnTime()
Control.Question(
SResultNeedTime { timestamp =>
SResultNeedTime(timestamp => machine.setControl(Control.Value(STimestamp(timestamp))))
)
}
}
/** $getTime :: Token -> Timestamp */
final case object SBSGetTime extends ScenarioBuiltin(1) {
protected def executeScenario(
args: util.ArrayList[SValue],
machine: ScenarioMachine,
): Control = {
checkToken(args, 0)
Control.Question(
SResultScenarioGetTime(timestamp =>
machine.setControl(Control.Value(STimestamp(timestamp)))
}
)
)
}
}
@ -1687,10 +1716,11 @@ private[lf] object SBuiltin {
}
}
final case class SBSSubmit(optLocation: Option[Location], mustFail: Boolean) extends SBuiltin(3) {
override private[speedy] def execute(
final case class SBSSubmit(optLocation: Option[Location], mustFail: Boolean)
extends ScenarioBuiltin(3) {
override protected def executeScenario(
args: util.ArrayList[SValue],
machine: Machine,
machine: ScenarioMachine,
): Control = {
checkToken(args, 2)
Control.Question(
@ -1699,16 +1729,14 @@ private[lf] object SBuiltin {
commands = args.get(1),
location = optLocation,
mustFail = mustFail,
callback = { newValue =>
machine.setControl(Control.Value(newValue))
},
callback = newValue => machine.setControl(Control.Value(newValue)),
)
)
}
}
/** $pure :: a -> Token -> a */
final case object SBSPure extends SBuiltin(2) {
final case object SBPure extends SBuiltin(2) {
override private[speedy] def execute(
args: util.ArrayList[SValue],
machine: Machine,
@ -1719,10 +1747,10 @@ private[lf] object SBuiltin {
}
/** $pass :: Int64 -> Token -> Timestamp */
final case object SBSPass extends SBuiltin(2) {
override private[speedy] def execute(
final case object SBSPass extends ScenarioBuiltin(2) {
override protected def executeScenario(
args: util.ArrayList[SValue],
machine: Machine,
machine: ScenarioMachine,
): Control = {
checkToken(args, 1)
val relTime = getSInt64(args, 0)
@ -1738,19 +1766,17 @@ private[lf] object SBuiltin {
}
/** $getParty :: Text -> Token -> Party */
final case object SBSGetParty extends SBuiltin(2) {
override private[speedy] def execute(
final case object SBSGetParty extends ScenarioBuiltin(2) {
override protected def executeScenario(
args: util.ArrayList[SValue],
machine: Machine,
machine: ScenarioMachine,
): Control = {
checkToken(args, 1)
val name = getSText(args, 0)
Control.Question(
SResultScenarioGetParty(
name,
callback = { party =>
machine.setControl(Control.Value(SParty(party)))
},
callback = (party => machine.setControl(Control.Value(SParty(party)))),
)
)
}
@ -2117,11 +2143,11 @@ private[lf] object SBuiltin {
/** $cacheDisclosedContract[T] :: ContractId T -> CachedContract T -> Unit */
private[speedy] final case class SBCacheDisclosedContract(contractId: V.ContractId)
extends OnLedgerBuiltin(1) {
extends UpdateBuiltin(1) {
override protected def executeWithLedger(
override protected def executeUpdate(
args: util.ArrayList[SValue],
machine: OnLedgerMachine,
machine: UpdateMachine,
): Control = {
val cachedContract = extractCachedContract(args.get(0))
val templateId = cachedContract.templateId

View File

@ -328,10 +328,11 @@ private[lf] object SExpr {
/** Exception handler */
final case class SETryCatch(body: SExpr, handler: SExpr) extends SExpr {
override def execute(machine: Machine): Control = machine.asOnLedger(productPrefix) { machine =>
machine.pushKont(KTryCatchHandler(machine, handler))
machine.ptx = machine.ptx.beginTry
Control.Expression(body)
override def execute(machine: Machine): Control = machine.asUpdateMachine(productPrefix) {
machine =>
machine.pushKont(KTryCatchHandler(machine, handler))
machine.ptx = machine.ptx.beginTry
Control.Expression(body)
}
}

View File

@ -25,8 +25,7 @@ object SResult {
*/
final case class SResultFinal(v: SValue) extends SResult
/** Update or scenario interpretation requires the current
* ledger time.
/** Update interpretation requires the current ledger time.
*/
final case class SResultNeedTime(callback: Time.Timestamp => Unit) extends SResult
@ -58,7 +57,10 @@ object SResult {
callback: SValue => Unit,
) extends SResult
/** Pass the ledger time and return back the new ledger time. */
/** Update interpretation requires the current time. */
final case class SResultScenarioGetTime(callback: Time.Timestamp => Unit) extends SResult
/** Pass the ledger time and return back the new time. */
final case class SResultScenarioPassTime(
relTime: Long,
callback: Time.Timestamp => Unit,

View File

@ -134,7 +134,7 @@ private[lf] object Speedy {
if (actual > limit)
throw SError.SErrorDamlException(IError.Limit(error(limit)))
final class OnLedgerMachine(
final class UpdateMachine(
override val sexpr: SExpr,
override val traceLog: TraceLog,
override val warningLog: WarningLog,
@ -156,13 +156,23 @@ private[lf] object Speedy {
)(implicit loggingContext: LoggingContext)
extends Machine {
private[speedy] override def asUpdateMachine(location: String)(
f: UpdateMachine => Control
): Control =
f(this)
override private[speedy] def asScenarioMachine(location: String)(
f: ScenarioMachine => Control
): Nothing =
throw SErrorCrash(location, "unexpected update machine")
/** unwindToHandler is called when an exception is thrown by the builtin SBThrow or
* re-thrown by the builtin SBTryHandler. If a catch-handler is found, we initiate
* execution of the handler code (which might decide to re-throw). Otherwise we call
* throwUnhandledException to apply the message function to the exception payload,
* producing a text message.
*/
private[speedy] def handleException(excep: SValue.SAny): Control = {
private[speedy] override def handleException(excep: SValue.SAny): Control = {
@tailrec def unwind(): Option[KTryCatchHandler] =
if (kontDepth() == 0) {
None
@ -306,9 +316,9 @@ private[lf] object Speedy {
IError.Limit.ChoiceObservers(cid, templateId, choiceName, arg, observers, _),
)
def finish: Either[SErrorCrash, OnLedgerMachine.Result] = ptx.finish.map { case (tx, seeds) =>
def finish: Either[SErrorCrash, UpdateMachine.Result] = ptx.finish.map { case (tx, seeds) =>
val inputContracts = tx.inputContracts
OnLedgerMachine.Result(
UpdateMachine.Result(
tx,
ptx.locationInfo(),
seeds zip ptx.actionNodeSeeds.toImmArray,
@ -418,7 +428,7 @@ private[lf] object Speedy {
}
}
object OnLedgerMachine {
object UpdateMachine {
@throws[SErrorDamlException]
def apply(
@ -436,11 +446,11 @@ private[lf] object Speedy {
commitLocation: Option[Location] = None,
limits: interpretation.Limits = interpretation.Limits.Lenient,
disclosedContracts: ImmArray[speedy.DisclosedContract],
)(implicit loggingContext: LoggingContext): OnLedgerMachine = {
)(implicit loggingContext: LoggingContext): UpdateMachine = {
val exprWithDisclosures =
compiledPackages.compiler.unsafeCompileWithContractDisclosures(expr, disclosedContracts)
new OnLedgerMachine(
new UpdateMachine(
sexpr = exprWithDisclosures,
validating = validating,
submissionTime = submissionTime,
@ -511,7 +521,7 @@ private[lf] object Speedy {
private[speedy] def toMap: Map[crypto.Hash, SValue.SContractId] = keyMap.toMap
}
final class OffLedgerMachine(
final class ScenarioMachine(
override val sexpr: SExpr,
/* The trace log. */
override val traceLog: TraceLog,
@ -524,11 +534,56 @@ private[lf] object Speedy {
)(implicit loggingContext: LoggingContext)
extends Machine {
/** OffLegder Machine does not handle exceptions */
private[speedy] def handleException(excep: SValue.SAny): Control =
private[speedy] override def asUpdateMachine(location: String)(
f: UpdateMachine => Control
): Nothing =
throw SErrorCrash(location, "unexpected scenario machine")
private[speedy] override def asScenarioMachine(location: String)(
f: ScenarioMachine => Control
): Control = f(this)
/** Scenario Machine does not handle exceptions */
private[speedy] override def handleException(excep: SValue.SAny): Control =
unhandledException(excep)
}
final class PureMachine(
override val sexpr: SExpr,
/* The trace log. */
override val traceLog: TraceLog,
/* Engine-generated warnings. */
override val warningLog: WarningLog,
/* Compiled packages (Daml-LF ast + compiled speedy expressions). */
override var compiledPackages: CompiledPackages,
/* Profile of the run when the packages haven been compiled with profiling enabled. */
override val profile: Profile,
)(implicit loggingContext: LoggingContext)
extends Machine {
private[speedy] override def asUpdateMachine(location: String)(
f: UpdateMachine => Control
): Nothing =
throw SErrorCrash(location, "unexpected pure machine")
private[speedy] override def asScenarioMachine(location: String)(
f: ScenarioMachine => Control
): Nothing =
throw SErrorCrash(location, "unexpected pure machine")
/** Pure Machine does not handle exceptions */
private[speedy] override def handleException(excep: SValue.SAny): Control =
unhandledException(excep)
def runPure(): Either[SError, SValue] =
run() match {
case SResultError(err) => Left(err)
case SResultFinal(v) => Right(v)
case otherwise =>
Left(SErrorCrash(NameOf.qualifiedNameOfCurrentFunc, s"unexpected $otherwise"))
}
}
/** The speedy CEK machine. */
private[lf] sealed abstract class Machine(implicit val loggingContext: LoggingContext) {
@ -606,11 +661,9 @@ private[lf] object Speedy {
@inline
private[speedy] final def kontDepth(): Int = kontStack.size()
final def asOnLedger[T](location: String)(f: OnLedgerMachine => T): T =
this match {
case onLedger: OnLedgerMachine => f(onLedger)
case _ => throw SErrorCrash(location, "unexpected off-ledger machine")
}
private[speedy] def asUpdateMachine(location: String)(f: UpdateMachine => Control): Control
private[speedy] def asScenarioMachine(location: String)(f: ScenarioMachine => Control): Control
@inline
private[speedy] final def pushKont(k: Kont): Unit = {
@ -1095,9 +1148,8 @@ private[lf] object Speedy {
authorizationChecker: AuthorizationChecker = DefaultAuthorizationChecker,
disclosedContracts: ImmArray[speedy.DisclosedContract] = ImmArray.Empty,
limits: interpretation.Limits = interpretation.Limits.Lenient,
)(implicit loggingContext: LoggingContext): OnLedgerMachine = {
)(implicit loggingContext: LoggingContext): UpdateMachine = {
val updateSE: SExpr = compiledPackages.compiler.unsafeCompile(updateE)
fromUpdateSExpr(
compiledPackages,
transactionSeed,
@ -1121,8 +1173,8 @@ private[lf] object Speedy {
disclosedContracts: ImmArray[speedy.DisclosedContract] = ImmArray.Empty,
limits: interpretation.Limits = interpretation.Limits.Lenient,
traceLog: TraceLog = newTraceLog,
)(implicit loggingContext: LoggingContext): OnLedgerMachine = {
OnLedgerMachine(
)(implicit loggingContext: LoggingContext): UpdateMachine = {
UpdateMachine(
compiledPackages = compiledPackages,
submissionTime = Time.Timestamp.MinValue,
initialSeeding = InitialSeeding.TransactionSeed(transactionSeed),
@ -1142,10 +1194,15 @@ private[lf] object Speedy {
def fromScenarioSExpr(
compiledPackages: CompiledPackages,
scenario: SExpr,
)(implicit loggingContext: LoggingContext): OffLedgerMachine =
fromPureSExpr(
traceLog: TraceLog = newTraceLog,
warningLog: WarningLog = newWarningLog,
)(implicit loggingContext: LoggingContext): ScenarioMachine =
new ScenarioMachine(
sexpr = SEApp(scenario, Array(SValue.SToken)),
traceLog = traceLog,
warningLog = warningLog,
compiledPackages = compiledPackages,
expr = SEApp(scenario, Array(SValue.SToken)),
profile = new Profile(),
)
@throws[PackageNotFound]
@ -1154,7 +1211,7 @@ private[lf] object Speedy {
def fromScenarioExpr(
compiledPackages: CompiledPackages,
scenario: Expr,
)(implicit loggingContext: LoggingContext): OffLedgerMachine =
)(implicit loggingContext: LoggingContext): ScenarioMachine =
fromScenarioSExpr(
compiledPackages = compiledPackages,
scenario = compiledPackages.compiler.unsafeCompile(scenario),
@ -1168,15 +1225,14 @@ private[lf] object Speedy {
expr: SExpr,
traceLog: TraceLog = newTraceLog,
warningLog: WarningLog = newWarningLog,
)(implicit loggingContext: LoggingContext): OffLedgerMachine = {
new OffLedgerMachine(
)(implicit loggingContext: LoggingContext): PureMachine =
new PureMachine(
sexpr = expr,
traceLog = traceLog,
warningLog = warningLog,
compiledPackages = compiledPackages,
profile = new Profile(),
)
}
@throws[PackageNotFound]
@throws[CompilationError]
@ -1184,12 +1240,28 @@ private[lf] object Speedy {
def fromPureExpr(
compiledPackages: CompiledPackages,
expr: Expr,
)(implicit loggingContext: LoggingContext): OffLedgerMachine =
)(implicit loggingContext: LoggingContext): PureMachine =
fromPureSExpr(
compiledPackages,
compiledPackages.compiler.unsafeCompile(expr),
)
@throws[PackageNotFound]
@throws[CompilationError]
def runPureExpr(
expr: Expr,
compiledPackages: CompiledPackages = PureCompiledPackages.Empty,
)(implicit loggingContext: LoggingContext): Either[SError, SValue] =
fromPureExpr(compiledPackages, expr).runPure()
@throws[PackageNotFound]
@throws[CompilationError]
def runPureSExpr(
expr: SExpr,
compiledPackages: CompiledPackages = PureCompiledPackages.Empty,
)(implicit loggingContext: LoggingContext): Either[SError, SValue] =
fromPureSExpr(compiledPackages, expr).runPure()
}
// Environment
@ -1602,7 +1674,7 @@ private[lf] object Speedy {
private[speedy] final case class KCacheContract(cid: V.ContractId) extends Kont {
override def execute(machine: Machine, sv: SValue): Control =
machine.asOnLedger(productPrefix) { machine =>
machine.asUpdateMachine(productPrefix) { machine =>
val cached = SBuiltin.extractCachedContract(sv)
machine.checkContractVisibility(cid, cached)
machine.addGlobalContract(cid, cached)
@ -1616,9 +1688,7 @@ private[lf] object Speedy {
handleKeyFound: V.ContractId => Control.Value,
) extends Kont {
override def execute(machine: Machine, sv: SValue): Control =
machine.asOnLedger(productPrefix) { machine =>
machine.checkKeyVisibility(gKey, cid, handleKeyFound)
}
machine.asUpdateMachine(productPrefix)(_.checkKeyVisibility(gKey, cid, handleKeyFound))
}
/** KCloseExercise. Marks an open-exercise which needs to be closed. Either:
@ -1628,7 +1698,7 @@ private[lf] object Speedy {
private[speedy] final case object KCloseExercise extends Kont {
override def execute(machine: Machine, exerciseResult: SValue): Control =
machine.asOnLedger(productPrefix) { machine =>
machine.asUpdateMachine(productPrefix) { machine =>
machine.ptx = machine.ptx.endExercises(exerciseResult.toNormalizedValue)
Control.Value(exerciseResult)
}
@ -1648,13 +1718,13 @@ private[lf] object Speedy {
with SomeArrayEquals
with NoCopy {
// we must restore when catching a throw, or for normal execution
def restore(machine: OnLedgerMachine): Unit = {
def restore(machine: UpdateMachine): Unit = {
machine.restoreBase(savedBase)
machine.restoreFrameAndActuals(frame, actuals)
}
override def execute(machine: Machine, v: SValue): Control =
machine.asOnLedger(productPrefix) { machine =>
machine.asUpdateMachine(productPrefix) { machine =>
restore(machine)
machine.ptx = machine.ptx.endTry
Control.Value(v)

View File

@ -566,7 +566,7 @@ object CompilerTest {
(SValue, Map[ContractId, CachedContract], Map[crypto.Hash, SValue.SContractId]),
] = {
val machine =
Speedy.OnLedgerMachine(
Speedy.UpdateMachine(
compiledPackages = compiledPackages,
submissionTime = Time.Timestamp.MinValue,
initialSeeding = InitialSeeding.TransactionSeed(crypto.Hash.hashPrivateKey("CompilerTest")),

View File

@ -264,7 +264,7 @@ object ExplicitDisclosureLib {
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance] =
PartialFunction.empty,
getKey: PartialFunction[GlobalKeyWithMaintainers, Value.ContractId] = PartialFunction.empty,
): (Either[SError.SError, SValue], Speedy.OnLedgerMachine) = {
): (Either[SError.SError, SValue], Speedy.UpdateMachine) = {
import SpeedyTestLib.loggingContext
// A token function closure is added as part of compiling the Expr
@ -305,7 +305,7 @@ object ExplicitDisclosureLib {
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance] =
PartialFunction.empty,
getKey: PartialFunction[GlobalKeyWithMaintainers, Value.ContractId] = PartialFunction.empty,
): (Either[SError.SError, SValue], Speedy.OnLedgerMachine) = {
): (Either[SError.SError, SValue], Speedy.UpdateMachine) = {
import SpeedyTestLib.loggingContext
val machine =
@ -325,7 +325,7 @@ object ExplicitDisclosureLib {
(result, machine)
}
def haveInactiveContractIds(contractIds: ContractId*): Matcher[Speedy.OnLedgerMachine] = Matcher {
def haveInactiveContractIds(contractIds: ContractId*): Matcher[Speedy.UpdateMachine] = Matcher {
machine =>
val expectedResult = contractIds.toSet
val actualResult = machine.ptx.contractState.activeState.consumedBy.keySet
@ -343,7 +343,7 @@ object ExplicitDisclosureLib {
def haveDisclosedContracts(
disclosedContracts: DisclosedContract*
): Matcher[Speedy.OnLedgerMachine] =
): Matcher[Speedy.UpdateMachine] =
Matcher { machine =>
val expectedResult = ImmArray(disclosedContracts: _*)
val actualResult = machine.ptx.disclosedContracts

View File

@ -6,7 +6,6 @@ package speedy
import com.daml.lf.data._
import com.daml.lf.language.Ast._
import com.daml.lf.speedy.SExpr._
import com.daml.lf.speedy.SValue.{SValue => _, _}
import com.daml.lf.testing.parser.Implicits._
import org.scalatest.Inside.inside
@ -378,19 +377,6 @@ object SBuiltinBigNumericTest {
val compiledPackages =
PureCompiledPackages.assertBuild(Map(defaultParserParameters.defaultPackageId -> pkg))
private def eval(e: Expr, onLedger: Boolean = true): Either[SError.SError, SValue] =
evalSExpr(compiledPackages.compiler.unsafeCompile(e), onLedger)
private def evalSExpr(e: SExpr, onLedger: Boolean): Either[SError.SError, SValue] = {
val machine =
if (onLedger)
Speedy.Machine.fromScenarioSExpr(
compiledPackages,
scenario = SELet1(e, SEMakeClo(Array(SELocS(1)), 1, SELocF(0))),
)
else
Speedy.Machine.fromPureSExpr(compiledPackages, e)
SpeedyTestLib.run(machine)
}
private def eval(e: Expr): Either[SError.SError, SValue] =
Speedy.Machine.runPureExpr(e, compiledPackages)
}

View File

@ -20,12 +20,7 @@ import com.daml.lf.speedy.SBuiltin.{
import com.daml.lf.speedy.SError.{SError, SErrorCrash}
import com.daml.lf.speedy.SExpr._
import com.daml.lf.speedy.SValue.{SValue => _, _}
import com.daml.lf.speedy.Speedy.{
CachedContract,
OnLedgerMachine,
OffLedgerMachine,
SKeyWithMaintainers,
}
import com.daml.lf.speedy.Speedy.{CachedContract, Machine, SKeyWithMaintainers}
import com.daml.lf.testing.parser.Implicits._
import com.daml.lf.transaction.{GlobalKey, GlobalKeyWithMaintainers, TransactionVersion}
import com.daml.lf.value.Value
@ -1316,20 +1311,12 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
"returns None on-ledger" in {
val f = """(\(c:(ContractId Mod:T)) -> CONTRACT_ID_TO_TEXT @Mod:T c)"""
val cid = Value.ContractId.V1(Hash.hashPrivateKey("abc"))
evalApp(
e"$f",
Array(SContractId(cid)),
onLedger = true,
) shouldBe Right(SOptional(None))
evalAppOnLedger(e"$f", Array(SContractId(cid))) shouldBe Right(SOptional(None))
}
"returns Some(abc) off-ledger" in {
val f = """(\(c:(ContractId Mod:T)) -> CONTRACT_ID_TO_TEXT @Mod:T c)"""
val cid = Value.ContractId.V1(Hash.hashPrivateKey("abc"))
evalApp(
e"$f",
Array(SContractId(cid)),
onLedger = false,
) shouldBe Right(SOptional(Some(SText(cid.coid))))
evalApp(e"$f", Array(SContractId(cid))) shouldBe Right(SOptional(Some(SText(cid.coid))))
}
}
@ -1522,9 +1509,7 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
)
forAll(cases) { (builtin, args, name) =>
inside(
evalSExpr(SEAppAtomicSaturatedBuiltin(builtin, args.map(SEValue(_)).toArray), false)
) {
inside(eval(SEAppAtomicSaturatedBuiltin(builtin, args.map(SEValue(_)).toArray))) {
case Left(
SError.SErrorDamlException(
IE.UnhandledException(ValueArithmeticError.typ, ValueArithmeticError(msg))
@ -1547,28 +1532,24 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
"SBCrash" - {
"throws an exception" in {
val result = evalSExpr(
SEApp(SEBuiltin(SBCrash("test message")), Array(SUnit)),
onLedger = false,
)
inside(result) { case Left(SErrorCrash(_, message)) =>
message should endWith("test message")
inside(eval(SEApp(SEBuiltin(SBCrash("test message")), Array(SUnit)))) {
case Left(SErrorCrash(_, message)) =>
message should endWith("test message")
}
}
}
"AnyExceptionMessage" - {
"request unknown packageId" in {
eval(
evalOnLedger(
e"""ANY_EXCEPTION_MESSAGE (to_any_exception @Mod:Exception (Mod:Exception {}))"""
) shouldBe Right(SText("some nice error message"))
eval(
evalOnLedger(
e"""ANY_EXCEPTION_MESSAGE (to_any_exception @Mod:ExceptionAppend (Mod:ExceptionAppend { front = "Hello", back = "world"}))"""
) shouldBe Right(SText("Helloworld"))
inside(
Try(
eval(
evalOnLedger(
e"""ANY_EXCEPTION_MESSAGE (to_any_exception @'-unknown-package-':Mod:Exception ('-unknown-package-':Mod:Exception {}))"""
)
)
@ -1631,19 +1612,15 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
"SBCheckTemplate" - {
"detects templates that exist" in {
val templateId = Ref.Identifier.assertFromString("-pkgId-:Mod:Iou")
evalSExpr(
SEApp(SEBuiltin(SBCheckTemplate(templateId)), Array(SUnit)),
onLedger = false,
eval(
SEApp(SEBuiltin(SBCheckTemplate(templateId)), Array(SUnit))
) shouldBe Right(SBool(true))
}
"detects non-existent templates" in {
val templateId = Ref.Identifier.assertFromString("-pkgId-:Mod:NonExistent")
evalSExpr(
SEApp(SEBuiltin(SBCheckTemplate(templateId)), Array(SUnit)),
onLedger = false,
eval(
SEApp(SEBuiltin(SBCheckTemplate(templateId)), Array(SUnit))
) shouldBe Right(SBool(false))
}
}
@ -1651,19 +1628,15 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
"SBCheckTemplateKey" - {
"detects keys that exist for the template" in {
val templateId = Ref.Identifier.assertFromString("-pkgId-:Mod:IouWithKey")
evalSExpr(
SEApp(SEBuiltin(SBCheckTemplateKey(templateId)), Array(SUnit)),
onLedger = false,
eval(
SEApp(SEBuiltin(SBCheckTemplateKey(templateId)), Array(SUnit))
) shouldBe Right(SBool(true))
}
"detects non-existent template keys" in {
val templateId = Ref.Identifier.assertFromString("-pkgId-:Mod:Iou")
evalSExpr(
SEApp(SEBuiltin(SBCheckTemplateKey(templateId)), Array(SUnit)),
onLedger = false,
eval(
SEApp(SEBuiltin(SBCheckTemplateKey(templateId)), Array(SUnit))
) shouldBe Right(SBool(false))
}
}
@ -1695,7 +1668,7 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
)
inside(
evalSExpr(
evalOnLedger(
SELet1(
cachedContractSExpr,
SEAppAtomic(SEBuiltin(SBCacheDisclosedContract(contractId)), Array(SELocS(1))),
@ -1707,7 +1680,6 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
disclosedContract.argument.toUnnormalizedValue,
)
),
onLedger = true,
)
) { case Right((SUnit, contractCache, disclosedContractKeys)) =>
contractCache shouldBe Map(
@ -1740,7 +1712,7 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
)
inside(
evalSExpr(
evalOnLedger(
SELet1(
cachedContractSExpr,
SEAppAtomic(SEBuiltin(SBCacheDisclosedContract(contractId)), Array(SELocS(1))),
@ -1752,7 +1724,6 @@ class SBuiltinTest extends AnyFreeSpec with Matchers with TableDrivenPropertyChe
disclosedContract.argument.toUnnormalizedValue,
)
),
onLedger = true,
)
) { case Right((SUnit, contractCache, disclosedContractKeys)) =>
contractCache shouldBe Map(
@ -1843,51 +1814,52 @@ object SBuiltinTest {
val compiledPackages: PureCompiledPackages =
PureCompiledPackages.assertBuild(Map(defaultParserParameters.defaultPackageId -> pkg))
private def eval(e: Expr, onLedger: Boolean = true): Either[SError, SValue] =
evalSExpr(compiledPackages.compiler.unsafeCompile(e), onLedger)
private def eval(e: Expr): Either[SError, SValue] =
Machine.runPureExpr(e, compiledPackages)
private def evalApp(
e: Expr,
args: Array[SValue],
onLedger: Boolean = true,
): Either[SError, SValue] =
evalSExpr(SEApp(compiledPackages.compiler.unsafeCompile(e), args), onLedger)
eval(SEApp(compiledPackages.compiler.unsafeCompile(e), args))
val alice: Party = Ref.Party.assertFromString("Alice")
val committers: Set[Party] = Set(alice)
private def evalSExpr(sexpr: SExpr, onLedger: Boolean): Either[SError, SValue] = {
evalSExpr(sexpr, PartialFunction.empty, onLedger).map(_._1)
}
private def eval(sexpr: SExpr): Either[SError, SValue] =
Machine.runPureSExpr(sexpr, compiledPackages)
private def evalSExpr(
private def evalAppOnLedger(
e: Expr,
args: Array[SValue],
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance] = Map.empty,
): Either[SError, SValue] =
evalOnLedger(SEApp(compiledPackages.compiler.unsafeCompile(e), args), getContract).map(_._1)
private def evalOnLedger(
e: Expr,
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance] = Map.empty,
): Either[SError, SValue] =
evalOnLedger(compiledPackages.compiler.unsafeCompile(e), getContract).map(_._1)
private def evalOnLedger(
sexpr: SExpr,
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance],
onLedger: Boolean,
): Either[
SError,
(SValue, Map[ContractId, CachedContract], Map[crypto.Hash, SValue.SContractId]),
] = {
val machine =
if (onLedger) {
Speedy.Machine.fromUpdateSExpr(
compiledPackages,
transactionSeed = crypto.Hash.hashPrivateKey("SBuiltinTest"),
updateSE = SELet1(sexpr, SEMakeClo(Array(SELocS(1)), 1, SELocF(0))),
committers = committers,
)
} else {
Speedy.Machine.fromPureSExpr(compiledPackages, sexpr)
}
Speedy.Machine.fromUpdateSExpr(
compiledPackages,
transactionSeed = crypto.Hash.hashPrivateKey("SBuiltinTest"),
updateSE = SELet1(sexpr, SEMakeClo(Array(SELocS(1)), 1, SELocF(0))),
committers = committers,
)
SpeedyTestLib.run(machine, getContract = getContract).map { value =>
machine match {
case machine: OnLedgerMachine =>
(value, machine.getCachedContracts, machine.disclosureKeyTable.toMap)
case _: OffLedgerMachine =>
(value, Map.empty, Map.empty)
}
}
SpeedyTestLib
.run(machine, getContract = getContract)
.map((_, machine.getCachedContracts, machine.disclosureKeyTable.toMap))
}
def intList(xs: Long*): String =

View File

@ -637,7 +637,7 @@ object SpeedyTest {
val disclosedCachedContract: CachedContract =
buildHouseCachedContract(alice, alice, label = "disclosed-label")
val testLogger: WarningLog = new WarningLog(ContextualizedLogger.createFor("daml.warnings"))
val machine: Speedy.OnLedgerMachine = Speedy.Machine
val machine: Speedy.UpdateMachine = Speedy.Machine
.fromUpdateSExpr(
pkg,
crypto.Hash.hashPrivateKey("VisibilityChecking"),

View File

@ -9,12 +9,11 @@ import data.Ref.{PackageId, TypeConName}
import data.Time
import SResult._
import com.daml.lf.language.{Ast, PackageInterface}
import com.daml.lf.speedy.Speedy.{CachedContract, OnLedgerMachine}
import com.daml.lf.speedy.Speedy.{CachedContract, UpdateMachine}
import com.daml.lf.testing.parser.ParserParameters
import com.daml.lf.validation.{Validation, ValidationError}
import com.daml.lf.value.Value.ContractId
import com.daml.logging.LoggingContext
import com.daml.nameof.NameOf
import transaction.{GlobalKey, GlobalKeyWithMaintainers, SubmittedTransaction}
import value.Value
import scalautil.Statement.discard
@ -96,7 +95,7 @@ private[speedy] object SpeedyTestLib {
case SResultError(err) =>
Left(err)
case _: SResultFinal | _: SResultScenarioGetParty | _: SResultScenarioPassTime |
_: SResultScenarioSubmit =>
_: SResultScenarioSubmit | _: SResultScenarioGetTime =>
throw UnexpectedSResultScenarioX
}
}
@ -106,7 +105,7 @@ private[speedy] object SpeedyTestLib {
@throws[SError.SErrorCrash]
def buildTransaction(
machine: Speedy.Machine,
machine: Speedy.UpdateMachine,
getPkg: PartialFunction[PackageId, CompiledPackages] = PartialFunction.empty,
getContract: PartialFunction[Value.ContractId, Value.VersionedContractInstance] =
PartialFunction.empty,
@ -115,7 +114,7 @@ private[speedy] object SpeedyTestLib {
): Either[SError.SError, SubmittedTransaction] =
runTx(machine, getPkg, getContract, getKey, getTime) match {
case Right(SResultFinal(_)) =>
machine.asOnLedger(NameOf.qualifiedNameOfCurrentFunc)(_.finish.map(_.tx))
machine.finish.map(_.tx)
case Left(err) =>
Left(err)
}
@ -137,10 +136,10 @@ private[speedy] object SpeedyTestLib {
private[speedy] object Implicits {
implicit class AddTestMethodsToMachine(machine: OnLedgerMachine) {
implicit class AddTestMethodsToMachine(machine: UpdateMachine) {
private[lf] def withWarningLog(warningLog: WarningLog): OnLedgerMachine =
new OnLedgerMachine(
private[lf] def withWarningLog(warningLog: WarningLog): UpdateMachine =
new UpdateMachine(
sexpr = machine.sexpr,
traceLog = machine.traceLog,
warningLog = warningLog,
@ -157,7 +156,7 @@ private[speedy] object SpeedyTestLib {
disclosureKeyTable = machine.disclosureKeyTable,
)
def withCachedContracts(cachedContracts: (ContractId, CachedContract)*): OnLedgerMachine = {
def withCachedContracts(cachedContracts: (ContractId, CachedContract)*): UpdateMachine = {
for {
entry <- cachedContracts
(contractId, cachedContract) = entry
@ -168,7 +167,7 @@ private[speedy] object SpeedyTestLib {
private[speedy] def withLocalContractKey(
contractId: ContractId,
key: GlobalKey,
): OnLedgerMachine = {
): UpdateMachine = {
machine.ptx = machine.ptx.copy(
contractState = machine.ptx.contractState.copy(
locallyCreated = machine.ptx.contractState.locallyCreated + contractId,
@ -181,7 +180,7 @@ private[speedy] object SpeedyTestLib {
private[speedy] def withDisclosedContractKeys(
templateId: TypeConName,
disclosedContractKeys: (crypto.Hash, ContractId)*
): OnLedgerMachine = {
): UpdateMachine = {
for {
entry <- disclosedContractKeys
(keyHash, contractId) = entry

View File

@ -203,7 +203,7 @@ object Repl {
def run(
expr: Expr
): (Speedy.Machine, ScenarioRunner.ScenarioResult) = {
): (Speedy.ScenarioMachine, ScenarioRunner.ScenarioResult) = {
val machine =
Speedy.Machine.fromScenarioExpr(
compiledPackages,
@ -520,8 +520,7 @@ object Repl {
def invokeTest(state: State, idAndArgs: Seq[String]): (Boolean, State) = {
buildExprFromTest(state, idAndArgs)
.map { expr =>
val (_, errOrLedger) =
state.scenarioRunner.run(expr)
val (_, errOrLedger) = state.scenarioRunner.run(expr)
errOrLedger match {
case error: ScenarioRunner.ScenarioError =>
println(PrettyScenario.prettyError(error.error).render(128))

View File

@ -14,7 +14,6 @@ import com.daml.lf.value.Value.{ContractId, VersionedContractInstance}
import com.daml.lf.speedy._
import com.daml.lf.speedy.SExpr.{SExpr, SEValue, SEApp}
import com.daml.lf.speedy.SResult._
import com.daml.lf.speedy.Speedy.OnLedgerMachine
import com.daml.lf.transaction.IncompleteTransaction
import com.daml.lf.value.Value
import com.daml.logging.LoggingContext
@ -28,7 +27,7 @@ import scala.util.{Failure, Success, Try}
* @constructor Creates a runner using an instance of [[Speedy.Machine]].
*/
final class ScenarioRunner private (
machine: Speedy.OffLedgerMachine,
machine: Speedy.ScenarioMachine,
initialSeed: crypto.Hash,
) {
import ScenarioRunner._
@ -55,7 +54,7 @@ final class ScenarioRunner private (
case SResultError(err) =>
throw scenario.Error.RunnerException(err)
case SResultNeedTime(callback) =>
case SResultScenarioGetTime(callback) =>
callback(ledger.currentTime)
case SResultScenarioPassTime(delta, callback) =>
@ -97,14 +96,9 @@ final class ScenarioRunner private (
}
}
case SResultNeedPackage(pkgId, context, _) =>
crash(LookupError.MissingPackage.pretty(pkgId, context))
case _: SResultNeedContract =>
crash("SResultNeedContract outside of submission")
case _: SResultNeedKey =>
crash("SResultNeedKey outside of submission")
case _: SResultNeedPackage | _: SResultNeedContract | _: SResultNeedKey |
_: SResultNeedTime =>
crash(s"unexpected $res")
}
}
val endTime = System.nanoTime()
@ -127,7 +121,7 @@ final class ScenarioRunner private (
private[lf] object ScenarioRunner {
def run(
buildMachine: () => Speedy.OffLedgerMachine,
buildMachine: () => Speedy.ScenarioMachine,
initialSeed: crypto.Hash,
)(implicit loggingContext: LoggingContext): ScenarioResult = {
val machine = buildMachine()
@ -183,7 +177,6 @@ private[lf] object ScenarioRunner {
cbPresent: VersionedContractInstance => Unit,
): Either[Error, Unit]
def lookupKey(
machine: Speedy.Machine,
gk: GlobalKey,
actAs: Set[Party],
readAs: Set[Party],
@ -244,7 +237,6 @@ private[lf] object ScenarioRunner {
}
override def lookupKey(
machine: Speedy.Machine,
gk: GlobalKey,
actAs: Set[Party],
readAs: Set[Party],
@ -389,7 +381,7 @@ private[lf] object ScenarioRunner {
warningLog: WarningLog = Speedy.Machine.newWarningLog,
doEnrichment: Boolean = true,
)(implicit loggingContext: LoggingContext): SubmissionResult[R] = {
val ledgerMachine = Speedy.OnLedgerMachine(
val ledgerMachine = Speedy.UpdateMachine(
compiledPackages = compiledPackages,
submissionTime = Time.Timestamp.MinValue,
initialSeeding = InitialSeeding.TransactionSeed(seed),
@ -412,7 +404,7 @@ private[lf] object ScenarioRunner {
ledgerMachine.run() match {
case SResult.SResultFinal(resultValue) =>
ledgerMachine.finish match {
case Right(OnLedgerMachine.Result(tx, locationInfo, _, _, _)) =>
case Right(Speedy.UpdateMachine.Result(tx, locationInfo, _, _, _)) =>
ledger.commit(committers, readAs, location, enrich(tx), locationInfo) match {
case Left(err) =>
SubmissionError(err, enrich(ledgerMachine.incompleteTransaction))
@ -436,7 +428,6 @@ private[lf] object ScenarioRunner {
}
case SResultNeedKey(keyWithMaintainers, committers, callback) =>
ledger.lookupKey(
ledgerMachine,
keyWithMaintainers.globalKey,
committers,
readAs,
@ -448,14 +439,9 @@ private[lf] object ScenarioRunner {
case SResultNeedTime(callback) =>
callback(ledger.currentTime)
go()
case SResultNeedPackage(pkgId, context, _) =>
throw Error.Internal(LookupError.MissingPackage.pretty(pkgId, context))
case _: SResultScenarioGetParty =>
throw Error.Internal("SResultScenarioGetParty in submission")
case _: SResultScenarioPassTime =>
throw Error.Internal("SResultScenarioPassTime in submission")
case _: SResultScenarioSubmit =>
throw Error.Internal("SResultScenarioSubmit in submission")
case res @ (_: SResultNeedPackage | _: SResultScenarioGetParty |
_: SResultScenarioPassTime | _: SResultScenarioSubmit | _: SResultScenarioGetTime) =>
throw Error.Internal(s"unexpected $res")
}
}
go()

View File

@ -16,12 +16,11 @@ import com.daml.lf.transaction.{GlobalKey, NodeId, SubmittedTransaction}
import com.daml.lf.value.Value
import com.daml.lf.value.Value.ContractId
import com.daml.lf.scenario.{ScenarioLedger, ScenarioRunner}
import com.daml.lf.speedy.Speedy.{Machine, Control}
import com.daml.lf.speedy.Speedy.{Control, Machine, ScenarioMachine}
import com.daml.logging.LoggingContext
import java.io.File
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations._
private[lf] class CollectAuthority {
@ -41,7 +40,7 @@ private[lf] class CollectAuthorityState {
private[this] implicit def logContext: LoggingContext = LoggingContext.ForTesting
var machine: Machine = null
var machine: ScenarioMachine = null
var the_sexpr: SExpr = null
@Setup(Level.Trial)
@ -198,7 +197,6 @@ private[lf] class CannedLedgerApi(
Right(callback(coinst))
}
override def lookupKey(
machine: Machine,
gk: GlobalKey,
actAs: Set[Party],
readAs: Set[Party],

View File

@ -425,7 +425,7 @@ private[lf] class Runner(
ec: ExecutionContext,
esf: ExecutionSequencerFactory,
mat: Materializer,
): (Speedy.OffLedgerMachine, Future[SValue]) = {
): (Speedy.PureMachine, Future[SValue]) = {
val machine =
Speedy.Machine.fromPureSExpr(
extendedCompiledPackages,

View File

@ -19,7 +19,7 @@ import com.daml.lf.speedy.SExpr.{SEAppAtomic, SEValue}
import com.daml.lf.speedy.{ArrayList, SError, SValue}
import com.daml.lf.speedy.SExpr.SExpr
import com.daml.lf.speedy.SValue._
import com.daml.lf.speedy.Speedy.OffLedgerMachine
import com.daml.lf.speedy.Speedy.PureMachine
import com.daml.lf.value.Value
import com.daml.lf.value.Value.ContractId
import scalaz.{Foldable, OneAnd}
@ -60,7 +60,7 @@ object ScriptF {
val scriptIds: ScriptIds,
val timeMode: ScriptTimeMode,
private var _clients: Participants[ScriptLedgerClient],
machine: OffLedgerMachine,
machine: PureMachine,
) {
def clients = _clients
def compiledPackages = machine.compiledPackages

View File

@ -162,8 +162,8 @@ class IdeLedgerClient(
None
case res @ (_: SResultNeedPackage | _: SResultNeedContract | _: SResultNeedKey |
_: SResultNeedTime | _: SResultScenarioGetParty | _: SResultScenarioPassTime |
_: SResultScenarioSubmit) =>
_: SResultNeedTime | _: SResultScenarioGetParty | _: SResultScenarioGetTime |
_: SResultScenarioPassTime | _: SResultScenarioSubmit) =>
sys.error(s"computeView: expected SResultFinal, got: $res")
}
}

View File

@ -109,7 +109,7 @@ private[lf] object Machine {
// Run speedy until we arrive at a value.
def stepToValue(
machine: Speedy.OffLedgerMachine
machine: Speedy.PureMachine
)(implicit triggerContext: TriggerLogContext): SValue = {
machine.run() match {
case SResultFinal(v) => v
@ -502,8 +502,7 @@ private[lf] class Runner private (
triggerContext: TriggerLogContext
): UnfoldState[SValue, TriggerContext[SubmitRequest]] = {
def evaluate(se: SExpr): SValue = {
val machine: Speedy.OffLedgerMachine =
Speedy.Machine.fromPureSExpr(compiledPackages, se)
val machine = Speedy.Machine.fromPureSExpr(compiledPackages, se)
// Evaluate it.
machine.setExpressionToEvaluate(se)
Machine.stepToValue(machine)
@ -693,7 +692,7 @@ private[lf] class Runner private (
initialStateArgs,
)
// Prepare a speedy machine for evaluating expressions.
val machine: Speedy.OffLedgerMachine =
val machine: Speedy.PureMachine =
Speedy.Machine.fromPureSExpr(compiledPackages, initialState)
// Evaluate it.
machine.setExpressionToEvaluate(initialState)
@ -738,7 +737,7 @@ private[lf] class Runner private (
Timestamp.assertFromInstant(Runner.getTimeProvider(timeProviderType).getCurrentTime)
val (initialStateFree, evaluatedUpdate) = getInitialStateFreeAndUpdate(acs)
// Prepare another speedy machine for evaluating expressions.
val machine: Speedy.OffLedgerMachine =
val machine: Speedy.PureMachine =
Speedy.Machine.fromPureSExpr(compiledPackages, SEValue(SUnit))
import UnfoldState.{flatMapConcatNode, flatMapConcatNodeOps, toSource, toSourceOps}