mirror of
https://github.com/digital-asset/daml.git
synced 2024-11-13 00:16:19 +03:00
Remove optLocation field from transaction nodes. (#10301)
* Remove Optional[Location] field from transaction nodes. xoptLocation working but very messy! cleanup PartialTransaction code pass locationInfo in CompleteTransaction dont pass locationInfo in SumittedTransaction pass optLocation to CheckAuthorization temp reinstate xoptLocation in ExerciseContextInfo cleanup Conversions.scala remove Optional[Location] field from transaction nodes. CHANGELOG_BEGIN CHANGELOG_END * fix bad merge * fix build after merge * rename * doc comment, and question
This commit is contained in:
parent
e62852fbfd
commit
0e09f3ba08
@ -439,7 +439,7 @@ final class Conversions(
|
||||
val tx = incomplete.transaction
|
||||
|
||||
val builder = proto.PartialTransaction.newBuilder
|
||||
.addAllNodes(tx.nodes.map(convertNode).asJava)
|
||||
.addAllNodes(tx.nodes.map(convertIncompleteTransactionNode(incomplete.locationInfo)).asJava)
|
||||
.addAllRoots(tx.roots.toList.map(convertTxNodeId).asJava)
|
||||
|
||||
builder.build
|
||||
@ -495,7 +495,7 @@ final class Conversions(
|
||||
.addAllSignatories(create.signatories.map(convertParty).asJava)
|
||||
.addAllStakeholders(create.stakeholders.map(convertParty).asJava)
|
||||
|
||||
create.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
nodeInfo.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
builder.setCreate(createBuilder.build)
|
||||
case fetch: N.NodeFetch[V.ContractId] =>
|
||||
builder.setFetch(
|
||||
@ -507,7 +507,7 @@ final class Conversions(
|
||||
.build
|
||||
)
|
||||
case ex: N.NodeExercises[NodeId, V.ContractId] =>
|
||||
ex.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
nodeInfo.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
val exerciseBuilder =
|
||||
proto.Node.Exercise.newBuilder
|
||||
.setTargetContractId(coidToEventId(ex.targetCoid).toLedgerString)
|
||||
@ -532,7 +532,7 @@ final class Conversions(
|
||||
builder.setExercise(exerciseBuilder.build)
|
||||
|
||||
case lbk: N.NodeLookupByKey[V.ContractId] =>
|
||||
lbk.optLocation.foreach(loc => builder.setLocation(convertLocation(loc)))
|
||||
nodeInfo.optLocation.foreach(loc => builder.setLocation(convertLocation(loc)))
|
||||
val lbkBuilder = proto.Node.LookupByKey.newBuilder
|
||||
.setTemplateId(convertIdentifier(lbk.templateId))
|
||||
.setKeyWithMaintainers(convertKeyWithMaintainers(lbk.versionedKey))
|
||||
@ -553,10 +553,11 @@ final class Conversions(
|
||||
.build()
|
||||
}
|
||||
|
||||
def convertNode(
|
||||
nodeWithId: (NodeId, N.GenNode[NodeId, V.ContractId])
|
||||
): proto.Node = {
|
||||
def convertIncompleteTransactionNode(
|
||||
locationInfo: Map[NodeId, Ref.Location]
|
||||
)(nodeWithId: (NodeId, N.GenNode[NodeId, V.ContractId])): proto.Node = {
|
||||
val (nodeId, node) = nodeWithId
|
||||
val optLocation = locationInfo.get(nodeId)
|
||||
val builder = proto.Node.newBuilder
|
||||
builder
|
||||
.setNodeId(proto.NodeId.newBuilder.setId(nodeId.index.toString).build)
|
||||
@ -586,7 +587,7 @@ final class Conversions(
|
||||
create.versionedKey.foreach(key =>
|
||||
createBuilder.setKeyWithMaintainers(convertKeyWithMaintainers(key))
|
||||
)
|
||||
create.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
builder.setCreate(createBuilder.build)
|
||||
case fetch: N.NodeFetch[V.ContractId] =>
|
||||
builder.setFetch(
|
||||
@ -598,7 +599,7 @@ final class Conversions(
|
||||
.build
|
||||
)
|
||||
case ex: N.NodeExercises[NodeId, V.ContractId] =>
|
||||
ex.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
builder.setExercise(
|
||||
proto.Node.Exercise.newBuilder
|
||||
.setTargetContractId(coidToEventId(ex.targetCoid).toLedgerString)
|
||||
@ -619,7 +620,7 @@ final class Conversions(
|
||||
)
|
||||
|
||||
case lookup: N.NodeLookupByKey[V.ContractId] =>
|
||||
lookup.optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
optLocation.map(loc => builder.setLocation(convertLocation(loc)))
|
||||
builder.setLookupByKey({
|
||||
val builder = proto.Node.LookupByKey.newBuilder
|
||||
.setKeyWithMaintainers(convertKeyWithMaintainers(lookup.versionedKey))
|
||||
|
@ -394,7 +394,7 @@ class Engine(val config: EngineConfig = new EngineConfig(LanguageVersion.StableV
|
||||
}
|
||||
|
||||
onLedger.finish match {
|
||||
case PartialTransaction.CompleteTransaction(tx, nodeSeeds) =>
|
||||
case PartialTransaction.CompleteTransaction(tx, _, nodeSeeds) =>
|
||||
val meta = Tx.Metadata(
|
||||
submissionSeed = None,
|
||||
submissionTime = onLedger.ptxInternal.submissionTime,
|
||||
|
@ -1408,7 +1408,6 @@ class EngineTest
|
||||
coid,
|
||||
_,
|
||||
choice,
|
||||
_,
|
||||
consuming,
|
||||
actingParties,
|
||||
_,
|
||||
@ -1504,7 +1503,7 @@ class EngineTest
|
||||
|
||||
def actFetchActors[Nid, Cid](n: Node.GenNode[Nid, Cid]): Set[Party] = {
|
||||
n match {
|
||||
case Node.NodeFetch(_, _, _, actingParties, _, _, _, _, _) => actingParties
|
||||
case Node.NodeFetch(_, _, actingParties, _, _, _, _, _) => actingParties
|
||||
case _ => Set()
|
||||
}
|
||||
}
|
||||
@ -1683,7 +1682,7 @@ class EngineTest
|
||||
def firstLookupNode[Nid, Cid](
|
||||
tx: GenTx[Nid, Cid]
|
||||
): Option[(Nid, Node.NodeLookupByKey[Cid])] =
|
||||
tx.nodes.collectFirst { case (nid, nl @ Node.NodeLookupByKey(_, _, _, _, _)) =>
|
||||
tx.nodes.collectFirst { case (nid, nl @ Node.NodeLookupByKey(_, _, _, _)) =>
|
||||
nid -> nl
|
||||
}
|
||||
|
||||
@ -1892,7 +1891,7 @@ class EngineTest
|
||||
)
|
||||
|
||||
tx.transaction.nodes.values.headOption match {
|
||||
case Some(Node.NodeFetch(_, _, _, _, _, _, key, _, _)) =>
|
||||
case Some(Node.NodeFetch(_, _, _, _, _, key, _, _)) =>
|
||||
key match {
|
||||
// just test that the maintainers match here, getting the key out is a bit hairier
|
||||
case Some(Node.KeyWithMaintainers(_, maintainers)) =>
|
||||
@ -2102,7 +2101,7 @@ class EngineTest
|
||||
"be partially reinterpretable" in {
|
||||
val Right((tx, txMeta)) = run(3)
|
||||
val ImmArray(_, exeNode1) = tx.transaction.roots
|
||||
val Node.NodeExercises(_, _, _, _, _, _, _, _, _, _, children, _, _, _, _) =
|
||||
val Node.NodeExercises(_, _, _, _, _, _, _, _, _, children, _, _, _, _) =
|
||||
tx.transaction.nodes(exeNode1)
|
||||
val nids = children.toSeq.take(2).toImmArray
|
||||
|
||||
@ -2792,7 +2791,6 @@ object EngineTest {
|
||||
targetCoid: ContractId,
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
consuming @ true,
|
||||
_,
|
||||
_,
|
||||
|
@ -49,6 +49,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
|
||||
effectiveAt = effectiveAt,
|
||||
optLocation = None,
|
||||
tx = tx,
|
||||
locationInfo = Map.empty,
|
||||
l = ledger,
|
||||
)
|
||||
.fold(
|
||||
|
@ -170,6 +170,7 @@ object ScenarioLedger {
|
||||
*/
|
||||
final case class LedgerNodeInfo(
|
||||
node: Node,
|
||||
optLocation: Option[Location],
|
||||
transaction: TransactionId,
|
||||
effectiveAt: Time.Timestamp,
|
||||
disclosures: Map[Party, Disclosure],
|
||||
@ -240,13 +241,14 @@ object ScenarioLedger {
|
||||
effectiveAt: Time.Timestamp,
|
||||
optLocation: Option[Location],
|
||||
tx: SubmittedTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
l: ScenarioLedger,
|
||||
): Either[CommitError, CommitResult] = {
|
||||
// transactionId is small enough (< 20 chars), so we do no exceed the 255
|
||||
// chars limit when concatenate in EventId#toLedgerString method.
|
||||
val transactionId = l.scenarioStepId.id
|
||||
val richTr = RichTransaction(actAs, readAs, effectiveAt, transactionId, tx)
|
||||
processTransaction(l.scenarioStepId, richTr, l.ledgerData) match {
|
||||
processTransaction(l.scenarioStepId, richTr, locationInfo, l.ledgerData) match {
|
||||
case Left(err) => Left(CommitError.UniqueKeyViolation(err))
|
||||
case Right(updatedCache) =>
|
||||
Right(
|
||||
@ -396,6 +398,7 @@ object ScenarioLedger {
|
||||
private def processTransaction(
|
||||
trId: TransactionId,
|
||||
richTr: RichTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
ledgerData: LedgerData,
|
||||
): Either[UniqueKeyViolation, LedgerData] = {
|
||||
|
||||
@ -444,6 +447,7 @@ object ScenarioLedger {
|
||||
case Some(node) =>
|
||||
val newLedgerNodeInfo = LedgerNodeInfo(
|
||||
node = node,
|
||||
optLocation = locationInfo.get(nodeId),
|
||||
transaction = trId,
|
||||
effectiveAt = richTr.effectiveAt,
|
||||
disclosures = Map.empty,
|
||||
@ -485,7 +489,7 @@ object ScenarioLedger {
|
||||
}
|
||||
processNodes(mbNewCache2, idsToProcess)
|
||||
|
||||
case NodeFetch(referencedCoid, templateId @ _, optLoc @ _, _, _, _, _, _, _) =>
|
||||
case NodeFetch(referencedCoid, templateId @ _, _, _, _, _, _, _) =>
|
||||
val newCacheP =
|
||||
newCache.updateLedgerNodeInfo(referencedCoid)(info =>
|
||||
info.copy(referencedBy = info.referencedBy + eventId)
|
||||
|
@ -4,6 +4,7 @@
|
||||
package com.daml.lf
|
||||
package speedy
|
||||
|
||||
import com.daml.lf.data.Ref.Location
|
||||
import com.daml.lf.ledger.Authorize
|
||||
import com.daml.lf.ledger.FailedAuthorization
|
||||
import com.daml.lf.transaction.Node.{NodeCreate, NodeFetch, NodeLookupByKey}
|
||||
@ -23,21 +24,24 @@ private[lf] object CheckAuthorization {
|
||||
List(failWith)
|
||||
}
|
||||
|
||||
private[lf] def authorizeCreate(create: NodeCreate[_])(
|
||||
private[lf] def authorizeCreate(
|
||||
optLocation: Option[Location],
|
||||
create: NodeCreate[_],
|
||||
)(
|
||||
auth: Authorize
|
||||
): List[FailedAuthorization] = {
|
||||
authorize(
|
||||
passIf = create.signatories subsetOf auth.authParties,
|
||||
failWith = FailedAuthorization.CreateMissingAuthorization(
|
||||
templateId = create.coinst.template,
|
||||
optLocation = create.optLocation,
|
||||
optLocation = optLocation,
|
||||
authorizingParties = auth.authParties,
|
||||
requiredParties = create.signatories,
|
||||
),
|
||||
) ++
|
||||
authorize(
|
||||
passIf = create.signatories.nonEmpty,
|
||||
failWith = FailedAuthorization.NoSignatories(create.coinst.template, create.optLocation),
|
||||
failWith = FailedAuthorization.NoSignatories(create.coinst.template, optLocation),
|
||||
) ++
|
||||
(create.key match {
|
||||
case None => List()
|
||||
@ -47,7 +51,7 @@ private[lf] object CheckAuthorization {
|
||||
passIf = maintainers subsetOf create.signatories,
|
||||
failWith = FailedAuthorization.MaintainersNotSubsetOfSignatories(
|
||||
templateId = create.coinst.template,
|
||||
optLocation = create.optLocation,
|
||||
optLocation = optLocation,
|
||||
signatories = create.signatories,
|
||||
maintainers = maintainers,
|
||||
),
|
||||
@ -55,47 +59,56 @@ private[lf] object CheckAuthorization {
|
||||
})
|
||||
}
|
||||
|
||||
private[lf] def authorizeFetch(fetch: NodeFetch[_])(
|
||||
private[lf] def authorizeFetch(
|
||||
optLocation: Option[Location],
|
||||
fetch: NodeFetch[_],
|
||||
)(
|
||||
auth: Authorize
|
||||
): List[FailedAuthorization] = {
|
||||
authorize(
|
||||
passIf = fetch.stakeholders.intersect(auth.authParties).nonEmpty,
|
||||
failWith = FailedAuthorization.FetchMissingAuthorization(
|
||||
templateId = fetch.templateId,
|
||||
optLocation = fetch.optLocation,
|
||||
optLocation = optLocation,
|
||||
stakeholders = fetch.stakeholders,
|
||||
authorizingParties = auth.authParties,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
private[lf] def authorizeLookupByKey(lbk: NodeLookupByKey[_])(
|
||||
private[lf] def authorizeLookupByKey(
|
||||
optLocation: Option[Location],
|
||||
lbk: NodeLookupByKey[_],
|
||||
)(
|
||||
auth: Authorize
|
||||
): List[FailedAuthorization] = {
|
||||
authorize(
|
||||
passIf = lbk.key.maintainers subsetOf auth.authParties,
|
||||
failWith = FailedAuthorization.LookupByKeyMissingAuthorization(
|
||||
lbk.templateId,
|
||||
lbk.optLocation,
|
||||
optLocation,
|
||||
lbk.key.maintainers,
|
||||
auth.authParties,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
private[lf] def authorizeExercise(ex: ExercisesContextInfo)(
|
||||
private[lf] def authorizeExercise(
|
||||
optLocation: Option[Location],
|
||||
ex: ExercisesContextInfo,
|
||||
)(
|
||||
auth: Authorize
|
||||
): List[FailedAuthorization] = {
|
||||
authorize(
|
||||
passIf = ex.actingParties.nonEmpty,
|
||||
failWith = FailedAuthorization.NoControllers(ex.templateId, ex.choiceId, ex.optLocation),
|
||||
failWith = FailedAuthorization.NoControllers(ex.templateId, ex.choiceId, optLocation),
|
||||
) ++
|
||||
authorize(
|
||||
passIf = ex.actingParties subsetOf auth.authParties,
|
||||
failWith = FailedAuthorization.ExerciseMissingAuthorization(
|
||||
templateId = ex.templateId,
|
||||
choiceId = ex.choiceId,
|
||||
optLocation = ex.optLocation,
|
||||
optLocation = optLocation,
|
||||
authorizingParties = auth.authParties,
|
||||
requiredParties = ex.actingParties,
|
||||
),
|
||||
|
@ -16,7 +16,6 @@ import com.daml.lf.transaction.{
|
||||
SubmittedTransaction,
|
||||
Transaction => Tx,
|
||||
TransactionVersion => TxVersion,
|
||||
IncompleteTransaction => TxIncompleteTransaction,
|
||||
}
|
||||
import com.daml.lf.value.Value
|
||||
|
||||
@ -40,8 +39,9 @@ private[lf] object PartialTransaction {
|
||||
private type ExerciseNode = Node.NodeExercises[NodeId, Value.ContractId]
|
||||
|
||||
private final case class IncompleteTxImpl(
|
||||
val transaction: TX
|
||||
) extends TxIncompleteTransaction
|
||||
val transaction: TX,
|
||||
val locationInfo: Map[NodeId, Location],
|
||||
) extends transaction.IncompleteTransaction
|
||||
|
||||
sealed abstract class ContextInfo {
|
||||
val actionChildSeed: Int => crypto.Hash
|
||||
@ -140,7 +140,6 @@ private[lf] object PartialTransaction {
|
||||
templateId: TypeConName,
|
||||
contractKey: Option[Node.KeyWithMaintainers[Value[Nothing]]],
|
||||
choiceId: ChoiceName,
|
||||
optLocation: Option[Location],
|
||||
consuming: Boolean,
|
||||
actingParties: Set[Party],
|
||||
chosenValue: Value[Value.ContractId],
|
||||
@ -192,12 +191,17 @@ private[lf] object PartialTransaction {
|
||||
keys = Map.empty,
|
||||
globalKeyInputs = Map.empty,
|
||||
localContracts = Set.empty,
|
||||
actionNodeLocations = BackStack.empty,
|
||||
)
|
||||
|
||||
type NodeSeeds = ImmArray[(NodeId, crypto.Hash)]
|
||||
|
||||
sealed abstract class Result extends Product with Serializable
|
||||
final case class CompleteTransaction(tx: SubmittedTransaction, seeds: NodeSeeds) extends Result
|
||||
final case class CompleteTransaction(
|
||||
tx: SubmittedTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
seeds: NodeSeeds,
|
||||
) extends Result
|
||||
final case class IncompleteTransaction(ptx: PartialTransaction) extends Result
|
||||
|
||||
sealed abstract class KeyMapping extends Product with Serializable
|
||||
@ -252,6 +256,9 @@ private[lf] object PartialTransaction {
|
||||
* same key might have been created since. This is updated on creates with keys with KeyInactive
|
||||
* (implying that no key must have been active at the beginning of the transaction)
|
||||
* and on failing and successful lookup and fetch by key.
|
||||
*
|
||||
* @param actionNodeLocations The optional locations of create/exercise/fetch/lookup nodes in pre-order.
|
||||
* Used by 'locationInfo()', called by 'finish()' and 'finishIncomplete()'
|
||||
*/
|
||||
private[lf] case class PartialTransaction(
|
||||
packageToTransactionVersion: Ref.PackageId => TxVersion,
|
||||
@ -266,6 +273,7 @@ private[lf] case class PartialTransaction(
|
||||
keys: Map[GlobalKey, PartialTransaction.KeyMapping],
|
||||
globalKeyInputs: Map[GlobalKey, PartialTransaction.KeyMapping],
|
||||
localContracts: Set[Value.ContractId],
|
||||
actionNodeLocations: BackStack[Option[Location]],
|
||||
) {
|
||||
|
||||
import PartialTransaction._
|
||||
@ -321,6 +329,12 @@ private[lf] case class PartialTransaction(
|
||||
sb.toString
|
||||
}
|
||||
|
||||
private def locationInfo(): Map[NodeId, Location] = {
|
||||
this.actionNodeLocations.toImmArray.toSeq.zipWithIndex.collect { case (Some(loc), n) =>
|
||||
(NodeId(n), loc)
|
||||
}.toMap
|
||||
}
|
||||
|
||||
/** Finish building a transaction; i.e., try to extract a complete
|
||||
* transaction from the given 'PartialTransaction'. This returns:
|
||||
* - a SubmittedTransaction in case of success ;
|
||||
@ -339,6 +353,7 @@ private[lf] case class PartialTransaction(
|
||||
SubmittedTransaction(
|
||||
TxVersion.asVersionedTransaction(tx)
|
||||
),
|
||||
locationInfo(),
|
||||
seeds.zip(actionNodeSeeds.toImmArray),
|
||||
)
|
||||
case _ =>
|
||||
@ -346,10 +361,9 @@ private[lf] case class PartialTransaction(
|
||||
}
|
||||
|
||||
// construct an IncompleteTransaction from the partial-transaction
|
||||
def finishIncomplete: TxIncompleteTransaction = {
|
||||
|
||||
def finishIncomplete: transaction.IncompleteTransaction = {
|
||||
@tailrec
|
||||
def unwindToExercise(
|
||||
def unwindToExercise( //TODO: remove as never called
|
||||
contextInfo: PartialTransaction.ContextInfo
|
||||
): Option[PartialTransaction.ExercisesContextInfo] = contextInfo match {
|
||||
case ctx: PartialTransaction.ExercisesContextInfo => Some(ctx)
|
||||
@ -358,13 +372,14 @@ private[lf] case class PartialTransaction(
|
||||
case _: PartialTransaction.RootContextInfo => None
|
||||
}
|
||||
|
||||
val ptx = unwind
|
||||
val ptx = unwind()
|
||||
|
||||
IncompleteTxImpl(
|
||||
GenTransaction(
|
||||
ptx.nodes,
|
||||
ImmArray(ptx.context.children.toImmArray.toSeq.sortBy(_.index)),
|
||||
)
|
||||
),
|
||||
ptx.locationInfo(),
|
||||
)
|
||||
}
|
||||
|
||||
@ -398,7 +413,6 @@ private[lf] case class PartialTransaction(
|
||||
templateId,
|
||||
arg,
|
||||
agreementText,
|
||||
optLocation,
|
||||
signatories,
|
||||
stakeholders,
|
||||
key,
|
||||
@ -406,12 +420,13 @@ private[lf] case class PartialTransaction(
|
||||
)
|
||||
val nid = NodeId(nextNodeIdx)
|
||||
val ptx = copy(
|
||||
actionNodeLocations = actionNodeLocations :+ optLocation,
|
||||
nextNodeIdx = nextNodeIdx + 1,
|
||||
context = context.addActionChild(nid, version),
|
||||
nodes = nodes.updated(nid, createNode),
|
||||
actionNodeSeeds = actionNodeSeeds :+ actionNodeSeed,
|
||||
localContracts = localContracts + cid,
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeCreate(createNode), auth)
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeCreate(optLocation, createNode), auth)
|
||||
|
||||
// if we have a contract key being added, include it in the list of
|
||||
// active keys
|
||||
@ -492,7 +507,6 @@ private[lf] case class PartialTransaction(
|
||||
val node = Node.NodeFetch(
|
||||
coid,
|
||||
templateId,
|
||||
optLocation,
|
||||
actingParties,
|
||||
signatories,
|
||||
stakeholders,
|
||||
@ -503,8 +517,8 @@ private[lf] case class PartialTransaction(
|
||||
mustBeActive(
|
||||
coid,
|
||||
templateId,
|
||||
insertLeafNode(node, version),
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeFetch(node), auth)
|
||||
insertLeafNode(node, version, optLocation),
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeFetch(optLocation, node), auth)
|
||||
}
|
||||
|
||||
def insertLookup(
|
||||
@ -518,13 +532,12 @@ private[lf] case class PartialTransaction(
|
||||
val version = packageToTransactionVersion(templateId.packageId)
|
||||
val node = Node.NodeLookupByKey(
|
||||
templateId,
|
||||
optLocation,
|
||||
key,
|
||||
result,
|
||||
version,
|
||||
)
|
||||
insertLeafNode(node, version)
|
||||
.noteAuthFails(nid, CheckAuthorization.authorizeLookupByKey(node), auth)
|
||||
insertLeafNode(node, version, optLocation)
|
||||
.noteAuthFails(nid, CheckAuthorization.authorizeLookupByKey(optLocation, node), auth)
|
||||
}
|
||||
|
||||
/** Open an exercises context.
|
||||
@ -559,7 +572,6 @@ private[lf] case class PartialTransaction(
|
||||
templateId = templateId,
|
||||
contractKey = mbKey,
|
||||
choiceId = choiceId,
|
||||
optLocation = optLocation,
|
||||
consuming = consuming,
|
||||
actingParties = actingParties,
|
||||
chosenValue = chosenValue,
|
||||
@ -576,6 +588,7 @@ private[lf] case class PartialTransaction(
|
||||
targetId,
|
||||
templateId,
|
||||
copy(
|
||||
actionNodeLocations = actionNodeLocations :+ optLocation,
|
||||
nextNodeIdx = nextNodeIdx + 1,
|
||||
context = Context(ec),
|
||||
actionNodeSeeds = actionNodeSeeds :+ ec.actionNodeSeed, // must push before children
|
||||
@ -597,7 +610,7 @@ private[lf] case class PartialTransaction(
|
||||
case _ => keys
|
||||
},
|
||||
),
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeExercise(ec), auth)
|
||||
).noteAuthFails(nid, CheckAuthorization.authorizeExercise(optLocation, ec), auth)
|
||||
)
|
||||
}
|
||||
}
|
||||
@ -632,7 +645,8 @@ private[lf] case class PartialTransaction(
|
||||
context =
|
||||
ec.parent.addActionChild(nodeId, exerciseNode.version min context.minChildVersion),
|
||||
nodes = nodes.updated(nodeId, exerciseNode),
|
||||
actionNodeSeeds = actionNodeSeeds :+ actionNodeSeed,
|
||||
actionNodeSeeds =
|
||||
actionNodeSeeds :+ actionNodeSeed, //(NC) pushed by 'beginExercises'; why push again?
|
||||
)
|
||||
case _ => throw new RuntimeException("abortExercises called in non-exercise context")
|
||||
}
|
||||
@ -643,7 +657,6 @@ private[lf] case class PartialTransaction(
|
||||
targetCoid = ec.targetId,
|
||||
templateId = ec.templateId,
|
||||
choiceId = ec.choiceId,
|
||||
optLocation = ec.optLocation,
|
||||
consuming = ec.consuming,
|
||||
actingParties = ec.actingParties,
|
||||
chosenValue = ec.chosenValue,
|
||||
@ -751,10 +764,15 @@ private[lf] case class PartialTransaction(
|
||||
}
|
||||
|
||||
/** Insert the given `LeafNode` under a fresh node-id, and return it */
|
||||
def insertLeafNode(node: LeafNode, version: TxVersion): PartialTransaction = {
|
||||
def insertLeafNode(
|
||||
node: LeafNode,
|
||||
version: TxVersion,
|
||||
optLocation: Option[Location],
|
||||
): PartialTransaction = {
|
||||
val _ = version
|
||||
val nid = NodeId(nextNodeIdx)
|
||||
copy(
|
||||
actionNodeLocations = actionNodeLocations :+ optLocation,
|
||||
nextNodeIdx = nextNodeIdx + 1,
|
||||
context = context.addActionChild(nid, version),
|
||||
nodes = nodes.updated(nid, node),
|
||||
@ -762,7 +780,7 @@ private[lf] case class PartialTransaction(
|
||||
}
|
||||
|
||||
/** Unwind the transaction aborting all incomplete nodes */
|
||||
def unwind: PartialTransaction = {
|
||||
def unwind(): PartialTransaction = {
|
||||
@tailrec
|
||||
def go(ptx: PartialTransaction): PartialTransaction = ptx.context.info match {
|
||||
case _: PartialTransaction.ExercisesContextInfo => go(ptx.abortExercises)
|
||||
|
@ -301,7 +301,6 @@ object NormalizeRollbackSpec {
|
||||
templateId = Ref.Identifier.assertFromString("-dummyPkg-:DummyModule:dummyName"),
|
||||
arg = V.ValueInt64(n),
|
||||
agreementText = "dummyAgreement",
|
||||
optLocation = None,
|
||||
signatories = Set.empty,
|
||||
stakeholders = Set.empty,
|
||||
key = None,
|
||||
@ -318,7 +317,6 @@ object NormalizeRollbackSpec {
|
||||
Ref.QualifiedName.assertFromString("DummyModule:dummyName"),
|
||||
),
|
||||
choiceId = Ref.Name.assertFromString("dummyChoice"),
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set.empty,
|
||||
chosenValue = V.ValueUnit,
|
||||
|
@ -30,7 +30,7 @@ class PartialTransactionSpec extends AnyWordSpec with Matchers with Inside {
|
||||
)
|
||||
|
||||
private[this] def contractIdsInOrder(ptx: PartialTransaction): Seq[Value.ContractId] = {
|
||||
inside(ptx.finish) { case CompleteTransaction(tx, _) =>
|
||||
inside(ptx.finish) { case CompleteTransaction(tx, _, _) =>
|
||||
tx.fold(Vector.empty[Value.ContractId]) {
|
||||
case (acc, (_, create: Node.NodeCreate[Value.ContractId])) => acc :+ create.coid
|
||||
case (acc, _) => acc
|
||||
|
@ -79,7 +79,7 @@ class ProfilerTest extends AnyWordSpec with Matchers with ScalaCheckDrivenProper
|
||||
onLedger.ptx.finish match {
|
||||
case IncompleteTransaction(_) =>
|
||||
sys.error("unexpected IncompleteTransaction")
|
||||
case CompleteTransaction(_, _) =>
|
||||
case CompleteTransaction(_, _, _) =>
|
||||
machine.profile.events.asScala.toList.map(ev => (ev.open, ev.rawLabel))
|
||||
}
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ class ExceptionTest extends AnyWordSpec with Matchers with TableDrivenPropertyCh
|
||||
onLedger.ptx.finish match {
|
||||
case IncompleteTransaction(_) =>
|
||||
sys.error("unexpected IncompleteTransaction")
|
||||
case CompleteTransaction(tx, _) =>
|
||||
case CompleteTransaction(tx, _, _) =>
|
||||
tx
|
||||
}
|
||||
}
|
||||
|
@ -8,7 +8,7 @@ import com.daml.lf.data.Ref._
|
||||
import com.daml.lf.data.{ImmArray, Ref, Time}
|
||||
import com.daml.lf.engine.Engine
|
||||
import com.daml.lf.language.Ast
|
||||
import com.daml.lf.transaction.{GlobalKey, SubmittedTransaction}
|
||||
import com.daml.lf.transaction.{NodeId, GlobalKey, SubmittedTransaction}
|
||||
import com.daml.lf.value.Value.{ContractId, ContractInst}
|
||||
import com.daml.lf.speedy._
|
||||
import com.daml.lf.speedy.SResult._
|
||||
@ -101,7 +101,7 @@ final case class ScenarioRunner(
|
||||
currentSubmission = None
|
||||
// TODO (MK) This is gross, we need to unwind the transaction to
|
||||
// get the right root context to derived the seed for the next transaction.
|
||||
val rootCtx = err.ptx.unwind.context
|
||||
val rootCtx = err.ptx.unwind().context
|
||||
seed = nextSeed(
|
||||
rootCtx.nextActionChildSeed
|
||||
)
|
||||
@ -238,6 +238,7 @@ object ScenarioRunner {
|
||||
readAs: Set[Party],
|
||||
location: Option[Location],
|
||||
tx: SubmittedTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
): Either[Error, R]
|
||||
}
|
||||
|
||||
@ -364,6 +365,7 @@ object ScenarioRunner {
|
||||
readAs: Set[Party],
|
||||
location: Option[Location],
|
||||
tx: SubmittedTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
): Either[Error, ScenarioLedger.CommitResult] =
|
||||
ScenarioLedger.commitTransaction(
|
||||
actAs = committers,
|
||||
@ -371,6 +373,7 @@ object ScenarioRunner {
|
||||
effectiveAt = ledger.currentTime,
|
||||
optLocation = location,
|
||||
tx = tx,
|
||||
locationInfo = locationInfo,
|
||||
l = ledger,
|
||||
) match {
|
||||
case Left(fas) =>
|
||||
@ -408,8 +411,8 @@ object ScenarioRunner {
|
||||
ledgerMachine.run() match {
|
||||
case SResult.SResultFinalValue(resultValue) =>
|
||||
onLedger.ptxInternal.finish match {
|
||||
case PartialTransaction.CompleteTransaction(tx, _) =>
|
||||
ledger.commit(committers, readAs, location, tx) match {
|
||||
case PartialTransaction.CompleteTransaction(tx, locationInfo, _) =>
|
||||
ledger.commit(committers, readAs, location, tx, locationInfo) match {
|
||||
case Left(err) =>
|
||||
SubmissionError(err, onLedger.ptxInternal)
|
||||
case Right(r) =>
|
||||
|
@ -11,7 +11,7 @@ import com.daml.lf.data.Ref.{Identifier, Location, Party, QualifiedName}
|
||||
import com.daml.lf.data.Time
|
||||
import com.daml.lf.language.Ast.EVal
|
||||
import com.daml.lf.speedy.SResult._
|
||||
import com.daml.lf.transaction.{GlobalKey, SubmittedTransaction}
|
||||
import com.daml.lf.transaction.{NodeId, GlobalKey, SubmittedTransaction}
|
||||
import com.daml.lf.value.Value
|
||||
import com.daml.lf.value.Value.{ContractId, ContractInst}
|
||||
import com.daml.lf.scenario.{ScenarioLedger, ScenarioRunner}
|
||||
@ -204,5 +204,6 @@ class CannedLedgerApi(
|
||||
readAs: Set[Party],
|
||||
location: Option[Location],
|
||||
tx: SubmittedTransaction,
|
||||
locationInfo: Map[NodeId, Location],
|
||||
) = Right(())
|
||||
}
|
||||
|
@ -132,7 +132,6 @@ final class TransactionBuilder(
|
||||
templateId = templateId,
|
||||
arg = argument,
|
||||
agreementText = "",
|
||||
optLocation = None,
|
||||
signatories = signatories.map(Ref.Party.assertFromString).toSet,
|
||||
stakeholders = signatories.toSet.union(observers.toSet).map(Ref.Party.assertFromString),
|
||||
key = key.map(keyWithMaintainers(maintainers = maintainers, _)),
|
||||
@ -155,7 +154,6 @@ final class TransactionBuilder(
|
||||
targetCoid = contract.coid,
|
||||
templateId = contract.coinst.template,
|
||||
choiceId = Ref.ChoiceName.assertFromString(choice),
|
||||
optLocation = None,
|
||||
consuming = consuming,
|
||||
actingParties = actingParties.map(Ref.Party.assertFromString),
|
||||
chosenValue = argument,
|
||||
@ -181,7 +179,6 @@ final class TransactionBuilder(
|
||||
Fetch(
|
||||
coid = contract.coid,
|
||||
templateId = contract.coinst.template,
|
||||
optLocation = None,
|
||||
actingParties = contract.signatories.map(Ref.Party.assertFromString),
|
||||
signatories = contract.signatories,
|
||||
stakeholders = contract.stakeholders,
|
||||
@ -196,7 +193,6 @@ final class TransactionBuilder(
|
||||
def lookupByKey(contract: Create, found: Boolean): LookupByKey =
|
||||
LookupByKey(
|
||||
templateId = contract.coinst.template,
|
||||
optLocation = None,
|
||||
key = contract.key.get,
|
||||
result = if (found) Some(contract.coid) else None,
|
||||
version = pkgTxVersion(contract.coinst.template.packageId),
|
||||
|
@ -322,7 +322,6 @@ object ValueGenerators {
|
||||
templateId,
|
||||
arg,
|
||||
agreement,
|
||||
None,
|
||||
signatories,
|
||||
stakeholders,
|
||||
key,
|
||||
@ -348,7 +347,6 @@ object ValueGenerators {
|
||||
} yield NodeFetch(
|
||||
coid,
|
||||
templateId,
|
||||
None,
|
||||
actingParties,
|
||||
signatories,
|
||||
stakeholders,
|
||||
@ -400,7 +398,6 @@ object ValueGenerators {
|
||||
targetCoid,
|
||||
templateId,
|
||||
choiceId,
|
||||
None,
|
||||
consume,
|
||||
actingParties,
|
||||
chosenValue,
|
||||
@ -424,7 +421,6 @@ object ValueGenerators {
|
||||
result <- Gen.option(targetCoid)
|
||||
} yield NodeLookupByKey(
|
||||
templateId,
|
||||
None,
|
||||
key,
|
||||
result,
|
||||
version,
|
||||
|
@ -4,6 +4,7 @@
|
||||
package com.daml.lf
|
||||
package transaction
|
||||
|
||||
import com.daml.lf.data.Ref.Location
|
||||
import com.daml.lf.value.Value
|
||||
|
||||
trait IncompleteTransaction {
|
||||
@ -14,4 +15,6 @@ trait IncompleteTransaction {
|
||||
type ExerciseNode = Node.NodeExercises[Nid, Cid]
|
||||
|
||||
def transaction: TX
|
||||
|
||||
def locationInfo: Map[Nid, Location]
|
||||
}
|
||||
|
@ -95,7 +95,6 @@ object Node {
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
key,
|
||||
_,
|
||||
) =>
|
||||
@ -110,7 +109,6 @@ object Node {
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
key,
|
||||
_,
|
||||
_,
|
||||
@ -125,7 +123,6 @@ object Node {
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
_,
|
||||
chosenValue,
|
||||
_,
|
||||
_,
|
||||
@ -144,7 +141,6 @@ object Node {
|
||||
key = key.map(KeyWithMaintainers.map1(Value.map1(f2))),
|
||||
)
|
||||
case self @ NodeLookupByKey(
|
||||
_,
|
||||
_,
|
||||
key,
|
||||
result,
|
||||
@ -165,7 +161,6 @@ object Node {
|
||||
templateI @ _,
|
||||
arg,
|
||||
agreementText @ _,
|
||||
optLocation @ _,
|
||||
signatories @ _,
|
||||
stakeholders @ _,
|
||||
key,
|
||||
@ -177,7 +172,6 @@ object Node {
|
||||
case NodeFetch(
|
||||
coid,
|
||||
templateId @ _,
|
||||
optLocationd @ _,
|
||||
actingPartiesd @ _,
|
||||
signatoriesd @ _,
|
||||
stakeholdersd @ _,
|
||||
@ -191,7 +185,6 @@ object Node {
|
||||
targetCoid,
|
||||
templateId @ _,
|
||||
choiceId @ _,
|
||||
optLocation @ _,
|
||||
consuming @ _,
|
||||
actingParties @ _,
|
||||
chosenValue,
|
||||
@ -211,7 +204,6 @@ object Node {
|
||||
children.foreach(f1)
|
||||
case NodeLookupByKey(
|
||||
templateId @ _,
|
||||
optLocation @ _,
|
||||
key,
|
||||
result,
|
||||
_,
|
||||
@ -230,7 +222,6 @@ object Node {
|
||||
override val templateId: TypeConName,
|
||||
arg: Value[Cid],
|
||||
agreementText: String,
|
||||
optLocation: Option[Location], // Optional location of the create expression
|
||||
signatories: Set[Party],
|
||||
stakeholders: Set[Party],
|
||||
key: Option[KeyWithMaintainers[Value[Cid]]],
|
||||
@ -258,7 +249,6 @@ object Node {
|
||||
final case class NodeFetch[+Cid](
|
||||
coid: Cid,
|
||||
override val templateId: TypeConName,
|
||||
optLocation: Option[Location], // Optional location of the fetch expression
|
||||
actingParties: Set[Party],
|
||||
signatories: Set[Party],
|
||||
stakeholders: Set[Party],
|
||||
@ -285,7 +275,6 @@ object Node {
|
||||
targetCoid: Cid,
|
||||
override val templateId: TypeConName,
|
||||
choiceId: ChoiceName,
|
||||
optLocation: Option[Location], // Optional location of the exercise expression
|
||||
consuming: Boolean,
|
||||
actingParties: Set[Party],
|
||||
chosenValue: Value[Cid],
|
||||
@ -318,7 +307,6 @@ object Node {
|
||||
|
||||
final case class NodeLookupByKey[+Cid](
|
||||
override val templateId: TypeConName,
|
||||
optLocation: Option[Location],
|
||||
key: KeyWithMaintainers[Value[Cid]],
|
||||
result: Option[Cid],
|
||||
// For the sake of consistency between types with a version field, keep this field the last.
|
||||
|
@ -454,11 +454,11 @@ sealed abstract class HasTxNodes[Nid, +Cid] {
|
||||
*/
|
||||
final def inputContracts[Cid2 >: Cid]: Set[Cid2] =
|
||||
fold(Set.empty[Cid2]) {
|
||||
case (acc, (_, Node.NodeExercises(coid, _, _, _, _, _, _, _, _, _, _, _, _, _, _))) =>
|
||||
case (acc, (_, Node.NodeExercises(coid, _, _, _, _, _, _, _, _, _, _, _, _, _))) =>
|
||||
acc + coid
|
||||
case (acc, (_, Node.NodeFetch(coid, _, _, _, _, _, _, _, _))) =>
|
||||
case (acc, (_, Node.NodeFetch(coid, _, _, _, _, _, _, _))) =>
|
||||
acc + coid
|
||||
case (acc, (_, Node.NodeLookupByKey(_, _, _, Some(coid), _))) =>
|
||||
case (acc, (_, Node.NodeLookupByKey(_, _, Some(coid), _))) =>
|
||||
acc + coid
|
||||
case (acc, _) => acc
|
||||
} -- localContracts.keySet
|
||||
@ -810,15 +810,15 @@ object GenTransaction extends value.CidContainer2[GenTransaction] {
|
||||
|
||||
tx.fold(State(Set.empty, Set.empty)) { case (state, (_, node)) =>
|
||||
node match {
|
||||
case Node.NodeCreate(_, tmplId, _, _, _, _, _, Some(key), _) =>
|
||||
case Node.NodeCreate(_, tmplId, _, _, _, _, Some(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))
|
||||
case Node.NodeExercises(_, tmplId, _, _, false, _, _, _, _, _, _, _, Some(key), _, _) =>
|
||||
case Node.NodeExercises(_, tmplId, _, false, _, _, _, _, _, _, _, Some(key), _, _) =>
|
||||
state.referenced(globalKey(tmplId, key.key))
|
||||
case Node.NodeFetch(_, tmplId, _, _, _, _, Some(key), _, _) =>
|
||||
case Node.NodeFetch(_, tmplId, _, _, _, Some(key), _, _) =>
|
||||
state.referenced(globalKey(tmplId, key.key))
|
||||
case Node.NodeLookupByKey(tmplId, _, key, Some(_), _) =>
|
||||
case Node.NodeLookupByKey(tmplId, key, Some(_), _) =>
|
||||
state.referenced(globalKey(tmplId, key.key))
|
||||
case _ =>
|
||||
state
|
||||
|
@ -279,7 +279,7 @@ object TransactionCoder {
|
||||
|
||||
node match {
|
||||
|
||||
case nc @ NodeCreate(_, _, _, _, _, _, _, _, _) =>
|
||||
case nc @ NodeCreate(_, _, _, _, _, _, _, _) =>
|
||||
val builder = TransactionOuterClass.NodeCreate.newBuilder()
|
||||
nc.stakeholders.foreach(builder.addStakeholders)
|
||||
nc.signatories.foreach(builder.addSignatories)
|
||||
@ -310,7 +310,7 @@ object TransactionCoder {
|
||||
)
|
||||
} yield nodeBuilder.setCreate(builder).build()
|
||||
|
||||
case nf @ NodeFetch(_, _, _, _, _, _, _, _, _) =>
|
||||
case nf @ NodeFetch(_, _, _, _, _, _, _, _) =>
|
||||
val builder = TransactionOuterClass.NodeFetch.newBuilder()
|
||||
builder.setTemplateId(ValueCoder.encodeIdentifier(nf.templateId))
|
||||
nf.stakeholders.foreach(builder.addStakeholders)
|
||||
@ -329,7 +329,7 @@ object TransactionCoder {
|
||||
)
|
||||
} yield nodeBuilder.setFetch(builder).build()
|
||||
|
||||
case ne @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _, _, _, _, _) =>
|
||||
case ne @ NodeExercises(_, _, _, _, _, _, _, _, _, _, _, _, _, _) =>
|
||||
val builder = TransactionOuterClass.NodeExercise.newBuilder()
|
||||
builder.setContractIdStruct(encodeCid.encode(ne.targetCoid))
|
||||
builder.setChoice(ne.choiceId)
|
||||
@ -381,7 +381,7 @@ object TransactionCoder {
|
||||
)
|
||||
} yield nodeBuilder.setExercise(builder).build()
|
||||
|
||||
case nlbk @ NodeLookupByKey(_, _, _, _, _) =>
|
||||
case nlbk @ NodeLookupByKey(_, _, _, _) =>
|
||||
val builder = TransactionOuterClass.NodeLookupByKey.newBuilder()
|
||||
builder.setTemplateId(ValueCoder.encodeIdentifier(nlbk.templateId))
|
||||
nlbk.result.foreach(cid => builder.setContractIdStruct(encodeCid.encode(cid)))
|
||||
@ -516,7 +516,6 @@ object TransactionCoder {
|
||||
ci.template,
|
||||
ci.arg,
|
||||
ci.agreementText,
|
||||
None,
|
||||
signatories,
|
||||
stakeholders,
|
||||
key,
|
||||
@ -543,7 +542,6 @@ object TransactionCoder {
|
||||
} yield ni -> NodeFetch(
|
||||
coid = c,
|
||||
templateId = templateId,
|
||||
optLocation = None,
|
||||
actingParties = actingParties,
|
||||
signatories = signatories,
|
||||
stakeholders = stakeholders,
|
||||
@ -602,7 +600,6 @@ object TransactionCoder {
|
||||
targetCoid = targetCoid,
|
||||
templateId = templateId,
|
||||
choiceId = choiceName,
|
||||
optLocation = None,
|
||||
consuming = protoExe.getConsuming,
|
||||
actingParties = actingParties,
|
||||
chosenValue = cv,
|
||||
@ -623,7 +620,7 @@ object TransactionCoder {
|
||||
key <-
|
||||
decodeKeyWithMaintainers(decodeCid, nodeVersion, protoLookupByKey.getKeyWithMaintainers)
|
||||
cid <- decodeCid.decodeOptional(protoLookupByKey.getContractIdStruct)
|
||||
} yield ni -> NodeLookupByKey[Cid](templateId, None, key, cid, nodeVersion)
|
||||
} yield ni -> NodeLookupByKey[Cid](templateId, key, cid, nodeVersion)
|
||||
case NodeTypeCase.NODETYPE_NOT_SET => Left(DecodeError("Unset Node type"))
|
||||
}
|
||||
}
|
||||
@ -783,7 +780,6 @@ object TransactionCoder {
|
||||
case (Right(acc), s) =>
|
||||
decodeVersionedNode(decodeNid, decodeCid, txVersion, s).map(acc + _)
|
||||
}
|
||||
|
||||
for {
|
||||
rs <- roots
|
||||
ns <- nodes
|
||||
|
@ -193,7 +193,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
templateId1,
|
||||
arg1,
|
||||
agreementText1,
|
||||
optLocation1 @ _,
|
||||
signatories1,
|
||||
stakeholders1,
|
||||
key1,
|
||||
@ -204,7 +203,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
templateId2,
|
||||
arg2,
|
||||
agreementText2,
|
||||
optLocation2 @ _,
|
||||
signatories2,
|
||||
stakeholders2,
|
||||
key2,
|
||||
@ -224,7 +222,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
Node.NodeFetch(
|
||||
coid1,
|
||||
templateId1,
|
||||
optLocation1 @ _,
|
||||
actingParties1,
|
||||
signatories1,
|
||||
stakeholders1,
|
||||
@ -235,7 +232,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
Node.NodeFetch(
|
||||
coid2,
|
||||
templateId2,
|
||||
optLocation2 @ _,
|
||||
actingParties2,
|
||||
signatories2,
|
||||
stakeholders2,
|
||||
@ -258,7 +254,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
targetCoid1,
|
||||
templateId1,
|
||||
choiceId1,
|
||||
optLocation1 @ _,
|
||||
consuming1,
|
||||
actingParties1,
|
||||
chosenValue1,
|
||||
@ -275,7 +270,6 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
targetCoid2,
|
||||
templateId2,
|
||||
choiceId2,
|
||||
optLocation2 @ _,
|
||||
consuming2,
|
||||
actingParties2,
|
||||
chosenValue2,
|
||||
@ -308,8 +302,8 @@ private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) {
|
||||
ExerciseEntry(Exercise(nid1, exe1, rest1), Exercise(nid2, exe2, rest2)) :: stack,
|
||||
)
|
||||
case (
|
||||
Node.NodeLookupByKey(templateId1, optLocation1 @ _, key1, result1, version1),
|
||||
Node.NodeLookupByKey(templateId2, optLocation2 @ _, key2, result2, version2),
|
||||
Node.NodeLookupByKey(templateId1, key1, result1, version1),
|
||||
Node.NodeLookupByKey(templateId2, key2, result2, version2),
|
||||
)
|
||||
if version1 == version2 &&
|
||||
templateId1 == templateId2 &&
|
||||
|
@ -228,7 +228,6 @@ class TransactionCoderSpec
|
||||
templateId = Identifier.assertFromString("pkg-id:Test:Name"),
|
||||
arg = ValueParty(Party.assertFromString("francesco")),
|
||||
agreementText = "agreement",
|
||||
optLocation = None,
|
||||
signatories = Set(Party.assertFromString("alice")),
|
||||
stakeholders = Set(Party.assertFromString("alice"), Party.assertFromString("bob")),
|
||||
key = None,
|
||||
|
@ -224,22 +224,6 @@ class TransactionSpec
|
||||
na should not be nb
|
||||
}
|
||||
}
|
||||
|
||||
"ignores location" in forAll(genEmptyNode) { n =>
|
||||
val withoutLocation = {
|
||||
val nodeWithoutLocation = n match {
|
||||
case nr: Node.NodeRollback[Nothing] => nr
|
||||
case nc: Node.NodeCreate[V.ContractId] => nc copy (optLocation = None)
|
||||
case nf: Node.NodeFetch[V.ContractId] => nf copy (optLocation = None)
|
||||
case ne: Node.NodeExercises[Nothing, V.ContractId] =>
|
||||
ne copy (optLocation = None)
|
||||
case nl: Node.NodeLookupByKey[V.ContractId] => nl copy (optLocation = None)
|
||||
}
|
||||
nodeWithoutLocation
|
||||
}
|
||||
isReplayedBy(withoutLocation, n) shouldBe Right(())
|
||||
isReplayedBy(n, withoutLocation) shouldBe Right(())
|
||||
}
|
||||
}
|
||||
|
||||
"suffixCid" - {
|
||||
@ -699,7 +683,6 @@ object TransactionSpec {
|
||||
Ref.QualifiedName.assertFromString("DummyModule:dummyName"),
|
||||
),
|
||||
choiceId = "dummyChoice",
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set.empty,
|
||||
chosenValue = V.ValueUnit,
|
||||
@ -724,7 +707,6 @@ object TransactionSpec {
|
||||
templateId = Ref.Identifier.assertFromString("-dummyPkg-:DummyModule:dummyName"),
|
||||
arg = V.ValueContractId(dummyCid),
|
||||
agreementText = "dummyAgreement",
|
||||
optLocation = None,
|
||||
signatories = Set.empty,
|
||||
stakeholders = Set.empty,
|
||||
key = None,
|
||||
|
@ -5,15 +5,7 @@ package com.daml.lf
|
||||
package transaction
|
||||
|
||||
import com.daml.lf.data.ImmArray
|
||||
import com.daml.lf.data.Ref.{
|
||||
Location,
|
||||
PackageId,
|
||||
ModuleName,
|
||||
Identifier,
|
||||
TypeConName,
|
||||
ChoiceName,
|
||||
Party,
|
||||
}
|
||||
import com.daml.lf.data.Ref.{Identifier, TypeConName, ChoiceName, Party}
|
||||
import com.daml.lf.transaction.Node.{
|
||||
KeyWithMaintainers,
|
||||
GenNode,
|
||||
@ -71,16 +63,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
|
||||
//--[samples]--
|
||||
|
||||
private val samPackageId1 = PackageId.assertFromString("packageId1")
|
||||
private val samModuleName1 = ModuleName.assertFromString("moduleName1")
|
||||
|
||||
private val samLocation1 = Location(samPackageId1, samModuleName1, "someString1", (1, 2), (3, 4))
|
||||
private val samLocation2 =
|
||||
Location(samPackageId1, samModuleName1, "someString2", (11, 22), (33, 44))
|
||||
|
||||
private val samOptLocation1: Option[Location] = None
|
||||
private val samOptLocation2: Option[Location] = Some(samLocation1)
|
||||
|
||||
private val samBool1 = true
|
||||
private val samBool2 = false
|
||||
|
||||
@ -123,7 +105,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
templateId = samTemplateId1,
|
||||
arg = samValue1,
|
||||
agreementText = samText1,
|
||||
optLocation = samOptLocation1,
|
||||
signatories = samParties1,
|
||||
stakeholders = samParties2,
|
||||
key = key,
|
||||
@ -138,7 +119,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
} yield NodeFetch(
|
||||
coid = samContractId1,
|
||||
templateId = samTemplateId1,
|
||||
optLocation = samOptLocation1,
|
||||
actingParties = actingParties,
|
||||
signatories = samParties2,
|
||||
stakeholders = samParties3,
|
||||
@ -153,7 +133,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
result <- Seq(None, Some(samContractId1))
|
||||
} yield NodeLookupByKey(
|
||||
templateId = samTemplateId1,
|
||||
optLocation = samOptLocation1,
|
||||
result = result,
|
||||
key = samKWM3,
|
||||
version = version,
|
||||
@ -168,7 +147,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
targetCoid = samContractId2,
|
||||
templateId = samTemplateId2,
|
||||
choiceId = samChoiceName1,
|
||||
optLocation = samOptLocation2,
|
||||
consuming = samBool1,
|
||||
actingParties = samParties1,
|
||||
chosenValue = samValue1,
|
||||
@ -201,7 +179,11 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
val nid1 = NodeId(1)
|
||||
val parent = exe.copy(children = ImmArray(nid1))
|
||||
val version = TransactionVersion.minExceptions
|
||||
VersionedTransaction(version, HashMap(nid0 -> parent, nid1 -> child), ImmArray(nid0))
|
||||
VersionedTransaction(
|
||||
version,
|
||||
HashMap(nid0 -> parent, nid1 -> child),
|
||||
ImmArray(nid0),
|
||||
)
|
||||
}
|
||||
|
||||
private def preTweakedVTXs: Seq[VTX] = flatVTXs ++ nestedVTXs
|
||||
@ -237,15 +219,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
if (x != samVersion1) samVersion1 else samVersion2
|
||||
}
|
||||
|
||||
private def changeLocation(x: Location): Location = {
|
||||
if (x != samLocation1) samLocation1 else samLocation2
|
||||
}
|
||||
|
||||
private def tweakOptLocation = Tweak[Option[Location]] {
|
||||
case None => List(Some(samLocation1), Some(samLocation2))
|
||||
case Some(loc) => List(None, Some(changeLocation(loc)))
|
||||
}
|
||||
|
||||
private def changeText(x: String): String = {
|
||||
x + "_XXX"
|
||||
}
|
||||
@ -302,9 +275,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
private val tweakCreateAgreementText = Tweak.single[Node] { case nc: Node.NodeCreate[_] =>
|
||||
nc.copy(agreementText = changeText(nc.agreementText))
|
||||
}
|
||||
private val tweakCreateOptLocation = Tweak[Node] { case nc: Node.NodeCreate[_] => //insig
|
||||
tweakOptLocation.run(nc.optLocation).map { x => nc.copy(optLocation = x) }
|
||||
}
|
||||
private val tweakCreateSignatories = Tweak[Node] { case nc: Node.NodeCreate[_] =>
|
||||
tweakPartySet.run(nc.signatories).map { x => nc.copy(signatories = x) }
|
||||
}
|
||||
@ -332,11 +302,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
"tweakCreateVersion" -> tweakCreateVersion,
|
||||
)
|
||||
|
||||
private val insigCreateTweaks =
|
||||
Map(
|
||||
"tweakCreateOptLocation" -> tweakCreateOptLocation
|
||||
)
|
||||
|
||||
//--[Fetch node tweaks]--
|
||||
|
||||
private val tweakFetchCoid = Tweak.single[Node] { case nf: Node.NodeFetch[_] =>
|
||||
@ -345,9 +310,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
private val tweakFetchTemplateId = Tweak.single[Node] { case nf: Node.NodeFetch[_] =>
|
||||
nf.copy(templateId = changeTemplateId(nf.templateId))
|
||||
}
|
||||
private val tweakFetchOptLocation = Tweak[Node] { case nf: Node.NodeFetch[_] =>
|
||||
tweakOptLocation.run(nf.optLocation).map { x => nf.copy(optLocation = x) }
|
||||
}
|
||||
private val tweakFetchActingPartiesEmpty = Tweak[Node] {
|
||||
case nf: Node.NodeFetch[_] if nf.actingParties.isEmpty => //insig
|
||||
tweakPartySet.run(nf.actingParties).map { x => nf.copy(actingParties = x) }
|
||||
@ -388,7 +350,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
|
||||
private val insigFetchTweaks =
|
||||
Map(
|
||||
"tweakFetchOptLocation" -> tweakFetchOptLocation,
|
||||
"tweakFetchActingParties(Empty)" -> tweakFetchActingPartiesEmpty,
|
||||
"tweakFetchKey(None)" -> tweakFetchKey(tweakOptKeyMaintainersNone),
|
||||
"tweakFetchByKey(Old Version)" -> tweakFetchByKey(versionBeforeMinByKey),
|
||||
@ -399,9 +360,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
private val tweakLookupTemplateId = Tweak.single[Node] { case nl: Node.NodeLookupByKey[_] =>
|
||||
nl.copy(templateId = changeTemplateId(nl.templateId))
|
||||
}
|
||||
private val tweakLookupOptLocation = Tweak[Node] { case nl: Node.NodeLookupByKey[_] =>
|
||||
tweakOptLocation.run(nl.optLocation).map { x => nl.copy(optLocation = x) }
|
||||
}
|
||||
private val tweakLookupKey = Tweak[Node] { case nl: Node.NodeLookupByKey[_] =>
|
||||
tweakKeyMaintainers.run(nl.key).map { x => nl.copy(key = x) }
|
||||
}
|
||||
@ -422,8 +380,7 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
|
||||
private val insigLookupTweaks =
|
||||
Map(
|
||||
"tweakLookupOptLocation" -> tweakLookupOptLocation,
|
||||
"tweakExerciseKey(None)" -> tweakExerciseKey(tweakOptKeyMaintainersNone),
|
||||
"tweakExerciseKey(None)" -> tweakExerciseKey(tweakOptKeyMaintainersNone)
|
||||
)
|
||||
|
||||
//--[Exercise node tweaks]--
|
||||
@ -437,9 +394,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
private val tweakExerciseChoiceId = Tweak.single[Node] { case ne: Node.NodeExercises[_, _] =>
|
||||
ne.copy(choiceId = changeChoiceId(ne.choiceId))
|
||||
}
|
||||
private val tweakExerciseOptLocation = Tweak[Node] { case ne: Node.NodeExercises[_, _] =>
|
||||
tweakOptLocation.run(ne.optLocation).map { x => ne.copy(optLocation = x) }
|
||||
}
|
||||
private val tweakExerciseConsuming = Tweak.single[Node] { case ne: Node.NodeExercises[_, _] =>
|
||||
ne.copy(consuming = changeBoolean(ne.consuming))
|
||||
}
|
||||
@ -507,7 +461,6 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
|
||||
private val insigExeTweaks =
|
||||
Map(
|
||||
"tweakExerciseOptLocation" -> tweakExerciseOptLocation,
|
||||
"tweakExerciseExerciseResult(None)" -> tweakExerciseExerciseResultNone,
|
||||
"tweakExerciseKey(None)" -> tweakExerciseKey(tweakOptKeyMaintainersNone),
|
||||
"tweakExerciseByKey(Old Version)" -> tweakExerciseByKey(versionBeforeMinByKey),
|
||||
@ -536,7 +489,7 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC
|
||||
}
|
||||
|
||||
private def insignificantTweaks: Map[String, Tweak[VTX]] = {
|
||||
(insigCreateTweaks ++ insigFetchTweaks ++ insigLookupTweaks ++ insigExeTweaks)
|
||||
(insigFetchTweaks ++ insigLookupTweaks ++ insigExeTweaks)
|
||||
.map { case (name, tw) => (name, tweakTxNodes(tw)) }
|
||||
}
|
||||
|
||||
|
@ -202,7 +202,7 @@ class IdeLedgerClient(
|
||||
_currentSubmission = None
|
||||
_ledger = ledger.insertAssertMustFail(actAs.toSet, readAs, optLocation)
|
||||
seed = ScenarioRunner.nextSeed(
|
||||
error.ptx.unwind.context.nextActionChildSeed
|
||||
error.ptx.unwind().context.nextActionChildSeed
|
||||
)
|
||||
Right(())
|
||||
})
|
||||
|
@ -51,9 +51,8 @@ private[migration] class V5_1__Populate_Event_Data extends BaseJavaMigration {
|
||||
|
||||
val txs = loadTransactions(conn)
|
||||
val data = txs.flatMap { case (txId, tx) =>
|
||||
tx.nodes.collect {
|
||||
case (nodeId, NodeCreate(cid, _, _, _, _, signatories, stakeholders, _, _)) =>
|
||||
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
|
||||
tx.nodes.collect { case (nodeId, NodeCreate(cid, _, _, _, signatories, stakeholders, _, _)) =>
|
||||
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -51,9 +51,8 @@ private[migration] class V10_1__Populate_Event_Data extends BaseJavaMigration {
|
||||
|
||||
val txs = loadTransactions(conn)
|
||||
val data = txs.flatMap { case (txId, tx) =>
|
||||
tx.nodes.collect {
|
||||
case (nodeId, NodeCreate(cid, _, _, _, _, signatories, stakeholders, _, _)) =>
|
||||
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
|
||||
tx.nodes.collect { case (nodeId, NodeCreate(cid, _, _, _, signatories, stakeholders, _, _)) =>
|
||||
(cid, EventId(txId, nodeId), signatories, stakeholders -- signatories)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -31,7 +31,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
templateId = someTemplateId,
|
||||
arg = someContractArgument,
|
||||
agreementText = someAgreement,
|
||||
optLocation = None,
|
||||
signatories = Set(alice),
|
||||
stakeholders = Set(alice),
|
||||
key = None,
|
||||
@ -49,7 +48,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
someTemplateId,
|
||||
someContractArgument,
|
||||
someAgreement,
|
||||
optLocation = None,
|
||||
signatories = Set(bob),
|
||||
stakeholders = Set(bob),
|
||||
key = Some(
|
||||
@ -67,7 +65,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
targetCoid = create1,
|
||||
templateId = someTemplateId,
|
||||
choiceId = someChoiceName,
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set(bob),
|
||||
chosenValue = someChoiceArgument,
|
||||
@ -87,7 +84,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
NodeFetch(
|
||||
coid = create2,
|
||||
templateId = someTemplateId,
|
||||
optLocation = None,
|
||||
actingParties = Set(bob),
|
||||
signatories = Set(bob),
|
||||
stakeholders = Set(bob),
|
||||
@ -104,7 +100,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
targetCoid = create2,
|
||||
templateId = someTemplateId,
|
||||
choiceId = someChoiceName,
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set(bob),
|
||||
chosenValue = someChoiceArgument,
|
||||
@ -127,7 +122,6 @@ private[dao] trait JdbcLedgerDaoDivulgenceSpec extends LoneElement with Inside {
|
||||
someTemplateId,
|
||||
someContractArgument,
|
||||
someAgreement,
|
||||
optLocation = None,
|
||||
signatories = Set(bob),
|
||||
stakeholders = Set(alice, bob),
|
||||
key = Some(
|
||||
|
@ -197,7 +197,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
templateId = templateId,
|
||||
arg = contractArgument,
|
||||
agreementText = someAgreement,
|
||||
optLocation = None,
|
||||
signatories = signatories,
|
||||
stakeholders = stakeholders,
|
||||
key = key,
|
||||
@ -212,7 +211,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
targetCoid = targetCid,
|
||||
templateId = someTemplateId,
|
||||
choiceId = someChoiceName,
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set(alice),
|
||||
chosenValue = someChoiceArgument,
|
||||
@ -233,7 +231,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
NodeFetch(
|
||||
coid = contractId,
|
||||
templateId = someTemplateId,
|
||||
optLocation = None,
|
||||
actingParties = Set(party),
|
||||
signatories = Set(party),
|
||||
stakeholders = Set(party),
|
||||
@ -368,7 +365,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
targetCoid = id,
|
||||
templateId = someTemplateId,
|
||||
choiceId = someChoiceName,
|
||||
optLocation = None,
|
||||
consuming = false,
|
||||
actingParties = Set(alice),
|
||||
chosenValue = someChoiceArgument,
|
||||
@ -386,7 +382,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
NodeFetch(
|
||||
coid = id,
|
||||
templateId = someTemplateId,
|
||||
optLocation = None,
|
||||
actingParties = divulgees,
|
||||
signatories = Set(alice),
|
||||
stakeholders = Set(alice),
|
||||
@ -720,7 +715,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
templateId = someTemplateId,
|
||||
arg = someContractArgument,
|
||||
agreementText = someAgreement,
|
||||
optLocation = None,
|
||||
signatories = Set(party),
|
||||
stakeholders = Set(party),
|
||||
key = Some(KeyWithMaintainers(someContractKey(party, key), Set(party))),
|
||||
@ -753,7 +747,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
targetCoid = contractId,
|
||||
templateId = someTemplateId,
|
||||
choiceId = Ref.ChoiceName.assertFromString("Archive"),
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = Set(party),
|
||||
chosenValue = LfValue.ValueUnit,
|
||||
@ -790,7 +783,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
val lookupByKeyNodeId = txBuilder.add(
|
||||
NodeLookupByKey(
|
||||
someTemplateId,
|
||||
None,
|
||||
KeyWithMaintainers(someContractKey(party, key), Set(party)),
|
||||
result,
|
||||
version = TransactionVersion.minVersion,
|
||||
@ -818,7 +810,6 @@ private[dao] trait JdbcLedgerDaoSuite extends JdbcLedgerDaoBackend {
|
||||
NodeFetch(
|
||||
coid = contractId,
|
||||
templateId = someTemplateId,
|
||||
optLocation = None,
|
||||
actingParties = Set(party),
|
||||
signatories = Set(party),
|
||||
stakeholders = Set(party),
|
||||
|
@ -231,14 +231,12 @@ private[transaction] object ModelConformanceValidator {
|
||||
case (
|
||||
Node.NodeLookupByKey(
|
||||
recordedTemplateId,
|
||||
_,
|
||||
recordedKey,
|
||||
recordedResult,
|
||||
recordedVersion,
|
||||
),
|
||||
Node.NodeLookupByKey(
|
||||
replayedTemplateId,
|
||||
_,
|
||||
replayedKey,
|
||||
replayedResult,
|
||||
replayedVersion,
|
||||
|
@ -21,7 +21,6 @@ class ProjectionsSpec extends AnyWordSpec with Matchers {
|
||||
templateId = Identifier.assertFromString("some-package:Foo:Bar"),
|
||||
arg = ValueText("foo"),
|
||||
agreementText = "agreement",
|
||||
optLocation = None,
|
||||
signatories = signatories,
|
||||
stakeholders = stakeholders,
|
||||
key = None,
|
||||
@ -41,7 +40,6 @@ class ProjectionsSpec extends AnyWordSpec with Matchers {
|
||||
QualifiedName.assertFromString("Foo:Bar"),
|
||||
),
|
||||
choiceId = Name.assertFromString("someChoice"),
|
||||
optLocation = None,
|
||||
consuming = true,
|
||||
actingParties = actingParties,
|
||||
chosenValue = ValueText("foo"),
|
||||
|
@ -34,13 +34,11 @@ private[replay] final class Adapter(
|
||||
create.copy(
|
||||
templateId = adapt(create.templateId),
|
||||
arg = adapt(create.arg),
|
||||
optLocation = None,
|
||||
key = create.key.map(adapt),
|
||||
)
|
||||
case exe: Node.NodeExercises[NodeId, ContractId] =>
|
||||
exe.copy(
|
||||
templateId = adapt(exe.templateId),
|
||||
optLocation = None,
|
||||
chosenValue = adapt(exe.chosenValue),
|
||||
children = ImmArray.empty,
|
||||
exerciseResult = exe.exerciseResult.map(adapt),
|
||||
@ -49,14 +47,12 @@ private[replay] final class Adapter(
|
||||
case fetch: Node.NodeFetch[ContractId] =>
|
||||
fetch.copy(
|
||||
templateId = adapt(fetch.templateId),
|
||||
optLocation = None,
|
||||
key = fetch.key.map(adapt),
|
||||
)
|
||||
case lookup: Node.NodeLookupByKey[ContractId] =>
|
||||
lookup
|
||||
.copy(
|
||||
templateId = adapt(lookup.templateId),
|
||||
optLocation = None,
|
||||
key = adapt(lookup.key),
|
||||
)
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user