add result of exercise to transaction -- fixes #479 (#583)

Modify the DAML Engine, Ledger API and Sandbox to pass the result of the
exercise as a field of the transactions.
This commit is contained in:
Nick Smith 2019-05-06 14:33:44 +02:00 committed by Francesco Mazzoli
parent 32a168a999
commit d725d50be9
32 changed files with 345 additions and 100 deletions

View File

@ -276,7 +276,8 @@ final class Engine {
stakeholders @ _,
signatories @ _,
controllers @ _,
children @ _) =>
children @ _,
exerciseResult @ _) =>
val templateId = template
asValueWithAbsoluteContractIds(chosenVal).flatMap(
absChosenVal =>

View File

@ -57,6 +57,7 @@ final case class CreateEvent[Cid, Val](
* @param children consequence events. note that they're paired with the NodeId of the transaction that originated the event.
* @param stakeholders the stakeholders of the target contract -- must be a subset of witnesses. see comment for `collectEvents`
* @param witnesses additional witnesses induced by parent exercises
* @param exerciseResult result of exercise of the choice. Optional since this feature was introduced in transaction version 6.
*/
final case class ExerciseEvent[Nid, Cid, Val](
contractId: Cid,
@ -67,12 +68,17 @@ final case class ExerciseEvent[Nid, Cid, Val](
isConsuming: Boolean,
children: ImmArray[Nid],
stakeholders: Set[Party],
witnesses: Set[Party])
witnesses: Set[Party],
exerciseResult: Option[Val])
extends Event[Nid, Cid, Val] {
override def mapContractId[Cid2, Val2](
f: Cid => Cid2,
g: Val => Val2): ExerciseEvent[Nid, Cid2, Val2] =
copy(contractId = f(contractId), choiceArgument = g(choiceArgument))
copy(
contractId = f(contractId),
choiceArgument = g(choiceArgument),
exerciseResult = exerciseResult.map(g)
)
override def mapNodeId[Nid2](f: Nid => Nid2): ExerciseEvent[Nid2, Cid, Val] =
copy(children = children.map(f))
@ -166,7 +172,8 @@ object Event {
ne.consuming,
relevantChildren,
stakeholders intersect disclosure(nodeId),
disclosure(nodeId)
disclosure(nodeId),
ne.exerciseResult
)
evts += (nodeId -> evt)
go(relevantChildren ++: remaining)

View File

@ -9,7 +9,7 @@ class EngineInfoTest extends WordSpec with Matchers {
EngineInfo.getClass.getSimpleName should {
"show supported LF, Transaction and Value versions" in {
EngineInfo.show shouldBe
"DAML LF Engine supports LF versions: 0, 0.dev, 1.0, 1.1, 1.2, 1.3, 1.dev; Transaction versions: 1, 2, 3, 4, 5, 6; Value versions: 1, 2, 3, 4"
"DAML LF Engine supports LF versions: 0, 0.dev, 1.0, 1.1, 1.2, 1.3, 1.dev; Transaction versions: 1, 2, 3, 4, 5, 6, 7; Value versions: 1, 2, 3, 4"
}
"toString returns the same value as show" in {

View File

@ -753,7 +753,7 @@ class EngineTest extends WordSpec with Matchers {
postCommitForBob shouldBe 'right
bobView.nodes(Tx.NodeId.unsafeFromIndex(0)) match {
case NodeExercises(coid, _, choice, _, consuming, actingParties, _, _, _, _, children) =>
case NodeExercises(coid, _, choice, _, consuming, actingParties, _, _, _, _, children, _) =>
coid shouldBe AbsoluteContractId(originalCoid)
consuming shouldBe true
actingParties shouldBe Set(bob)
@ -897,6 +897,9 @@ class EngineTest extends WordSpec with Matchers {
children = ImmArray(Tx.NodeId.unsafeFromIndex(1)),
stakeholders = Set(bob, alice),
witnesses = Set(bob, alice),
exerciseResult = Some(
assertAsVersionedValue(
ValueContractId(RelativeContractId(Tx.NodeId.unsafeFromIndex(1)))))
)
val bobVisibleCreate = partyEvents.events(Tx.NodeId.unsafeFromIndex(1))
bobVisibleCreate shouldBe

View File

@ -724,7 +724,7 @@ final case class Compiler(packages: PackageId PartialFunction Package) {
// stack: <actors> <cid> <choice arg> <token> <template arg> ()
update(SEVar(3)),
// stack: <actors> <cid> <choice arg> <token> <template arg> () <ret value>
SBUEndExercise(tmplId)(SEVar(4))
SBUEndExercise(tmplId)(SEVar(4), SEVar(1))
) in
// stack: <actors> <cid> <choice arg> <token> <template arg> () <ret value> ()
SEVar(2)

View File

@ -735,11 +735,21 @@ object SBuiltin {
}
}
/** $endExercise[T] :: Token -> () */
final case class SBUEndExercise(templateId: TypeConName) extends SBuiltin(1) {
/** $endExercise[T]
* :: Token
* -> Value (result of the exercise)
* -> ()
*/
final case class SBUEndExercise(templateId: TypeConName) extends SBuiltin(2) {
def execute(args: util.ArrayList[SValue], machine: Machine): Unit = {
checkToken(args.get(0))
machine.ptx = machine.ptx.endExercises._2
val exerciseResult = args.get(1).toValue
machine.ptx = machine.ptx
.endExercises(asVersionedValue(exerciseResult) match {
case Left(err) => crash(err)
case Right(x) => x
})
._2
machine.ctrl = CtrlValue(SUnit(()))
checkAborted(machine.ptx)
}

View File

@ -227,7 +227,8 @@ object Ledger {
stakeholders = nex.stakeholders,
signatories = nex.signatories,
controllers = nex.controllers,
children = nex.children.map(NodeId(commitPrefix, _))
children = nex.children.map(NodeId(commitPrefix, _)),
exerciseResult = nex.exerciseResult.map(makeAbsolute(commitPrefix, _))
)
case nlbk: NodeLookupByKey.WithTxValue[ContractId] =>
NodeLookupByKey(

View File

@ -4,7 +4,7 @@
DAML-LF Transaction Specification
=================================
**version 6, 29 April 2019**
**version 7, 5 May 2019**
This specification, in concert with the ``transaction.proto``
machine-readable definition, defines a format for _transactions_, to be
@ -157,6 +157,8 @@ This table lists every version of this specification in ascending order
+--------------------+-----------------+
| 6 | 2019-04-29 |
+--------------------+-----------------+
| 7 | 2019-05-06 |
+--------------------+-----------------+
message Transaction
^^^^^^^^^^^^^^^^^^^
@ -486,6 +488,14 @@ The ``controllers`` field must be empty. Software needing to fill in
data structures that demand both actors and controllers must use
the ``actors`` field as the controllers.
*since version 7*
A new field ``result_value`` is required:
* `message VersionedValue`_ result_value
Containing the result of the exercised choice.
message NodeLookupByKey
^^^^^^^^^^^^^^^^^^^^^^^

View File

@ -3,18 +3,17 @@
package com.digitalasset.daml.lf.engine.testing
import com.digitalasset.daml.lf.command._
import com.digitalasset.daml.lf.PureCompiledPackages
import com.digitalasset.daml.lf.data.{FrontStack, FrontStackCons, ImmArray, Time}
import com.digitalasset.daml.lf.command._
import com.digitalasset.daml.lf.data.Ref.{PackageId, Party, QualifiedName, SimpleString}
import com.digitalasset.daml.lf.data.Relation.Relation
import com.digitalasset.daml.lf.engine._
import com.digitalasset.daml.lf.data.{FrontStack, FrontStackCons, ImmArray, Time}
import com.digitalasset.daml.lf.engine.Event.Events
import com.digitalasset.daml.lf.engine._
import com.digitalasset.daml.lf.lfpackage.Ast._
import com.digitalasset.daml.lf.speedy.ScenarioRunner
import com.digitalasset.daml.lf.speedy.Speedy
import com.digitalasset.daml.lf.transaction.{GenTransaction, Transaction => Tx}
import com.digitalasset.daml.lf.speedy.{ScenarioRunner, Speedy}
import com.digitalasset.daml.lf.transaction.Node._
import com.digitalasset.daml.lf.transaction.{GenTransaction, Transaction => Tx}
import com.digitalasset.daml.lf.types.{Ledger => L}
import com.digitalasset.daml.lf.value.Value.{
AbsoluteContractId,
@ -90,11 +89,24 @@ class SemanticTester(
val scenarioLedger = allScenarioLedgers(scenario)
val ledger: GenericLedger = createLedger(packageParties)
//
// Wrapper for the NodeId and a flag used for handling ExerciseNodes.
//
// The children of the exercise node need to be processed before the testing the exercise node as testing
// the result value of the exercise requires knowing the contract IDs of child contracts and records.
//
case class StackNode(
/** Identifier for the Node */
nid: L.NodeId,
/** True if the children of the exercise node has been added to the stack */
exerciseAddedChildren: Boolean
)
case class TestScenarioState(
/** Mapping between scenario contracts and ledger contracts */
scenarioCoidToLedgerCoid: Map[AbsoluteContractId, AbsoluteContractId],
/** Stack of remaining scenario nodes to visit */
remainingScenarioNodeIds: FrontStack[L.NodeId],
remainingScenarioNodeIds: FrontStack[StackNode],
/** Stack of remaining ledger events to visit */
remainingLedgerEventIds: FrontStack[ledger.EventNodeId])
@ -147,7 +159,9 @@ class SemanticTester(
}
// we still have nodes...
case FrontStackCons(scenarioNodeId, remainingScenarioNodeIds) =>
case FrontStackCons(
StackNode(scenarioNodeId, exerciseAddedChildren),
remainingScenarioNodeIds) =>
val scenarioNode = scenarioTransaction.nodes(scenarioNodeId)
// utility to assert that we still have a ledger event,
@ -201,7 +215,7 @@ class SemanticTester(
case scenarioExercisesNode: NodeExercises[
L.NodeId,
AbsoluteContractId,
Tx.Value[AbsoluteContractId]] =>
Tx.Value[AbsoluteContractId]] if exerciseAddedChildren =>
val (ledgerExerciseEvent, remainingLedgerEventIds) =
popEvent[ExerciseEvent[
ledger.EventNodeId,
@ -210,6 +224,7 @@ class SemanticTester(
"exercise event",
scenarioNode,
state.remainingLedgerEventIds)
// create synthetic exercise event, again rewriting the appropriate bits. note that we intentionally
// blank the children because we compare them in the recursive call anyway.
val scenarioExerciseEvent = ExerciseEvent(
@ -222,6 +237,8 @@ class SemanticTester(
ImmArray.empty,
scenarioExercisesNode.stakeholders intersect scenarioWitnesses(scenarioNodeId),
scenarioWitnesses(scenarioNodeId),
scenarioExercisesNode.exerciseResult.map(
_.mapContractId(state.scenarioCoidToLedgerCoid))
)
val ledgerExerciseEventToCompare =
ledgerExerciseEvent.copy(children = ImmArray.empty, stakeholders = Set.empty)
@ -233,11 +250,31 @@ class SemanticTester(
s"Expected exercise event $comparedScenarioExerciseEvent but got $ledgerExerciseEventToCompare"
)
}
// add the exercise children to the stack
state.copy(
remainingLedgerEventIds = ledgerExerciseEvent.children ++: remainingLedgerEventIds,
remainingScenarioNodeIds = scenarioExercisesNode.children ++: remainingScenarioNodeIds,
remainingScenarioNodeIds = remainingScenarioNodeIds,
remainingLedgerEventIds = remainingLedgerEventIds,
)
case scenarioExercisesNode: NodeExercises[
L.NodeId,
AbsoluteContractId,
Tx.Value[AbsoluteContractId]] if !exerciseAddedChildren =>
val exerciseChildren =
popEvent[ExerciseEvent[
ledger.EventNodeId,
AbsoluteContractId,
Tx.Value[AbsoluteContractId]]](
"exercise event",
scenarioNode,
state.remainingLedgerEventIds)._1.children
state.copy(
remainingLedgerEventIds = exerciseChildren ++: state.remainingLedgerEventIds,
remainingScenarioNodeIds = scenarioExercisesNode.children.map(
StackNode(_, false)) ++:
StackNode(scenarioNodeId, true) +: remainingScenarioNodeIds,
)
}
// keep looping
go(nextState)
@ -249,7 +286,7 @@ class SemanticTester(
go(
TestScenarioState(
remainingLedgerEventIds = FrontStack(ledgerEvents.roots),
remainingScenarioNodeIds = FrontStack(scenarioTransaction.roots),
remainingScenarioNodeIds = FrontStack(scenarioTransaction.roots.map(StackNode(_, false))),
scenarioCoidToLedgerCoid = scenarioToLedgerMap
))
}

View File

@ -310,6 +310,7 @@ object ValueGenerators {
.listOf(Arbitrary.arbInt.arbitrary)
.map(_.map(Transaction.NodeId.unsafeFromIndex))
.map(ImmArray(_))
exerciseResultValue <- versionedValueGen
} yield
NodeExercises(
targetCoid,
@ -322,7 +323,8 @@ object ValueGenerators {
stakeholders,
signatories,
actingParties,
children
children,
Some(exerciseResultValue)
)
}

View File

@ -11,7 +11,7 @@
// * 4: string contract_id replaced globally by ContractId message
// * 5: new field actors in NodeFetch
// * 6: removal of controllers in exercise nodes
// * 7: new field exerciseResult in NodeExercise
syntax = "proto3";
package com.digitalasset.daml.lf.transaction;
@ -87,6 +87,7 @@ message NodeExercise {
repeated string signatories = 9;
repeated string controllers = 10;
com.digitalasset.daml.lf.value.ContractId contract_id_struct = 11;
com.digitalasset.daml.lf.value.VersionedValue return_value = 12;
}
message NodeLookupByKey {

View File

@ -87,12 +87,16 @@ object Node {
* is why we removed the controllers field in transaction version 6.
*/
controllers: Set[Party],
children: ImmArray[Nid])
children: ImmArray[Nid],
exerciseResult: Option[Val])
extends GenNode[Nid, Cid, Val] {
override def mapContractIdAndValue[Cid2, Val2](
f: Cid => Cid2,
g: Val => Val2): NodeExercises[Nid, Cid2, Val2] =
copy(targetCoid = f(targetCoid), chosenValue = g(chosenValue))
copy(
targetCoid = f(targetCoid),
chosenValue = g(chosenValue),
exerciseResult = exerciseResult.map(g))
override def mapNodeId[Nid2](f: Nid => Nid2): NodeExercises[Nid2, Cid, Val] =
copy(
@ -116,7 +120,8 @@ object Node {
chosenValue: Val,
stakeholders: Set[Party],
signatories: Set[Party],
children: ImmArray[Nid]): NodeExercises[Nid, Cid, Val] =
children: ImmArray[Nid],
exerciseResult: Option[Val]): NodeExercises[Nid, Cid, Val] =
NodeExercises(
targetCoid,
templateId,
@ -128,7 +133,8 @@ object Node {
stakeholders,
signatories,
actingParties,
children)
children,
exerciseResult)
}
final case class NodeLookupByKey[+Cid, +Val](
@ -202,11 +208,13 @@ object Node {
stakeholders2,
signatories2,
controllers2,
_) =>
_,
exerciseResult2) =>
import ne._
targetCoid === targetCoid2 && templateId == templateId2 && choiceId == choiceId2 &&
consuming == consuming2 && actingParties == actingParties2 && chosenValue === chosenValue2 &&
stakeholders == stakeholders2 && signatories == signatories2 && controllers == controllers2
stakeholders == stakeholders2 && signatories == signatories2 && controllers == controllers2 &&
exerciseResult.fold(true)(_ => exerciseResult === exerciseResult2)
case _ => false
}
case nl: NodeLookupByKey[Cid, Val] =>

View File

@ -471,7 +471,7 @@ object Transaction {
private def computeRoots: Set[NodeId] = {
val allChildNodeIds: Set[NodeId] = nodes.values.flatMap {
case _: LeafOnlyNode[_, _] => Nil
case NodeExercises(_, _, _, _, _, _, _, _, _, _, children) => children.toSeq
case NodeExercises(_, _, _, _, _, _, _, _, _, _, children, _) => children.toSeq
}(breakOut)
nodes.keySet diff allChildNodeIds
@ -683,7 +683,7 @@ object Transaction {
}
}
def endExercises: (Option[NodeId], PartialTransaction) = {
def endExercises(value: Value[ContractId]): (Option[NodeId], PartialTransaction) = {
context match {
case ContextRoot =>
(None, noteAbort(EndExerciseInRootContext))
@ -700,7 +700,8 @@ object Transaction {
ec.stakeholders,
ec.signatories,
ec.controllers,
exercisesChildren
exercisesChildren,
Some(value)
)
val nodeId = ec.exercisesNodeId
val ptx =

View File

@ -17,6 +17,7 @@ import scala.collection.JavaConverters._
import scalaz.syntax.std.boolean._
import scalaz.syntax.traverse.ToTraverseOps
import scalaz.std.either.eitherMonad
import scalaz.std.option._
object TransactionCoder {
@ -99,7 +100,7 @@ object TransactionCoder {
nodeId: Nid,
node: GenNode[Nid, Cid, Val]): Either[EncodeError, TransactionOuterClass.Node] = {
val nodeBuilder = TransactionOuterClass.Node.newBuilder().setNodeId(encodeNid(nodeId))
import TransactionVersions.{minKeyOrLookupByKey, minNoControllers}
import TransactionVersions.{minKeyOrLookupByKey, minNoControllers, minExerciseResult}
node match {
case c: NodeCreate[Cid, Val] =>
encodeContractInstance(encodeVal, c.coinst).flatMap { inst =>
@ -151,35 +152,48 @@ object TransactionCoder {
}
case e: NodeExercises[Nid, Cid, Val] =>
encodeVal(e.chosenValue).flatMap {
case (vversion, arg) =>
val exBuilder =
TransactionOuterClass.NodeExercise
.newBuilder()
.setChoice(e.choiceId)
.setTemplateId(ValueCoder.encodeIdentifier(e.templateId, Some(vversion))._2)
.setChosenValue(arg)
.setConsuming(e.consuming)
.setContractIdOrStruct(encodeCid, transactionVersion, e.targetCoid)(
_.setContractId(_),
_.setContractIdStruct(_))
.addAllActors(e.actingParties.map(_.underlyingString).asJava)
.addAllChildren(e.children.map(encodeNid).toList.asJava)
.addAllSignatories(e.signatories.map(_.underlyingString).asJava)
.addAllStakeholders(e.stakeholders.map(_.underlyingString).asJava)
for {
argValue <- encodeVal(e.chosenValue)
(vversion, arg) = argValue
retValue <- e.exerciseResult traverseU encodeVal
} yield {
val exBuilder =
TransactionOuterClass.NodeExercise
.newBuilder()
.setChoice(e.choiceId)
.setTemplateId(ValueCoder.encodeIdentifier(e.templateId, Some(vversion))._2)
.setChosenValue(arg)
.setConsuming(e.consuming)
.setContractIdOrStruct(encodeCid, transactionVersion, e.targetCoid)(
_.setContractId(_),
_.setContractIdStruct(_))
.addAllActors(e.actingParties.map(_.underlyingString).asJava)
.addAllChildren(e.children.map(encodeNid).toList.asJava)
.addAllSignatories(e.signatories.map(_.underlyingString).asJava)
.addAllStakeholders(e.stakeholders.map(_.underlyingString).asJava)
if (transactionVersion precedes minNoControllers) {
if (e.controllers == e.actingParties) {
exBuilder.addAllControllers(e.controllers.map(_.underlyingString).asJava)
Right(nodeBuilder.setExercise(exBuilder).build())
} else {
Left(EncodeError(
s"As of version $transactionVersion, the controllers and actingParties of an exercise node _must_ be the same, but I got ${e.controllers} as controllers and ${e.actingParties} as actingParties."))
}
if (transactionVersion precedes minNoControllers) {
if (e.controllers == e.actingParties) {
exBuilder.addAllControllers(e.controllers.map(_.underlyingString).asJava)
} else {
Right(nodeBuilder.setExercise(exBuilder).build())
return Left(EncodeError(
s"As of version $transactionVersion, the controllers and actingParties of an exercise node _must_ be the same, but I got ${e.controllers} as controllers and ${e.actingParties} as actingParties."))
}
}
retValue match {
case Some(rv) =>
if (!(transactionVersion precedes minExerciseResult)) {
exBuilder.setReturnValue(rv._2)
}
case None =>
if (!(transactionVersion precedes minExerciseResult)) {
return Left(EncodeError(
s"Trying to encode transaction of version $transactionVersion, which requires the exercise return value, but did not get exercise return value in node."))
}
}
nodeBuilder.setExercise(exBuilder).build()
}
case nlbk: NodeLookupByKey[Cid, Val] =>
@ -231,7 +245,7 @@ object TransactionCoder {
protoNode: TransactionOuterClass.Node): Either[DecodeError, (Nid, GenNode[Nid, Cid, Val])] = {
val nodeId = decodeNid(protoNode.getNodeId)
import TransactionVersions.{minKeyOrLookupByKey, minNoControllers}
import TransactionVersions.{minKeyOrLookupByKey, minNoControllers, minExerciseResult}
protoNode.getNodeTypeCase match {
case NodeTypeCase.CREATE =>
for {
@ -277,6 +291,11 @@ object TransactionCoder {
.map(_.toImmArray)
for {
rv <- if (txVersion precedes minExerciseResult) {
if (protoExe.hasReturnValue)
Left(DecodeError(txVersion, isTooOldFor = "exercise result"))
else Right(None)
} else decodeVal(protoExe.getReturnValue).map(Some(_))
ni <- nodeId
targetCoid <- protoExe.decodeContractIdOrStruct(decodeCid, txVersion)(
_.getContractId,
@ -311,7 +330,8 @@ object TransactionCoder {
stakeholders = stakeholders,
signatories = signatories,
controllers = controllers,
children = children
children = children,
exerciseResult = rv
))
case NodeTypeCase.LOOKUP_BY_KEY =>
val protoLookupByKey = protoNode.getLookupByKey

View File

@ -15,17 +15,19 @@ final case class TransactionVersion(protoValue: String)
*/
object TransactionVersions
extends LfVersions(
maxVersion = TransactionVersion("6"),
previousVersions = List("1", "2", "3", "4", "5") map TransactionVersion)(_.protoValue) {
maxVersion = TransactionVersion("7"),
previousVersions = List("1", "2", "3", "4", "5", "6") map TransactionVersion)(_.protoValue) {
private[this] val minVersion = TransactionVersion("1")
private[transaction] val minKeyOrLookupByKey = TransactionVersion("3")
private[transaction] val minFetchActors = TransactionVersion("5")
private[transaction] val minNoControllers = TransactionVersion("6")
private[transaction] val minExerciseResult = TransactionVersion("7")
def assignVersion(a: GenTransaction[_, _, _ <: VersionedValue[_]]): TransactionVersion = {
require(a != null)
import VersionTimeline.Implicits._
VersionTimeline.latestWhenAllPresent(
minVersion,
// latest version used by any value
@ -45,6 +47,13 @@ object TransactionVersions
.exists { case nf: Node.NodeFetch[_] => nf.actingParties.nonEmpty; case _ => false })
minFetchActors
else minVersion,
if (a.nodes.values
.exists {
case ne: Node.NodeExercises[_, _, _] => ne.exerciseResult.isDefined
case _ => false
})
minExerciseResult
else minVersion,
)
}
}

View File

@ -49,6 +49,7 @@ private[lf] object VersionTimeline {
That(LanguageVersion(LMV.V1, "2")),
Both(This(ValueVersion("4")), LanguageVersion(LMV.V1, "3")),
This(That(TransactionVersion("6"))),
This(That(TransactionVersion("7"))),
That(LanguageVersion(LMV.V1, Dev)),
// add new versions above this line
// do *not* backfill to make more Boths, because such would

View File

@ -6,11 +6,13 @@ package com.digitalasset.daml.lf.transaction
import com.digitalasset.daml.lf.EitherAssertions
import com.digitalasset.daml.lf.data.ImmArray
import com.digitalasset.daml.lf.data.Ref.{Identifier, PackageId, Party, QualifiedName}
import com.digitalasset.daml.lf.transaction.Node.{NodeCreate, NodeExercises, NodeFetch}
import com.digitalasset.daml.lf.transaction.Node.{GenNode, NodeCreate, NodeExercises, NodeFetch}
import com.digitalasset.daml.lf.transaction.{Transaction => Tx, TransactionOuterClass => proto}
import com.digitalasset.daml.lf.value.Value.{ContractId, ContractInst, ValueParty, VersionedValue}
import com.digitalasset.daml.lf.value.ValueCoder.{DecodeCid, DecodeError, EncodeCid, EncodeError}
import com.digitalasset.daml.lf.value.{ValueVersion, ValueVersions}
import com.digitalasset.daml.lf.transaction.TransactionVersions._
import com.digitalasset.daml.lf.transaction.VersionTimeline.Implicits._
import org.scalatest.prop.PropertyChecks
import org.scalatest.{Inside, Matchers, WordSpec}
@ -134,9 +136,12 @@ class TransactionCoderSpec
// fuzzy sort of "failed because of the version override" test
msg should include(txVer.toString)
case Right(encodedTx) =>
val decodedTx = TransactionCoder
.decodeVersionedTransaction(defaultNidDecode, defaultCidDecode, encodedTx)
decodedTx shouldBe Right(VersionedTransaction(txVer, tx))
val decodedVersionedTx = assertRight(
TransactionCoder
.decodeVersionedTransaction(defaultNidDecode, defaultCidDecode, encodedTx))
decodedVersionedTx.transaction shouldBe (if (txVer precedes minExerciseResult)
withoutExerciseResult(tx)
else tx)
}
}
@ -144,8 +149,9 @@ class TransactionCoderSpec
malformedGenTransaction,
transactionVersionGen,
transactionVersionGen) { (tx, txVer1, txVer2) =>
import scalaz.syntax.bifunctor._, scalaz.std.tuple._
import VersionTimeline.Implicits._
import scalaz.std.tuple._
import scalaz.syntax.bifunctor._
whenever(txVer1 != txVer2) {
val orderedVers @ (txvMin, txvMax) =
if (txVer2 precedes txVer1) (txVer2, txVer1) else (txVer1, txVer2)
@ -165,8 +171,12 @@ class TransactionCoderSpec
inside((encWithMin, encWithMax) umap (TransactionCoder
.decodeVersionedTransaction(defaultNidDecode, defaultCidDecode, _))) {
case (Right(decWithMin), Right(decWithMax)) =>
decWithMin.transaction shouldBe tx
decWithMax.transaction shouldBe decWithMin.transaction
decWithMin.transaction shouldBe (if (txvMin precedes minExerciseResult)
withoutExerciseResult(tx)
else tx)
decWithMin.transaction shouldBe (if (txvMin precedes minExerciseResult)
withoutExerciseResult(decWithMax.transaction)
else decWithMax.transaction)
}
}
}
@ -279,4 +289,15 @@ class TransactionCoderSpec
private def changeAllValueVersions(tx: Tx.Transaction, ver: ValueVersion): Tx.Transaction =
tx.mapContractIdAndValue(identity, _.copy(version = ver))
def withoutExerciseResult[Nid, Cid, Val](gn: GenNode[Nid, Cid, Val]): GenNode[Nid, Cid, Val] =
gn match {
case ne: NodeExercises[Nid, Cid, Val] => ne copy (exerciseResult = None)
case _ => gn
}
def withoutExerciseResult[Nid, Cid, Val](
t: GenTransaction[Nid, Cid, Val]): GenTransaction[Nid, Cid, Val] =
t copy (nodes = t.nodes transform ((_, gn) => withoutExerciseResult(gn)))
}

View File

@ -117,7 +117,9 @@ object TransactionSpec {
nodes: Map[String, GenNode[String, String, Value[String]]],
roots: ImmArray[String]): StringTransaction = GenTransaction(nodes, roots, Set.empty)
def dummyExerciseNode(children: ImmArray[String]): NodeExercises[String, String, Value[String]] =
def dummyExerciseNode(
children: ImmArray[String],
hasExerciseResult: Boolean = true): NodeExercises[String, String, Value[String]] =
NodeExercises(
"dummyCoid",
Ref.Identifier(
@ -131,7 +133,8 @@ object TransactionSpec {
Set.empty,
Set.empty,
Set.empty,
children
children,
if (hasExerciseResult) Some(V.ValueUnit) else None
)
val dummyCreateNode: NodeCreate[String, Value[String]] =

View File

@ -25,6 +25,19 @@ class TransactionVersionSpec extends WordSpec with Matchers {
ValueOptional(Some(v)): Value[String])
assignVersion(assignValueVersions(usingOptional)) shouldBe TransactionVersion("2")
}
"pick version 7 when confronted with exercise result" in {
val hasExerciseResult = dummyExerciseWithResultTransaction mapContractIdAndValue (identity, v =>
ValueOptional(Some(v)): Value[String])
assignVersion(assignValueVersions(hasExerciseResult)) shouldBe TransactionVersion("7")
}
"pick version 2 when confronted with exercise result" in {
val hasExerciseResult = dummyExerciseTransaction mapContractIdAndValue (identity, v =>
ValueOptional(Some(v)): Value[String])
assignVersion(assignValueVersions(hasExerciseResult)) shouldBe TransactionVersion("2")
}
}
private[this] def assignValueVersions[Nid, Cid, Cid2](
@ -35,8 +48,13 @@ class TransactionVersionSpec extends WordSpec with Matchers {
}
object TransactionVersionSpec {
import TransactionSpec.{dummyCreateNode, StringTransaction}
import TransactionSpec.{dummyCreateNode, dummyExerciseNode, StringTransaction}
private[this] val singleId = "a"
private val dummyCreateTransaction =
StringTransaction(Map((singleId, dummyCreateNode)), ImmArray(singleId))
private val dummyExerciseWithResultTransaction =
StringTransaction(Map((singleId, dummyExerciseNode(ImmArray.empty))), ImmArray(singleId))
private val dummyExerciseTransaction =
StringTransaction(Map((singleId, dummyExerciseNode(ImmArray.empty, false))), ImmArray(singleId))
}

View File

@ -213,6 +213,7 @@ object TransactionGenerator {
consuming <- Arbitrary.arbBool.arbitrary
(scalaChildren, javaChildren) <- eventsGen
witnessParties <- Gen.listOf(nonEmptyId)
(scalaExerciseResult, javaExerciseResult) <- Gen.sized(valueGen)
} yield
(
Exercised(
@ -226,7 +227,9 @@ object TransactionGenerator {
actingParties,
consuming,
witnessParties,
Nil)), //TODO DEL-6007
Nil, //TODO DEL-6007
Some(scalaExerciseResult)
)),
new data.ExercisedEvent(
witnessParties.asJava,
eventId,
@ -237,8 +240,10 @@ object TransactionGenerator {
javaChoiceArgument,
actingParties.asJava,
consuming,
Collections.emptyList()
) //TODO DEL-6007
Collections.emptyList(),
//TODO DEL-6007
javaExerciseResult
)
)
val eventGen: Gen[(Event, data.Event)] =

View File

@ -43,7 +43,7 @@ public abstract class Event {
exercisedEvent.getContractId(), exercisedEvent.getContractCreatingEventId(),
exercisedEvent.getChoice(), Value.fromProto(exercisedEvent.getChoiceArgument()),
exercisedEvent.getActingPartiesList(), exercisedEvent.getConsuming(),
exercisedEvent.getChildEventIdsList());
exercisedEvent.getChildEventIdsList(), Value.fromProto(exercisedEvent.getExerciseResult()));
} else {
throw new UnsupportedEventTypeException(event.toString());
}
@ -63,7 +63,7 @@ public abstract class Event {
exercisedEvent.getContractId(), exercisedEvent.getContractCreatingEventId(),
exercisedEvent.getChoice(), Value.fromProto(exercisedEvent.getChoiceArgument()),
exercisedEvent.getActingPartiesList(), exercisedEvent.getConsuming(),
exercisedEvent.getChildEventIdsList());
exercisedEvent.getChildEventIdsList(), Value.fromProto(exercisedEvent.getExerciseResult()));
} else {
throw new UnsupportedEventTypeException(event.toString());
}

View File

@ -32,6 +32,8 @@ public class ExercisedEvent extends Event {
private final List<String> childEventIds;
private final Value exerciseResult;
public ExercisedEvent(@NonNull List<@NonNull String> witnessParties,
@NonNull String eventId,
@NonNull Identifier templateId,
@ -41,7 +43,8 @@ public class ExercisedEvent extends Event {
@NonNull Value choiceArgument,
@NonNull List<@NonNull String> actingParties,
boolean consuming,
@NonNull List<@NonNull String> childEventIds) {
@NonNull List<@NonNull String> childEventIds,
@NonNull Value exerciseResult) {
this.witnessParties = witnessParties;
this.eventId = eventId;
this.templateId = templateId;
@ -52,6 +55,7 @@ public class ExercisedEvent extends Event {
this.actingParties = actingParties;
this.consuming = consuming;
this.childEventIds = childEventIds;
this.exerciseResult = exerciseResult;
}
@NonNull
@ -106,6 +110,11 @@ public class ExercisedEvent extends Event {
return consuming;
}
@NonNull
public Value getExerciseResult() {
return exerciseResult;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
@ -119,13 +128,14 @@ public class ExercisedEvent extends Event {
Objects.equals(contractCreatingEventId, that.contractCreatingEventId) &&
Objects.equals(choice, that.choice) &&
Objects.equals(choiceArgument, that.choiceArgument) &&
Objects.equals(actingParties, that.actingParties);
Objects.equals(actingParties, that.actingParties) &&
Objects.equals(exerciseResult, that.exerciseResult);
}
@Override
public int hashCode() {
return Objects.hash(witnessParties, eventId, templateId, contractId, contractCreatingEventId, choice, choiceArgument, actingParties, consuming);
return Objects.hash(witnessParties, eventId, templateId, contractId, contractCreatingEventId, choice, choiceArgument, actingParties, consuming, exerciseResult);
}
@Override
@ -141,6 +151,7 @@ public class ExercisedEvent extends Event {
", actingParties=" + actingParties +
", consuming=" + consuming +
", childEventIds=" + childEventIds +
", exerciseResult=" + exerciseResult +
'}';
}
@ -156,6 +167,7 @@ public class ExercisedEvent extends Event {
.addAllActingParties(getActingParties())
.addAllWitnessParties(getWitnessParties())
.addAllChildEventIds(getChildEventIds())
.setExerciseResult(getExerciseResult().toProto())
.build();
}
@ -170,6 +182,7 @@ public class ExercisedEvent extends Event {
Value.fromProto(exercisedEvent.getChoiceArgument()),
exercisedEvent.getActingPartiesList(),
exercisedEvent.getConsuming(),
exercisedEvent.getChildEventIdsList());
exercisedEvent.getChildEventIdsList(),
Value.fromProto(exercisedEvent.getExerciseResult()));
}
}

View File

@ -236,7 +236,7 @@ object Generators {
isConsuming <- Arbitrary.arbBool.arbitrary
contractCreatingEventId <- Arbitrary.arbString.arbitrary
witnessParties <- Gen.listOf(Arbitrary.arbString.arbitrary)
exerciseResult <- valueGen
} yield
EventOuterClass.ExercisedEvent
.newBuilder()
@ -249,6 +249,7 @@ object Generators {
.setContractCreatingEventId(contractCreatingEventId)
.setEventId(eventId)
.addAllWitnessParties(witnessParties.asJava)
.setExerciseResult(exerciseResult)
.build()
def transactionFilterGen: Gen[TransactionFilterOuterClass.TransactionFilter] =

View File

@ -146,4 +146,9 @@ message ExercisedEvent {
// It contains only the immediate children of this event, not all members of the subtree rooted at this node.
// Optional
repeated string child_event_ids = 11;
// The result of exercising the choice
// Required
Value exercise_result = 12;
}

View File

@ -134,7 +134,16 @@ trait TransactionConversion {
convert(exercise.actingParties),
exercise.isConsuming,
convert(exercise.witnesses),
exercise.children.toSeq.sortBy(getEventIndex)
exercise.children.toSeq.sortBy(getEventIndex),
exercise.exerciseResult.map(
er =>
LfEngineToApi
.lfValueToApiValue(verbose, er.value)
.fold(
err =>
throw new RuntimeException(
s"Unexpected error when converting stored contract: $err"),
identity)),
)
}

View File

@ -1375,6 +1375,41 @@ class TransactionServiceIT
}
}
}
"reading transactions" should {
"serve a stream of transactions" in allFixtures { context =>
val treesF = context.transactionClient
.getTransactionTrees(
ledgerBegin,
Some(ledgerEnd),
TransactionFilter(Map("Bob" -> Filters())))
.runWith(Sink.seq)
val txsF = context.transactionClient
.getTransactions(
ledgerBegin,
Some(ledgerEnd),
TransactionFilter(Map("Bob" -> Filters())))
.runWith(Sink.seq)
for {
txs <- txsF
trees <- treesF
_ = txs.map(_.transactionId) shouldEqual trees.map(_.transactionId)
} yield {
for {
tx <- txs
exEvents = context.testingHelpers.exercisedEventsIn(tx)
_ = exEvents should not be empty
exEvent <- exEvents
} yield {
exEvent.exerciseResult should not be empty
}
}
succeed
}
}
}
}

View File

@ -109,7 +109,10 @@ class ApiScenarioTransform(ledgerId: String, packages: Map[Ref.PackageId, Ast.Pa
convertEvId: String => EventId)
: Either[StatusRuntimeException, P.ExerciseEvent[EventId, AbsoluteContractId]] = {
val witnesses = P.parties(exercisedEvent.witnessParties)
toLfVersionedValue(exercisedEvent.getChoiceArgument).map { value =>
for {
value <- toLfVersionedValue(exercisedEvent.getChoiceArgument)
result <- toLfVersionedValue(exercisedEvent.getExerciseResult)
} yield {
P.ExerciseEvent(
AbsoluteContractId(exercisedEvent.contractId),
Ref.Identifier(
@ -126,7 +129,8 @@ class ApiScenarioTransform(ledgerId: String, packages: Map[Ref.PackageId, Ast.Pa
ImmArray(exercisedEvent.childEventIds.map(convertEvId)),
// conversion is imperfect as stakeholders are not determinable from events yet
witnesses,
witnesses
witnesses,
Some(result)
)
}
}

View File

@ -15,7 +15,7 @@ import com.digitalasset.ledger.api.v1.command_service.{
import com.digitalasset.ledger.api.v1.command_submission_service.SubmitRequest
import com.digitalasset.ledger.api.v1.commands.{CreateCommand, ExerciseCommand}
import com.digitalasset.ledger.api.v1.completion.Completion
import com.digitalasset.ledger.api.v1.event.Event.Event.{Archived, Created}
import com.digitalasset.ledger.api.v1.event.Event.Event.{Archived, Created, Exercised}
import com.digitalasset.ledger.api.v1.event.{ArchivedEvent, CreatedEvent, Event, ExercisedEvent}
import com.digitalasset.ledger.api.v1.ledger_offset.LedgerOffset
import com.digitalasset.ledger.api.v1.testing.time_service.TimeServiceGrpc.TimeService
@ -283,6 +283,11 @@ class LedgerTestingHelpers(
case TreeEvent.Kind.Exercised(exercisedEvent) => exercisedEvent
}(breakOut)
def exercisedEventsIn(transaction: Transaction): Seq[ExercisedEvent] =
transaction.events.map(_.event).collect {
case Exercised(exercisedEvent) => exercisedEvent
}
def archivedEventsIn(transaction: Transaction): Seq[ArchivedEvent] =
transaction.events.map(_.event).collect {
case Archived(archivedEvent) => archivedEvent

View File

@ -43,7 +43,7 @@ object Transaction {
.explicitDisclosure(nodeEntry._1)
.intersect(node.stakeholders)))
)
case node @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _) =>
case node @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _, _) =>
TxDelta(
// get consumed external contracts
if (node.consuming) node.targetCoid match {

View File

@ -43,7 +43,7 @@ object Transaction {
delta.outputs + (node.coid ->
Contract(node.coinst, node.stakeholders))
)
case node @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _) =>
case node @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _, _) =>
TxDelta(
// get consumed external contracts
if (node.consuming) node.targetCoid match {

View File

@ -206,7 +206,8 @@ class EventConverterSpec
Set("Alice"),
Set("Alice"),
Set("Alice"),
ImmArray(Transaction.NodeId.unsafeFromIndex(1))
ImmArray(Transaction.NodeId.unsafeFromIndex(1)),
Some(asVersionedValueOrThrow(Lf.ValueUnit))
))
val node1 = (
@ -306,7 +307,11 @@ class EventConverterSpec
true,
ImmArray("#txId:2"),
Set("giver", "receiver"),
Set("giver", "receiver")
Set("giver", "receiver"),
Some(
asVersionedValueOrThrow(
Lf.ValueContractId(Lf.AbsoluteContractId("#txId:3"))
))
)
val events: Events[EventId, Lf.AbsoluteContractId, Lf.VersionedValue[Lf.AbsoluteContractId]] =
Events(
@ -317,13 +322,17 @@ class EventConverterSpec
Ref.Identifier(
"0d25e199ed26977b3082864c62f8d154ca6042ed521712e2b3eb172dc79c87a2",
"Test:TriProposal"),
"Accept",
"TriProposalAccept",
asVersionedValueOrThrow(Lf.ValueUnit),
Set("receiver", "giver"),
true,
ImmArray("#txId:3"),
Set("receiver", "giver", "operator"),
Set("receiver", "giver", "operator")
Set("receiver", "giver", "operator"),
Some(
asVersionedValueOrThrow(
Lf.ValueContractId(Lf.AbsoluteContractId("#txId:3"))
))
),
"#txId:3" -> CreateEvent(
Lf.AbsoluteContractId("#txId:3"),
@ -384,12 +393,13 @@ class EventConverterSpec
moduleName = "Test",
entityName = "TriProposal")),
"#6:0",
"Accept",
"TriProposalAccept",
Some(Value(Value.Sum.Unit(Empty()))),
Vector("receiver", "giver"),
consuming = true,
Vector("receiver", "giver", "operator"),
List(created.eventId)
List(created.eventId),
Some(Value(ContractId("#txId:3")))
)
val topLevelExercise = ExercisedEvent(
"#txId:0",
@ -415,7 +425,8 @@ class EventConverterSpec
Vector("giver"),
consuming = true,
Vector("giver", "receiver"),
List(nestedExercise.eventId)
List(nestedExercise.eventId),
Some(Value(ContractId("#txId:3")))
)
val expected = TransactionTreeNodes(

View File

@ -323,7 +323,11 @@ class PostgresDaoSpec
VersionedValue(ValueVersions.acceptedVersions.head, ValueText("some choice value")),
Set(SimpleString.assertFromString("Alice"), SimpleString.assertFromString("Bob")),
Set(SimpleString.assertFromString("Alice"), SimpleString.assertFromString("Bob")),
ImmArray.empty
ImmArray.empty,
Some(
VersionedValue(
ValueVersions.acceptedVersions.head,
ValueText("some exercise result"))),
)),
ImmArray(s"event$id"),
Set.empty