LF: Flatten ContractInstance inside NodeCreate (#8526)

The ContractInstance is a concept a bit off. For instance it is not
clear why `agreementText` is more important for a contract that the
signatories, the stackholder or the key. In this PR, we get ride of
the ContractInstance inside LF code, except from the Engine/Ledger API
(com.daml.lf.engine.ResultNeedContract$)

CHANGELOG_BEGIN
CHANGELOG_END
This commit is contained in:
Remy 2021-01-18 17:29:32 +01:00 committed by GitHub
parent 720cb71fb4
commit 441f503a95
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 157 additions and 96 deletions

View File

@ -6,7 +6,7 @@ package engine
import com.daml.lf.data.Ref.{Identifier, Name, PackageId} import com.daml.lf.data.Ref.{Identifier, Name, PackageId}
import com.daml.lf.language.Ast import com.daml.lf.language.Ast
import com.daml.lf.language.Ast.PackageSignature import com.daml.lf.language.Ast.{PackageSignature, TTyCon}
import com.daml.lf.transaction.Node.{GenNode, KeyWithMaintainers} import com.daml.lf.transaction.Node.{GenNode, KeyWithMaintainers}
import com.daml.lf.transaction.{CommittedTransaction, Node, NodeId, VersionedTransaction} import com.daml.lf.transaction.{CommittedTransaction, Node, NodeId, VersionedTransaction}
import com.daml.lf.value.Value import com.daml.lf.value.Value
@ -111,9 +111,9 @@ final class ValueEnricher(engine: Engine) {
node match { node match {
case create: Node.NodeCreate[ContractId] => case create: Node.NodeCreate[ContractId] =>
for { for {
contractInstance <- enrichContract(create.coinst) arg <- enrichValue(TTyCon(create.templateId), create.arg)
key <- enrichContractKey(create.templateId, create.key) key <- enrichContractKey(create.templateId, create.key)
} yield create.copy(coinst = contractInstance, key = key) } yield create.copy(arg = arg, key = key)
case fetch: Node.NodeFetch[ContractId] => case fetch: Node.NodeFetch[ContractId] =>
for { for {
key <- enrichContractKey(fetch.templateId, fetch.key) key <- enrichContractKey(fetch.templateId, fetch.key)

View File

@ -37,19 +37,20 @@ private[preprocessing] final class TransactionPreprocessor(
node match { node match {
case Node.NodeCreate( case Node.NodeCreate(
coid @ _, coid @ _,
coinst, templateId,
arg,
agreementText @ _,
optLoc @ _, optLoc @ _,
sigs @ _, sigs @ _,
stks @ _, stks @ _,
key @ _, key @ _,
version @ _, version @ _,
) => ) =>
val identifier = coinst.template
if (globalCids(coid)) if (globalCids(coid))
fail("Conflicting discriminators between a global and local contract ID.") fail("Conflicting discriminators between a global and local contract ID.")
val (cmd, newCids) = val (cmd, newCids) =
commandPreprocessor.unsafePreprocessCreate(identifier, coinst.arg) commandPreprocessor.unsafePreprocessCreate(templateId, arg)
val newGlobalCids = globalCids + coid val newGlobalCids = globalCids + coid
val newLocalCids = localCids | newCids.filterNot(globalCids) val newLocalCids = localCids | newCids.filterNot(globalCids)
cmd -> (newLocalCids -> newGlobalCids) cmd -> (newLocalCids -> newGlobalCids)

View File

@ -1258,9 +1258,9 @@ class EngineTest
} }
findNodeByIdx(bobView.nodes, 1).getOrElse(fail("node not found")) match { findNodeByIdx(bobView.nodes, 1).getOrElse(fail("node not found")) match {
case Node.NodeCreate(_, coins, _, _, stakeholders, _, _) => case create: Node.NodeCreate[ContractId] =>
coins.template shouldBe templateId create.templateId shouldBe templateId
stakeholders shouldBe Set(alice, clara) create.stakeholders shouldBe Set(alice, clara)
case _ => fail("create event is expected") case _ => fail("create event is expected")
} }
@ -1270,9 +1270,9 @@ class EngineTest
claraView.nodes.size shouldBe 1 claraView.nodes.size shouldBe 1
findNodeByIdx(claraView.nodes, 1).getOrElse(fail("node not found")) match { findNodeByIdx(claraView.nodes, 1).getOrElse(fail("node not found")) match {
case Node.NodeCreate(_, coins, _, _, stakeholders, _, _) => case create: Node.NodeCreate[ContractId] =>
coins.template shouldBe templateId create.templateId shouldBe templateId
stakeholders shouldBe Set(alice, clara) create.stakeholders shouldBe Set(alice, clara)
case _ => fail("create event is expected") case _ => fail("create event is expected")
} }
} }

View File

@ -101,8 +101,8 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
seed = hash("testLargeTransactionOneContract:create", txSize), seed = hash("testLargeTransactionOneContract:create", txSize),
) )
val contractId = firstRootNode(createCmdTx) match { val contractId = firstRootNode(createCmdTx) match {
case N.NodeCreate(coid, _, _, _, _, _, _) => coid case create: N.NodeCreate[ContractId] => create.coid
case n @ _ => fail(s"Expected NodeCreate, but got: $n") case n => fail(s"Expected NodeCreate, but got: $n")
} }
val exerciseCmd = toListContainerExerciseCmd(rangeOfIntsTemplateId, contractId) val exerciseCmd = toListContainerExerciseCmd(rangeOfIntsTemplateId, contractId)
val (exerciseCmdTx, quanity) = measureWithResult( val (exerciseCmdTx, quanity) = measureWithResult(
@ -131,7 +131,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
seed = hash("testLargeTransactionManySmallContracts:create", num), seed = hash("testLargeTransactionManySmallContracts:create", num),
) )
val contractId = firstRootNode(createCmdTx) match { val contractId = firstRootNode(createCmdTx) match {
case N.NodeCreate(coid, _, _, _, _, _, _) => coid case create: N.NodeCreate[ContractId] => create.coid
case n @ _ => fail(s"Expected NodeCreate, but got: $n") case n @ _ => fail(s"Expected NodeCreate, but got: $n")
} }
val exerciseCmd = toListOfIntContainers(rangeOfIntsTemplateId, contractId) val exerciseCmd = toListOfIntContainers(rangeOfIntsTemplateId, contractId)
@ -161,7 +161,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
seed = hash("testLargeChoiceArgument:create", size), seed = hash("testLargeChoiceArgument:create", size),
) )
val contractId = firstRootNode(createCmdTx) match { val contractId = firstRootNode(createCmdTx) match {
case N.NodeCreate(coid, _, _, _, _, _, _) => coid case create: N.NodeCreate[ContractId] => create.coid
case n @ _ => fail(s"Expected NodeCreate, but got: $n") case n @ _ => fail(s"Expected NodeCreate, but got: $n")
} }
val exerciseCmd = sizeExerciseCmd(listUtilTemplateId, contractId)(size) val exerciseCmd = sizeExerciseCmd(listUtilTemplateId, contractId)(size)
@ -208,7 +208,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
} }
newContracts.count { newContracts.count {
case N.NodeCreate(_, _, _, _, _, _, _) => true case _: N.NodeCreate[ContractId] => true
case n @ _ => fail(s"Unexpected match: $n") case n @ _ => fail(s"Unexpected match: $n")
} shouldBe expectedNumberOfContracts } shouldBe expectedNumberOfContracts
} }
@ -310,11 +310,11 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
fieldName: String, fieldName: String,
): Value[ContractId] = { ): Value[ContractId] = {
val contractInst: ContractInst[Value[ContractId]] = val arg: Value[ContractId] =
extractResultFromExerciseTransaction(exerciseCmdTx) extractResultFromExerciseTransaction(exerciseCmdTx)
val fields: ImmArray[(Option[String], Value[ContractId])] = val fields: ImmArray[(Option[String], Value[ContractId])] =
contractInst.arg match { arg match {
case ValueRecord(_, x: ImmArray[_]) => x case ValueRecord(_, x: ImmArray[_]) => x
case v @ _ => fail(s"Unexpected match: $v") case v @ _ => fail(s"Unexpected match: $v")
} }
@ -331,7 +331,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
private def extractResultFromExerciseTransaction( private def extractResultFromExerciseTransaction(
exerciseCmdTx: Transaction exerciseCmdTx: Transaction
): ContractInst[Value[ContractId]] = { ): Value[ContractId] = {
exerciseCmdTx.roots.length shouldBe 1 exerciseCmdTx.roots.length shouldBe 1
exerciseCmdTx.nodes.size shouldBe 2 exerciseCmdTx.nodes.size shouldBe 2
@ -342,7 +342,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
} }
createNode match { createNode match {
case N.NodeCreate(_, x: ContractInst[_], _, _, _, _, _) => x case create: N.NodeCreate[ContractId] => create.arg
case n @ _ => fail(s"Unexpected match: $n") case n @ _ => fail(s"Unexpected match: $n")
} }
} }

