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:
nickchapman-da 2021-07-19 14:03:30 +01:00 committed by GitHub
parent e62852fbfd
commit 0e09f3ba08
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 132 additions and 213 deletions

View File

@ -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))

View File

@ -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,

View File

@ -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,
_,
_,

View File

@ -49,6 +49,7 @@ class LargeTransactionTest extends AnyWordSpec with Matchers with BazelRunfiles
effectiveAt = effectiveAt,
optLocation = None,
tx = tx,
locationInfo = Map.empty,
l = ledger,
)
.fold(

View File

@ -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)

View File

@ -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,
),

View File

@ -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)

View File

@ -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,

View File

@ -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

View File

@ -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))
}
}

View File

@ -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
}
}

View File

@ -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) =>

View File

@ -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(())
}

View File

@ -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),

View File

@ -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,

View File

@ -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]
}

View File

@ -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.

View File

@ -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

View File

@ -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

View File

@ -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 &&

View File

@ -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,

View File

@ -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,

View File

@ -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)) }
}

View File

@ -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(())
})

View File

@ -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)
}
}

View File

@ -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)
}
}

View File

@ -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(

View File

@ -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),

View File

@ -231,14 +231,12 @@ private[transaction] object ModelConformanceValidator {
case (
Node.NodeLookupByKey(
recordedTemplateId,
_,
recordedKey,
recordedResult,
recordedVersion,
),
Node.NodeLookupByKey(
replayedTemplateId,
_,
replayedKey,
replayedResult,
replayedVersion,

View File

@ -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"),

View File

@ -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),
)
}