View File

@ -907,8 +907,9 @@ private[lf] object SBuiltin {
val (coid, newPtx) = onLedger.ptx val (coid, newPtx) = onLedger.ptx
.insertCreate( .insertCreate(
auth = auth, auth = auth,
coinst = templateId = templateId,
V.ContractInst(template = templateId, arg = createArgValue, agreementText = agreement), arg = createArgValue,
agreementText = agreement,
optLocation = machine.lastLocation, optLocation = machine.lastLocation,
signatories = sigs, signatories = sigs,
stakeholders = sigs union obs, stakeholders = sigs union obs,

View File

@ -241,13 +241,15 @@ private[lf] case class PartialTransaction(
*/ */
def insertCreate( def insertCreate(
auth: Option[Authorize], auth: Option[Authorize],
coinst: Value.ContractInst[Value[Value.ContractId]], templateId: Ref.Identifier,
arg: Value[Value.ContractId],
agreementText: String,
optLocation: Option[Location], optLocation: Option[Location],
signatories: Set[Party], signatories: Set[Party],
stakeholders: Set[Party], stakeholders: Set[Party],
key: Option[Node.KeyWithMaintainers[Value[Nothing]]], key: Option[Node.KeyWithMaintainers[Value[Nothing]]],
): Either[String, (Value.ContractId, PartialTransaction)] = { ): Either[String, (Value.ContractId, PartialTransaction)] = {
val serializableErrs = serializable(coinst.arg) val serializableErrs = serializable(arg)
if (serializableErrs.nonEmpty) { if (serializableErrs.nonEmpty) {
Left( Left(
s"""Trying to create a contract with a non-serializable value: ${serializableErrs.iterator s"""Trying to create a contract with a non-serializable value: ${serializableErrs.iterator
@ -260,12 +262,14 @@ private[lf] case class PartialTransaction(
val cid = Value.ContractId.V1(discriminator) val cid = Value.ContractId.V1(discriminator)
val createNode = Node.NodeCreate( val createNode = Node.NodeCreate(
cid, cid,
coinst, templateId,
arg,
agreementText,
optLocation, optLocation,
signatories, signatories,
stakeholders, stakeholders,
key, key,
packageToTransactionVersion(coinst.template.packageId), packageToTransactionVersion(templateId.packageId),
) )
val nid = NodeId(nextNodeIdx) val nid = NodeId(nextNodeIdx)
val ptx = copy( val ptx = copy(
@ -280,7 +284,7 @@ private[lf] case class PartialTransaction(
key match { key match {
case None => Right((cid, ptx)) case None => Right((cid, ptx))
case Some(kWithM) => case Some(kWithM) =>
val ck = GlobalKey(coinst.template, kWithM.key) val ck = GlobalKey(templateId, kWithM.key)
Right((cid, ptx.copy(keys = ptx.keys.updated(ck, Some(cid))))) Right((cid, ptx.copy(keys = ptx.keys.updated(ck, Some(cid)))))
} }
} }

View File

@ -95,11 +95,9 @@ final class TransactionBuilder(
val templateId = Ref.Identifier.assertFromString(template) val templateId = Ref.Identifier.assertFromString(template)
Create( Create(
coid = ContractId.assertFromString(id), coid = ContractId.assertFromString(id),
coinst = ContractInst( templateId = templateId,
template = templateId, arg = argument,
arg = argument, agreementText = "",
agreementText = "",
),
optLocation = None, optLocation = None,
signatories = signatories.map(Ref.Party.assertFromString).toSet, signatories = signatories.map(Ref.Party.assertFromString).toSet,
stakeholders = signatories.toSet.union(observers.toSet).map(Ref.Party.assertFromString), stakeholders = signatories.toSet.union(observers.toSet).map(Ref.Party.assertFromString),

View File

@ -299,11 +299,23 @@ object ValueGenerators {
for { for {
version <- transactionVersionGen() version <- transactionVersionGen()
coid <- coidGen coid <- coidGen
coinst <- contractInstanceGen templateId <- idGen
arg <- valueGen
agreement <- Arbitrary.arbitrary[String]
signatories <- genNonEmptyParties signatories <- genNonEmptyParties
stakeholders <- genNonEmptyParties stakeholders <- genNonEmptyParties
key <- Gen.option(keyWithMaintainersGen) key <- Gen.option(keyWithMaintainersGen)
} yield NodeCreate(coid, coinst, None, signatories, stakeholders, key, version) } yield NodeCreate(
coid,
templateId,
arg,
agreement,
None,
signatories,
stakeholders,
key,
version,
)
} }
val fetchNodeGen: Gen[NodeFetch[ContractId]] = { val fetchNodeGen: Gen[NodeFetch[ContractId]] = {

View File

@ -58,7 +58,9 @@ object Node {
): GenNode[A1, A2] => GenNode[B1, B2] = { ): GenNode[A1, A2] => GenNode[B1, B2] = {
case self @ NodeCreate( case self @ NodeCreate(
coid, coid,
coinst, _,
arg,
_,
_, _,
_, _,
_, _,
@ -67,7 +69,7 @@ object Node {
) => ) =>
self.copy( self.copy(
coid = f2(coid), coid = f2(coid),
coinst = Value.ContractInst.map1(Value.map1(f2))(coinst), arg = Value.map1(f2)(arg),
key = key.map(KeyWithMaintainers.map1(Value.map1(f2))), key = key.map(KeyWithMaintainers.map1(Value.map1(f2))),
) )
case self @ NodeFetch( case self @ NodeFetch(
@ -128,7 +130,9 @@ object Node {
): GenNode[A, B] => Unit = { ): GenNode[A, B] => Unit = {
case NodeCreate( case NodeCreate(
coid, coid,
coinst, templateI @ _,
arg,
agreementText @ _,
optLocation @ _, optLocation @ _,
signatories @ _, signatories @ _,
stakeholders @ _, stakeholders @ _,
@ -136,7 +140,7 @@ object Node {
_, _,
) => ) =>
f2(coid) f2(coid)
Value.ContractInst.foreach1(Value.foreach1(f2))(coinst) Value.foreach1(f2)(arg)
key.foreach(KeyWithMaintainers.foreach1(Value.foreach1(f2))) key.foreach(KeyWithMaintainers.foreach1(Value.foreach1(f2)))
case NodeFetch( case NodeFetch(
coid, coid,
@ -191,7 +195,9 @@ object Node {
/** Denotes the creation of a contract instance. */ /** Denotes the creation of a contract instance. */
final case class NodeCreate[+Cid]( final case class NodeCreate[+Cid](
coid: Cid, coid: Cid,
coinst: Value.ContractInst[Value[Cid]], override val templateId: TypeConName,
arg: Value[Cid],
agreementText: String,
optLocation: Option[Location], // Optional location of the create expression optLocation: Option[Location], // Optional location of the create expression
signatories: Set[Party], signatories: Set[Party],
stakeholders: Set[Party], stakeholders: Set[Party],
@ -201,14 +207,16 @@ object Node {
) extends LeafOnlyNode[Cid] ) extends LeafOnlyNode[Cid]
with NodeInfo.Create { with NodeInfo.Create {
override def templateId: TypeConName = coinst.template
override def byKey: Boolean = false override def byKey: Boolean = false
override private[lf] def updateVersion(version: TransactionVersion): NodeCreate[Cid] = override private[lf] def updateVersion(version: TransactionVersion): NodeCreate[Cid] =
copy(version = version) copy(version = version)
def coinst: Value.ContractInst[Value[Cid]] =
Value.ContractInst(templateId, arg, agreementText)
def versionedCoinst: Value.ContractInst[Value.VersionedValue[Cid]] = def versionedCoinst: Value.ContractInst[Value.VersionedValue[Cid]] =
Value.ContractInst.map1(versionValue)(coinst) Value.ContractInst(templateId, versionValue(arg), agreementText)
def versionedKey: Option[KeyWithMaintainers[Value.VersionedValue[Cid]]] = def versionedKey: Option[KeyWithMaintainers[Value.VersionedValue[Cid]]] =
key.map(KeyWithMaintainers.map1(versionValue)) key.map(KeyWithMaintainers.map1(versionValue))

View File

@ -233,7 +233,7 @@ final case class GenTransaction[Nid, +Cid](
fold(z) { case (z, (_, n)) => fold(z) { case (z, (_, n)) =>
n match { n match {
case c: Node.NodeCreate[_] => case c: Node.NodeCreate[_] =>
val z1 = f(z, c.coinst.arg) val z1 = f(z, c.arg)
val z2 = c.key match { val z2 = c.key match {
case None => z1 case None => z1
case Some(k) => f(z1, k.key) case Some(k) => f(z1, k.key)
@ -321,7 +321,7 @@ sealed abstract class HasTxNodes[Nid, +Cid] {
final def localContracts[Cid2 >: Cid]: Map[Cid2, Nid] = final def localContracts[Cid2 >: Cid]: Map[Cid2, Nid] =
fold(Map.empty[Cid2, Nid]) { fold(Map.empty[Cid2, Nid]) {
case (acc, (nid, create @ Node.NodeCreate(_, _, _, _, _, _, _))) => case (acc, (nid, create: Node.NodeCreate[Cid])) =>
acc.updated(create.coid, nid) acc.updated(create.coid, nid)
case (acc, _) => acc case (acc, _) => acc
} }
@ -456,8 +456,8 @@ object GenTransaction extends value.CidContainer2[GenTransaction] {
tx.fold(State(Set.empty, Set.empty)) { case (state, (_, node)) => tx.fold(State(Set.empty, Set.empty)) { case (state, (_, node)) =>
node match { node match {
case Node.NodeCreate(_, c, _, _, _, Some(key), _) => case Node.NodeCreate(_, tmplId, _, _, _, _, _, Some(key), _) =>
state.created(globalKey(c.template, key.key)) state.created(globalKey(tmplId, key.key))
case Node.NodeExercises(_, tmplId, _, _, true, _, _, _, _, _, _, _, Some(key), _, _) => case Node.NodeExercises(_, tmplId, _, _, true, _, _, _, _, _, _, _, Some(key), _, _) =>
state.consumed(globalKey(tmplId, key.key)) state.consumed(globalKey(tmplId, key.key))
case Node.NodeExercises(_, tmplId, _, _, false, _, _, _, _, _, _, _, Some(key), _, _) => case Node.NodeExercises(_, tmplId, _, _, false, _, _, _, _, _, _, _, Some(key), _, _) =>
@ -578,7 +578,9 @@ object Transaction {
case ( case (
Node.NodeCreate( Node.NodeCreate(
coid1, coid1,
coinst1, tmplId1,
arg1,
agreementText1,
optLocation1 @ _, optLocation1 @ _,
signatories1, signatories1,
stakeholders1, stakeholders1,
@ -587,7 +589,9 @@ object Transaction {
), ),
Node.NodeCreate( Node.NodeCreate(
coid2, coid2,
coinst2, tmplId2,
arg2,
agreementText2,
optLocation2 @ _, optLocation2 @ _,
signatories2, signatories2,
stakeholders2, stakeholders2,
@ -597,7 +601,9 @@ object Transaction {
) )
if version1 == version2 && if version1 == version2 &&
coid1 === coid2 && coid1 === coid2 &&
coinst1 === coinst2 && tmplId1 == tmplId2 &&
arg1 === arg2 &&
agreementText1 == agreementText2 &&
signatories1 == signatories2 && signatories1 == signatories2 &&
stakeholders1 == stakeholders2 && stakeholders1 == stakeholders2 &&
key1 === key2 => key1 === key2 =>

View File

@ -108,6 +108,22 @@ object TransactionCoder {
.build() .build()
) )
private def encodeContractInstance[Cid](
encodeCid: ValueCoder.EncodeCid[Cid],
version: TransactionVersion,
templateId: Ref.Identifier,
arg: Value[Cid],
agreementText: String,
) =
encodeVersionedValue(encodeCid, version, arg).map(
TransactionOuterClass.ContractInstance
.newBuilder()
.setTemplateId(ValueCoder.encodeIdentifier(templateId))
.setValue(_)
.setAgreement(agreementText)
.build()
)
/** Decode a contract instance from wire format /** Decode a contract instance from wire format
* @param protoCoinst protocol buffer encoded contract instance * @param protoCoinst protocol buffer encoded contract instance
* @param decodeCid cid decoding function * @param decodeCid cid decoding function
@ -182,13 +198,19 @@ object TransactionCoder {
nodeBuilder.setVersion(node.version.protoValue) nodeBuilder.setVersion(node.version.protoValue)
node match { node match {
case nc @ NodeCreate(_, _, _, _, _, _, _) => case nc @ NodeCreate(_, _, _, _, _, _, _, _, _) =>
val builder = TransactionOuterClass.NodeCreate.newBuilder() val builder = TransactionOuterClass.NodeCreate.newBuilder()
nc.stakeholders.foreach(builder.addStakeholders) nc.stakeholders.foreach(builder.addStakeholders)
nc.signatories.foreach(builder.addSignatories) nc.signatories.foreach(builder.addSignatories)
builder.setContractIdStruct(encodeCid.encode(nc.coid)) builder.setContractIdStruct(encodeCid.encode(nc.coid))
for { for {
inst <- encodeContractInstance(encodeCid, nc.versionedCoinst) inst <- encodeContractInstance(
encodeCid,
nc.version,
nc.templateId,
nc.arg,
nc.agreementText,
)
_ <- nc.key match { _ <- nc.key match {
case Some(key) => case Some(key) =>
encodeKeyWithMaintainers(encodeCid, nc.version, key) encodeKeyWithMaintainers(encodeCid, nc.version, key)
@ -335,7 +357,17 @@ object TransactionCoder {
else else
decodeKeyWithMaintainers(decodeCid, nodeVersion, protoCreate.getKeyWithMaintainers) decodeKeyWithMaintainers(decodeCid, nodeVersion, protoCreate.getKeyWithMaintainers)
.map(Some(_)) .map(Some(_))
} yield ni -> NodeCreate(c, ci, None, signatories, stakeholders, key, nodeVersion) } yield ni -> NodeCreate(
c,
ci.template,
ci.arg,
ci.agreementText,
None,
signatories,
stakeholders,
key,
nodeVersion,
)
case NodeTypeCase.FETCH => case NodeTypeCase.FETCH =>
val protoFetch = protoNode.getFetch val protoFetch = protoNode.getFetch
for { for {

View File

@ -5,10 +5,10 @@ package com.daml.lf
package transaction package transaction
import com.daml.lf.data.ImmArray import com.daml.lf.data.ImmArray
import com.daml.lf.data.Ref.{Identifier, PackageId, Party, QualifiedName} import com.daml.lf.data.Ref.{Identifier, Party}
import com.daml.lf.transaction.Node._ import com.daml.lf.transaction.Node._
import com.daml.lf.transaction.{TransactionOuterClass => proto} import com.daml.lf.transaction.{TransactionOuterClass => proto}
import com.daml.lf.value.Value.{ContractId, ContractInst, ValueParty} import com.daml.lf.value.Value.{ContractId, ValueParty}
import com.daml.lf.value.ValueCoder.{DecodeError, EncodeError} import com.daml.lf.value.ValueCoder.{DecodeError, EncodeError}
import com.daml.lf.value.ValueCoder import com.daml.lf.value.ValueCoder
import org.scalacheck.{Arbitrary, Gen} import org.scalacheck.{Arbitrary, Gen}
@ -249,14 +249,9 @@ class TransactionCoderSpec
val node = val node =
NodeCreate[ContractId]( NodeCreate[ContractId](
coid = absCid("#test-cid"), coid = absCid("#test-cid"),
coinst = ContractInst( templateId = Identifier.assertFromString("pkg-id:Test:Name"),
Identifier( arg = ValueParty(Party.assertFromString("francesco")),
PackageId.assertFromString("pkg-id"), agreementText = "agreement",
QualifiedName.assertFromString("Test:Name"),
),
ValueParty(Party.assertFromString("francesco")),
"agreement",
),
optLocation = None, optLocation = None,
signatories = Set(Party.assertFromString("alice")), signatories = Set(Party.assertFromString("alice")),
stakeholders = Set(Party.assertFromString("alice"), Party.assertFromString("bob")), stakeholders = Set(Party.assertFromString("alice"), Party.assertFromString("bob")),

View File

@ -221,7 +221,7 @@ class TransactionSpec extends AnyFreeSpec with Matchers with ScalaCheckDrivenPro
cid2 -> V.ContractId.V1.assertBuild(cid2.discriminator, suffix2), cid2 -> V.ContractId.V1.assertBuild(cid2.discriminator, suffix2),
) )
dummyCreateNode("dd").coinst.suffixCid(mapping1) dummyCreateNode("dd").arg.suffixCid(mapping1)
val tx1 = tx.suffixCid(mapping1) val tx1 = tx.suffixCid(mapping1)
val tx2 = tx.suffixCid(mapping1) val tx2 = tx.suffixCid(mapping1)
@ -275,14 +275,9 @@ object TransactionSpec {
def dummyCreateNode(cid: String): NodeCreate[V.ContractId] = def dummyCreateNode(cid: String): NodeCreate[V.ContractId] =
NodeCreate( NodeCreate(
coid = toCid(cid), coid = toCid(cid),
coinst = V.ContractInst( templateId = Ref.Identifier.assertFromString("-dummyPkg-:DummyModule:dummyName"),
Ref.Identifier( arg = V.ValueContractId(dummyCid),
Ref.PackageId.assertFromString("-dummyPkg-"), agreementText = "dummyAgreement",
Ref.QualifiedName.assertFromString("DummyModule:dummyName"),
),
V.ValueContractId(dummyCid),
"dummyAgreement",
),
optLocation = None, optLocation = None,
signatories = Set.empty, signatories = Set.empty,
stakeholders = Set.empty, stakeholders = Set.empty,

View File

@ -51,8 +51,9 @@ private[migration] class V5_1__Populate_Event_Data extends BaseJavaMigration {
val txs = loadTransactions(conn) val txs = loadTransactions(conn)
val data = txs.flatMap { case (txId, tx) => val data = txs.flatMap { case (txId, tx) =>
tx.nodes.collect { case (nodeId, NodeCreate(cid, _, _, signatories, stakeholders, _, _)) => tx.nodes.collect {
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories) case (nodeId, NodeCreate(cid, _, _, _, _, signatories, stakeholders, _, _)) =>
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
} }
} }

View File

@ -51,8 +51,9 @@ private[migration] class V10_1__Populate_Event_Data extends BaseJavaMigration {
val txs = loadTransactions(conn) val txs = loadTransactions(conn)
val data = txs.flatMap { case (txId, tx) => val data = txs.flatMap { case (txId, tx) =>
tx.nodes.collect { case (nodeId, NodeCreate(cid, _, _, signatories, stakeholders, _, _)) => tx.nodes.collect {
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories) case (nodeId, NodeCreate(cid, _, _, _, _, signatories, stakeholders, _, _)) =>
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
} }
} }

View File

@ -28,7 +28,9 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
builder.add( builder.add(
NodeCreate( NodeCreate(
coid = contractId, coid = contractId,
coinst = someContractInstance, templateId = someTemplateId,
arg = someValueRecord,
agreementText = someAgreement,
optLocation = None, optLocation = None,
signatories = Set(alice), signatories = Set(alice),
stakeholders = Set(alice), stakeholders = Set(alice),
@ -44,7 +46,9 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
builder.add( builder.add(
NodeCreate( NodeCreate(
coid = contractId, coid = contractId,
coinst = someContractInstance, someTemplateId,
someValueRecord,
someAgreement,
optLocation = None, optLocation = None,
signatories = Set(bob), signatories = Set(bob),
stakeholders = Set(bob), stakeholders = Set(bob),
@ -120,7 +124,9 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
builder.add( builder.add(
NodeCreate( NodeCreate(
coid = builder.newCid, coid = builder.newCid,
coinst = someContractInstance, someTemplateId,
someValueRecord,
someAgreement,
optLocation = None, optLocation = None,
signatories = Set(bob), signatories = Set(bob),
stakeholders = Set(alice, bob), stakeholders = Set(alice, bob),

View File

@ -129,7 +129,9 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
): NodeCreate[ContractId] = ): NodeCreate[ContractId] =
NodeCreate( NodeCreate(
coid = absCid, coid = absCid,
coinst = someContractInstance, templateId = someTemplateId,
arg = someValueRecord,
agreementText = someAgreement,
optLocation = None, optLocation = None,
signatories = signatories, signatories = signatories,
stakeholders = stakeholders, stakeholders = stakeholders,
@ -495,7 +497,7 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
contract.copy( contract.copy(
signatories = parties, signatories = parties,
stakeholders = parties, stakeholders = parties,
coinst = contract.coinst.copy(template = Identifier.assertFromString(template)), templateId = Identifier.assertFromString(template),
) )
) )
} yield nodeId -> parties } yield nodeId -> parties
@ -593,7 +595,9 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
val createNodeId = txBuilder.add( val createNodeId = txBuilder.add(
NodeCreate( NodeCreate(
coid = txBuilder.newCid, coid = txBuilder.newCid,
coinst = someContractInstance, templateId = someTemplateId,
arg = someValueRecord,
agreementText = someAgreement,
optLocation = None, optLocation = None,
signatories = Set(party), signatories = Set(party),
stakeholders = Set(party), stakeholders = Set(party),

View File

@ -9,7 +9,7 @@ import com.daml.lf.engine.Blinding
import com.daml.lf.transaction.Transaction.Transaction import com.daml.lf.transaction.Transaction.Transaction
import com.daml.lf.transaction.test.TransactionBuilder import com.daml.lf.transaction.test.TransactionBuilder
import com.daml.lf.transaction.{Node, TransactionVersion} import com.daml.lf.transaction.{Node, TransactionVersion}
import com.daml.lf.value.Value.{ContractId, ContractInst, ValueText} import com.daml.lf.value.Value.{ContractId, ValueText}
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec import org.scalatest.wordspec.AnyWordSpec
@ -18,14 +18,9 @@ class ProjectionsSpec extends AnyWordSpec with Matchers {
def makeCreateNode(cid: ContractId, signatories: Set[Party], stakeholders: Set[Party]) = def makeCreateNode(cid: ContractId, signatories: Set[Party], stakeholders: Set[Party]) =
Node.NodeCreate( Node.NodeCreate(
coid = cid, coid = cid,
coinst = ContractInst( templateId = Identifier.assertFromString("some-package:Foo:Bar"),
Identifier( arg = ValueText("foo"),
PackageId.assertFromString("some-package"), agreementText = "agreement",
QualifiedName.assertFromString("Foo:Bar"),
),
ValueText("foo"),
"agreement",
),
optLocation = None, optLocation = None,
signatories = signatories, signatories = signatories,
stakeholders = stakeholders, stakeholders = stakeholders,

View File

@ -28,7 +28,8 @@ private[benchmark] final class Adapter(
node match { node match {
case create: Node.NodeCreate[ContractId] => case create: Node.NodeCreate[ContractId] =>
create.copy( create.copy(
coinst = create.coinst.copy(adapt(create.coinst.template), adapt(create.coinst.arg)), templateId = adapt(create.templateId),
arg = adapt(create.arg),
optLocation = None, optLocation = None,
key = create.key.map(adapt), key = create.key.map(adapt),
) )
@ -48,11 +49,12 @@ private[benchmark] final class Adapter(
key = fetch.key.map(adapt), key = fetch.key.map(adapt),
) )
case lookup: Node.NodeLookupByKey[ContractId] => case lookup: Node.NodeLookupByKey[ContractId] =>
lookup.copy( lookup
templateId = adapt(lookup.templateId), .copy(
optLocation = None, templateId = adapt(lookup.templateId),
key = adapt(lookup.key), optLocation = None,
) key = adapt(lookup.key),
)
} }
// drop value version // drop value version