From 9db5ccfc4e26fa0c10f6ffdaf3eaa7d82a21b7fc Mon Sep 17 00:00:00 2001 From: nickchapman-da <49153372+nickchapman-da@users.noreply.github.com> Date: Mon, 16 Aug 2021 15:54:50 +0100 Subject: [PATCH] Normalize transactions & values as a separate pass (#10524) * Normalize transactions & values as a separate pass. Use for simpler defintiion of isReplayedBy. CHANGELOG_BEGIN CHANGELOG_END normalize transaction version * remove stray import from bad merge which breaks scala 2_12 build * change isReplayedBy to only norm its RIGHT (replay) argument * add forgotton normalization for ValueEmum * switch to use existing value normalization code (remove my newly coded duplicate code) * normalize submittedTransaction before calling engine.validate * dont call normalizeTx from Engine.validate * *do* call normalizeTx from Engine.validate --- .../digitalasset/daml/lf/engine/Engine.scala | 2 +- .../daml/lf/engine/EngineTest.scala | 38 +- .../daml/lf/transaction/Normalization.scala | 127 ++++++ .../daml/lf/transaction/Util.scala | 8 +- .../daml/lf/transaction/Validation.scala | 387 +----------------- .../daml/lf/transaction/TransactionSpec.scala | 3 +- .../daml/lf/validation/ValidationSpec.scala | 10 +- .../services/ApiSubmissionServiceSpec.scala | 4 +- .../ModelConformanceValidatorSpec.scala | 4 +- 9 files changed, 193 insertions(+), 390 deletions(-) create mode 100644 daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Normalization.scala diff --git a/daml-lf/engine/src/main/scala/com/digitalasset/daml/lf/engine/Engine.scala b/daml-lf/engine/src/main/scala/com/digitalasset/daml/lf/engine/Engine.scala index d97dd8084183..a21422ad824f 100644 --- a/daml-lf/engine/src/main/scala/com/digitalasset/daml/lf/engine/Engine.scala +++ b/daml-lf/engine/src/main/scala/com/digitalasset/daml/lf/engine/Engine.scala @@ -222,7 +222,7 @@ class Engine(val config: EngineConfig = new EngineConfig(LanguageVersion.StableV (rtx, _) = result validationResult <- transaction.Validation - .isReplayedBy(tx, rtx) + .isReplayedBy(transaction.Normalization.normalizeTx(tx), rtx) .fold( e => ResultError(Error.Validation.ReplayMismatch(e)), _ => ResultDone.Unit, diff --git a/daml-lf/engine/src/test/scala/com/digitalasset/daml/lf/engine/EngineTest.scala b/daml-lf/engine/src/test/scala/com/digitalasset/daml/lf/engine/EngineTest.scala index 96f3febab6e4..1104775e2c2e 100644 --- a/daml-lf/engine/src/test/scala/com/digitalasset/daml/lf/engine/EngineTest.scala +++ b/daml-lf/engine/src/test/scala/com/digitalasset/daml/lf/engine/EngineTest.scala @@ -24,7 +24,7 @@ import com.daml.lf.transaction.{ Transaction => Tx, TransactionVersion => TxVersions, } -import com.daml.lf.transaction.Validation.isReplayedBy +import com.daml.lf.transaction.{Normalization, Validation, ReplayMismatch} import com.daml.lf.value.Value import Value._ import com.daml.lf.speedy.{InitialSeeding, SValue, svalue} @@ -526,8 +526,9 @@ class EngineTest val Right((tx, meta)) = interpretResult val Right(submitter) = tx.guessSubmitter val submitters = Set(submitter) + val ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) val validated = engine - .validate(submitters, tx, let, participant, meta.submissionTime, submissionSeed) + .validate(submitters, ntx, let, participant, meta.submissionTime, submissionSeed) .consume(lookupContract, lookupPackage, lookupKey) validated match { case Left(e) => @@ -612,8 +613,9 @@ class EngineTest "be validated" in { forAll(cases) { case (templateId, signatories, submitters) => val Right((tx, meta)) = interpretResult(templateId, signatories, submitters) + val ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) val validated = engine - .validate(submitters, tx, let, participant, meta.submissionTime, submissionSeed) + .validate(submitters, ntx, let, participant, meta.submissionTime, submissionSeed) .consume( lookupContract, lookupPackage, @@ -736,8 +738,9 @@ class EngineTest } "be validated" in { + val ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) val validated = engine - .validate(Set(submitter), tx, let, participant, let, submissionSeed) + .validate(Set(submitter), ntx, let, participant, let, submissionSeed) .consume( lookupContract, lookupPackage, @@ -879,8 +882,9 @@ class EngineTest } "be validated" in { + val ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) val validated = engine - .validate(submitters, tx, let, participant, let, submissionSeed) + .validate(submitters, ntx, let, participant, let, submissionSeed) .consume( lookupContract, lookupPackage, @@ -1140,8 +1144,9 @@ class EngineTest } "be validated" in { + val ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) val validated = engine - .validate(Set(submitter), tx, let, participant, let, submissionSeed) + .validate(Set(submitter), ntx, let, participant, let, submissionSeed) .consume( lookupContract, lookupPackage, @@ -1586,7 +1591,8 @@ class EngineTest nid -> fetch } - fetchNodes.foreach { case (nid, n) => + fetchNodes.foreach { case (_, n) => + val nid = NodeId(0) //we must use node-0 so the constructed tx is normalized val fetchTx = VersionedTransaction(n.version, Map(nid -> n), ImmArray(nid)) val Right((reinterpreted, _)) = engine @@ -2103,8 +2109,16 @@ class EngineTest def validate(tx: SubmittedTransaction, metaData: Tx.Metadata) = for { submitter <- tx.guessSubmitter + ntx = SubmittedTransaction(Normalization.normalizeTx(tx)) res <- engine - .validate(Set(submitter), tx, let, participant, metaData.submissionTime, submissionSeed) + .validate( + Set(submitter), + ntx, + let, + participant, + metaData.submissionTime, + submissionSeed, + ) .consume( _ => None, lookupPackage, @@ -2748,6 +2762,14 @@ object EngineTest { case _ => false } + private def isReplayedBy[Nid, Cid]( + recorded: VersionedTransaction[Nid, Cid], + replayed: VersionedTransaction[Nid, Cid], + ): Either[ReplayMismatch[Nid, Cid], Unit] = { + // we normalize the LEFT arg before calling isReplayedBy to mimic the effect of serialization + Validation.isReplayedBy(Normalization.normalizeTx(recorded), replayed) + } + private def reinterpret( engine: Engine, submitters: Set[Party], diff --git a/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Normalization.scala b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Normalization.scala new file mode 100644 index 000000000000..ed46ab39ba53 --- /dev/null +++ b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Normalization.scala @@ -0,0 +1,127 @@ +// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved. +// SPDX-License-Identifier: Apache-2.0 + +package com.daml.lf +package transaction + +import com.daml.lf.value.{Value => V} + +import com.daml.lf.transaction.Node.{ + KeyWithMaintainers, + GenNode, + NodeCreate, + NodeFetch, + NodeLookupByKey, + NodeExercises, + NodeRollback, +} + +class Normalization[Nid, Cid] { + + /** This class provides methods to normalize a transaction and embedded values. + * + * Informal spec: normalization is the result of serialization and deserialization. + * + * Here we take care of the following: + * - type information is dropped from Variant and Record values + * - field names are dropped from Records + * - values are normalized recursively + * - all values embedded in transaction nodes are normalized + * - version-specific normalization is applied to the 'byKey' fields of 'NodeFetch' and 'NodeExercises' + * + * We do not normalize the node-ids in the transaction here, but rather assume that + * aspect of normalization has already been performed (by the engine, or by + * deserialization). + * + * Eventually we would like that all aspects of normalization are achieved directly by + * the transaction which is constructed by the engine. When this is done, we will no + * longer need this separate normalization pass. + */ + + private type Val = V[Cid] + private type KWM = KeyWithMaintainers[Val] + private type Node = GenNode[Nid, Cid] + private type VTX = VersionedTransaction[Nid, Cid] + + def normalizeTx(vtx: VTX): VTX = { + vtx match { + case VersionedTransaction(_, nodes, roots) => + // TODO: Normalized version calc should be shared with code in asVersionedTransaction + val version = roots.iterator.foldLeft(TransactionVersion.minVersion) { (acc, nodeId) => + import scala.Ordering.Implicits.infixOrderingOps + nodes(nodeId).optVersion match { + case Some(version) => acc max version + case None => acc max TransactionVersion.minExceptions + } + } + VersionedTransaction( + version, + nodes.map { case (k, v) => + (k, normNode(v)) + }, + vtx.roots, + ) + } + } + + private def normNode( + node: Node + ): Node = { + import scala.Ordering.Implicits.infixOrderingOps + node match { + + case old: NodeCreate[_] => + old + .copy(arg = normValue(old.version)(old.arg)) + .copy(key = old.key.map(normKWM(old.version))) + + case old: NodeFetch[_] => + (if (old.version >= TransactionVersion.minByKey) { + old + } else { + old.copy(byKey = false) + }) + .copy( + key = old.key.map(normKWM(old.version)) + ) + + case old: NodeExercises[_, _] => + (if (old.version >= TransactionVersion.minByKey) { + old + } else { + old.copy(byKey = false) + }) + .copy( + chosenValue = normValue(old.version)(old.chosenValue), + exerciseResult = old.exerciseResult.map(normValue(old.version)), + key = old.key.map(normKWM(old.version)), + ) + + case old: NodeLookupByKey[_] => + old.copy( + key = normKWM(old.version)(old.key) + ) + + case old: NodeRollback[_] => old + + } + } + + private def normValue(version: TransactionVersion)(x: Val): Val = { + Util.assertNormalizeValue(x, version) + } + + private def normKWM(version: TransactionVersion)(x: KWM): KWM = { + x match { + case KeyWithMaintainers(key, maintainers) => + KeyWithMaintainers(normValue(version)(key), maintainers) + } + } + +} + +object Normalization { + def normalizeTx[Nid, Cid](tx: VersionedTransaction[Nid, Cid]): VersionedTransaction[Nid, Cid] = { + new Normalization().normalizeTx(tx) + } +} diff --git a/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Util.scala b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Util.scala index 7c29688cfd19..0f15a1e8fb6f 100644 --- a/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Util.scala +++ b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Util.scala @@ -26,10 +26,10 @@ object Util { // unsafe version of `normalize` @throws[IllegalArgumentException] - def assertNormalizeValue( - value0: Value[ContractId], + def assertNormalizeValue[Cid]( + value0: Value[Cid], version: TransactionVersion, - ): Value[ContractId] = { + ): Value[Cid] = { import Ordering.Implicits.infixOrderingOps @@ -43,7 +43,7 @@ object Util { x } - def go(value: Value[ContractId]): Value[ContractId] = + def go(value: Value[Cid]): Value[Cid] = value match { case ValueEnum(tyCon, cons) => ValueEnum(handleTypeInfo(tyCon), cons) diff --git a/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Validation.scala b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Validation.scala index 0449d1bff28c..bdc1528b0dc8 100644 --- a/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Validation.scala +++ b/daml-lf/transaction/src/main/scala/com/digitalasset/daml/lf/transaction/Validation.scala @@ -4,395 +4,46 @@ package com.daml.lf package transaction -import com.daml.lf.data.ImmArray -import com.daml.lf.data.Ref.{Name, TypeConName} -import com.daml.lf.transaction.Node.KeyWithMaintainers -import com.daml.lf.value.Value -import scalaz.Equal - -import scala.annotation.tailrec -import scala.collection.compat._ -import scala.collection.compat.immutable.LazyList - -private final class Validation[Nid, Cid](implicit ECid: Equal[Cid]) { - - import scalaz.std.option._ - import scalaz.syntax.equal._ - - private[this] def tyConIsReplayedBy( - recordedTyCon: Option[TypeConName], - replayedTyCon: Option[TypeConName], - ): Boolean = - recordedTyCon.isEmpty || recordedTyCon == replayedTyCon - - private[this] def nameIsReplayedBy(tuple: (Option[Name], Option[Name])): Boolean = - tuple match { - case (None, _) => true - case (recordedName, replayedName) => recordedName == replayedName - } - - private[this] def keyIsReplayedBy( - recorded: KeyWithMaintainers[Value[Cid]], - replayed: KeyWithMaintainers[Value[Cid]], - ): Boolean = { - valueIsReplayedBy(recorded.key, replayed.key) && recorded.maintainers == replayed.maintainers - } - - private[this] def keyIsReplayedBy( - recorded: Option[KeyWithMaintainers[Value[Cid]]], - replayed: Option[KeyWithMaintainers[Value[Cid]]], - ): Boolean = { - (recorded, replayed) match { - case (None, None) => true - case (Some(recordedValue), Some(replayedValue)) => - keyIsReplayedBy(recordedValue, replayedValue) - case _ => false - } - } - - private[this] def resultIsReplayedBy( - recorded: Option[Value[Cid]], - replayed: Option[Value[Cid]], - ) = - (recorded, replayed) match { - case (None, None) => true - case (Some(recordedValue), Some(replayedValue)) => - valueIsReplayedBy(recordedValue, replayedValue) - case _ => false - } - - private[this] def byKeyIsReplacedBy( - version: TransactionVersion, - recorded: Boolean, - replayed: Boolean, - ): Boolean = { - import scala.Ordering.Implicits.infixOrderingOps - if (version >= TransactionVersion.minByKey) { - recorded == replayed - } else { - true - } - } - - private[this] def keys[K](entries: ImmArray[(K, _)]): Iterator[K] = - entries.iterator.map(_._1) - - private[this] def values[V](entries: ImmArray[(_, V)]): Iterator[V] = - entries.iterator.map(_._2) - - private def valueIsReplayedBy( - recorded: Value[Cid], - replayed: Value[Cid], - ): Boolean = { - - import Value._ - - @tailrec - def loop(tuples: LazyList[(Value[Cid], Value[Cid])]): Boolean = - tuples match { - case LazyList.cons(tuple, rest) => - tuple match { - case (ValueEnum(recordedTyCon, recordedName), ValueEnum(replayedTyCon, replayedName)) => - tyConIsReplayedBy(recordedTyCon, replayedTyCon) && - recordedName == replayedName && - loop(rest) - case (recordedLeaf: ValueCidlessLeaf, replayedLeaf: ValueCidlessLeaf) => - recordedLeaf == replayedLeaf && - loop(rest) - case (ValueContractId(recordedValue), ValueContractId(replayedValue)) => - recordedValue === replayedValue && - loop(rest) - case ( - ValueRecord(recordedTyCon, recordedFields), - ValueRecord(replayedTyCon, replayedFields), - ) => - tyConIsReplayedBy(recordedTyCon, replayedTyCon) && - recordedFields.length == replayedFields.length && - (keys(recordedFields) zip keys(replayedFields)).forall(nameIsReplayedBy) && - loop((values(recordedFields) zip values(recordedFields)) ++: rest) - case ( - ValueVariant(recordedTyCon, recordedVariant, recordedValue), - ValueVariant(replayedTyCon, replayedVariant, replayedValue), - ) => - tyConIsReplayedBy(recordedTyCon, replayedTyCon) && - recordedVariant == replayedVariant && - loop((recordedValue, replayedValue) +: rest) - case (ValueList(recordedValues), ValueList(replayedValues)) => - recordedValues.length == replayedValues.length && - loop((recordedValues.iterator zip replayedValues.iterator) ++: rest) - case (ValueOptional(recordedValue), ValueOptional(replayedValue)) => - (recordedValue, replayedValue) match { - case (Some(recorded), Some(replayed)) => loop((recorded, replayed) +: rest) - case (None, None) => loop(rest) - case _ => false - } - case (ValueTextMap(recordedEntries), ValueTextMap(replayedEntries)) => - recordedEntries.length == replayedEntries.length && - (keys(recordedEntries.toImmArray) sameElements keys(replayedEntries.toImmArray)) && - loop( - (values(recordedEntries.toImmArray) zip values(replayedEntries.toImmArray)) ++: - rest - ) - case (ValueGenMap(recordedEntries), ValueGenMap(replayedEntries)) => - recordedEntries.length == replayedEntries.length && - loop( - (keys(recordedEntries) zip keys(replayedEntries)) ++: - (values(recordedEntries) zip values(replayedEntries)) ++: - rest - ) - case _ => - false - } - case LazyList() => - true - } - loop(LazyList((recorded, replayed))) - } +private final class Validation[Nid, Cid]() { /** Whether `replayed` is the result of reinterpreting this transaction. * * @param recorded : the transaction to be validated. * @param replayed : the transaction resulting from the reinterpretation of * the root nodes of [[recorded]]. - * @note This function is asymmetric in order to provide backward compatibility. - * For instance, some field may be undefined in the [[recorded]] transaction - * while present in the [[replayed]] one. + * @note This function is symmetric + * + * 'isReplayedBy' normalizes its 2nd argument, and then determines structural equality. + * + * The RIGHT `replayed` arg must be normalized because the engine currently does not. */ + private def isReplayedBy( recorded: VersionedTransaction[Nid, Cid], replayed: VersionedTransaction[Nid, Cid], ): Either[ReplayMismatch[Nid, Cid], Unit] = { - - type Exe = Node.NodeExercises[Nid, Cid] - - sealed trait StackEntry - final case class ExerciseEntry(exe1: Exercise, exe2: Exercise) extends StackEntry - final case class RollbackEntry(rb1: Rollback, rb2: Rollback) extends StackEntry - final case class Exercise( - nid: Nid, - exe: Exe, - children: LazyList[Nid], - ) - final case class Rollback( - nid: Nid, - children: LazyList[Nid], - ) - - @tailrec - def loop( - nids1: LazyList[Nid], - nids2: LazyList[Nid], - stack: List[StackEntry] = List.empty, - ): Either[ReplayMismatch[Nid, Cid], Unit] = - (nids1, nids2) match { - case (LazyList.cons(nid1, rest1), LazyList.cons(nid2, rest2)) => - (recorded.nodes(nid1), replayed.nodes(nid2)) match { - case ( - Node.NodeCreate( - coid1, - templateId1, - arg1, - agreementText1, - signatories1, - stakeholders1, - key1, - version1, - ), - Node.NodeCreate( - coid2, - templateId2, - arg2, - agreementText2, - signatories2, - stakeholders2, - key2, - version2, - ), - ) - if version1 == version2 && - coid1 === coid2 && - templateId1 == templateId2 && - valueIsReplayedBy(arg1, arg2) && - agreementText1 == agreementText2 && - signatories1 == signatories2 && - stakeholders1 == stakeholders2 && - keyIsReplayedBy(key1, key2) => - loop(rest1, rest2, stack) - case ( - Node.NodeFetch( - coid1, - templateId1, - actingParties1, - signatories1, - stakeholders1, - key1, - byKey1, - version1, - ), - Node.NodeFetch( - coid2, - templateId2, - actingParties2, - signatories2, - stakeholders2, - key2, - byKey2, - version2, - ), - ) - if version1 == version2 && - coid1 === coid2 && - templateId1 == templateId2 && - actingParties1 == actingParties2 && - signatories1 == signatories2 && - stakeholders1 == stakeholders2 && - (keyIsReplayedBy(key1, key2)) && - byKeyIsReplacedBy(version1, byKey1, byKey2) => - loop(rest1, rest2, stack) - case ( - exe1 @ Node.NodeExercises( - targetCoid1, - templateId1, - choiceId1, - consuming1, - actingParties1, - chosenValue1, - stakeholders1, - signatories1, - choiceObservers1, - children1 @ _, - exerciseResult1 @ _, - key1, - byKey1, - version1, - ), - exe2 @ Node.NodeExercises( - targetCoid2, - templateId2, - choiceId2, - consuming2, - actingParties2, - chosenValue2, - stakeholders2, - signatories2, - choiceObservers2, - children2 @ _, - exerciseResult2 @ _, - key2, - byKey2, - version2, - ), - ) - // results are checked after the children - if version1 == version2 && - targetCoid1 === targetCoid2 && - templateId1 == templateId2 && - choiceId1 == choiceId2 && - consuming1 == consuming2 && - actingParties1 == actingParties2 && - valueIsReplayedBy(chosenValue1, chosenValue2) && - stakeholders1 == stakeholders2 && - signatories1 == signatories2 && - choiceObservers1 == choiceObservers2 && - (keyIsReplayedBy(key1, key2)) && - byKeyIsReplacedBy(version1, byKey1, byKey2) => - loop( - children1.iterator.to(LazyList), - children2.iterator.to(LazyList), - ExerciseEntry(Exercise(nid1, exe1, rest1), Exercise(nid2, exe2, rest2)) :: stack, - ) - case ( - Node.NodeLookupByKey(templateId1, key1, result1, version1), - Node.NodeLookupByKey(templateId2, key2, result2, version2), - ) - if version1 == version2 && - templateId1 == templateId2 && - keyIsReplayedBy(key1, key2) && - result1 === result2 => - loop(rest1, rest2, stack) - case ( - Node.NodeRollback( - children1 - ), - Node.NodeRollback( - children2 - ), - ) => - loop( - children1.iterator.to(LazyList), - children2.iterator.to(LazyList), - RollbackEntry(Rollback(nid1, rest1), Rollback(nid2, rest2)) :: stack, - ) - case _ => - Left(ReplayNodeMismatch(recorded, nid1, replayed, nid2)) - } - - case (LazyList(), LazyList()) => - stack match { - case ExerciseEntry(Exercise(nid1, exe1, nids1), Exercise(nid2, exe2, nids2)) :: rest => - if (resultIsReplayedBy(exe1.exerciseResult, exe2.exerciseResult)) - loop(nids1, nids2, rest) - else - Left(ReplayNodeMismatch(recorded, nid1, replayed, nid2)) - case RollbackEntry(Rollback(_, nids1), Rollback(_, nids2)) :: rest => - loop(nids1, nids2, rest) - case Nil => - Right(()) - } - - case (LazyList.cons(nid1, _), LazyList()) => - Left(ReplayedNodeMissing(recorded, nid1, replayed)) - - case (LazyList(), LazyList.cons(nid2, _)) => - Left(RecordedNodeMissing(recorded, replayed, nid2)) - - } - - loop(recorded.roots.iterator.to(LazyList), replayed.roots.iterator.to(LazyList)) - + val replayedN = Normalization.normalizeTx(replayed) + if (recorded == replayedN) { + Right(()) + } else { + Left(ReplayMismatch(recorded, replayedN)) + } } - } object Validation { - def isReplayedBy[Nid, Cid]( recorded: VersionedTransaction[Nid, Cid], replayed: VersionedTransaction[Nid, Cid], - )(implicit ECid: Equal[Cid]): Either[ReplayMismatch[Nid, Cid], Unit] = + ): Either[ReplayMismatch[Nid, Cid], Unit] = new Validation().isReplayedBy(recorded, replayed) - - // package private for test. - private[lf] def valueIsReplayedBy[Cid](recorded: Value[Cid], replayed: Value[Cid])(implicit - ECid: Equal[Cid] - ): Boolean = - new Validation().valueIsReplayedBy(recorded, replayed) - } -sealed abstract class ReplayMismatch[Nid, Cid] extends Product with Serializable { - def recordedTransaction: VersionedTransaction[Nid, Cid] - def replayedTransaction: VersionedTransaction[Nid, Cid] - +final case class ReplayMismatch[Nid, Cid]( + recordedTransaction: VersionedTransaction[Nid, Cid], + replayedTransaction: VersionedTransaction[Nid, Cid], +) extends Product + with Serializable { def message: String = s"recreated and original transaction mismatch $recordedTransaction expected, but $replayedTransaction is recreated" } - -final case class ReplayNodeMismatch[Nid, Cid]( - override val recordedTransaction: VersionedTransaction[Nid, Cid], - recordedNode: Nid, - override val replayedTransaction: VersionedTransaction[Nid, Cid], - replayedNode: Nid, -) extends ReplayMismatch[Nid, Cid] - -final case class RecordedNodeMissing[Nid, Cid]( - override val recordedTransaction: VersionedTransaction[Nid, Cid], - override val replayedTransaction: VersionedTransaction[Nid, Cid], - replayedNode: Nid, -) extends ReplayMismatch[Nid, Cid] - -final case class ReplayedNodeMissing[Nid, Cid]( - override val recordedTransaction: VersionedTransaction[Nid, Cid], - recordedNode: Nid, - override val replayedTransaction: VersionedTransaction[Nid, Cid], -) extends ReplayMismatch[Nid, Cid] diff --git a/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/transaction/TransactionSpec.scala b/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/transaction/TransactionSpec.scala index 232ddf0f8e29..0f11bb0bb49a 100644 --- a/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/transaction/TransactionSpec.scala +++ b/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/transaction/TransactionSpec.scala @@ -197,7 +197,8 @@ class TransactionSpec } yield node "is reflexive" in forAll(genEmptyNode) { n => - isReplayedBy(n, n) shouldBe Right(()) + val tx = Normalization.normalizeTx(genTrans(n)) + Validation.isReplayedBy(tx, tx) shouldBe Right(()) } "fail if version is different" in { diff --git a/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/validation/ValidationSpec.scala b/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/validation/ValidationSpec.scala index 099a978bef2a..f82e1b13a6d4 100644 --- a/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/validation/ValidationSpec.scala +++ b/daml-lf/transaction/src/test/scala/com/digitalasset/daml/lf/validation/ValidationSpec.scala @@ -28,7 +28,7 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC // A 'Tweak[X]' is a family of (small) modifications to a value of type X. // // This test file constructs tweaks for 'VersionedTransaction' (VTX) and classifies them - // as either SIGNIFICANT or INSIGNIFICANT (as reported by `isReplayedBy`). + // as either SIGNIFICANT or INSIGNIFICANT (as reported by 'isReplayedBy'). // // We aim to tweak every field of every ActionNode in a TX. // @@ -186,7 +186,10 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC ) } - private def preTweakedVTXs: Seq[VTX] = flatVTXs ++ nestedVTXs + private def preTweakedVTXs: Seq[VTX] = { + // we ensure the preTweaked txs are properly normalized. + (flatVTXs ++ nestedVTXs).map(Normalization.normalizeTx) + } private def runTweak(tweak: Tweak[VTX]): Seq[(VTX, VTX)] = for { @@ -450,8 +453,7 @@ class ValidationSpec extends AnyFreeSpec with Matchers with TableDrivenPropertyC nodeB <- tweakNode.run(nodeMapA(nid)) } yield { val nodeMapB = nodeMapA + (nid -> nodeB) - val txB = GenTransaction(nodeMapB, roots) - TransactionVersion.asVersionedTransaction(txB) + VersionedTransaction(vtx.version, nodeMapB, roots) } } } diff --git a/ledger/participant-integration-api/src/test/suite/scala/platform/apiserver/services/ApiSubmissionServiceSpec.scala b/ledger/participant-integration-api/src/test/suite/scala/platform/apiserver/services/ApiSubmissionServiceSpec.scala index 574770603b24..e00f0912b44c 100644 --- a/ledger/participant-integration-api/src/test/suite/scala/platform/apiserver/services/ApiSubmissionServiceSpec.scala +++ b/ledger/participant-integration-api/src/test/suite/scala/platform/apiserver/services/ApiSubmissionServiceSpec.scala @@ -30,7 +30,7 @@ import com.daml.lf.engine.{Error => LfError} import com.daml.lf.interpretation.{Error => LfInterpretationError} import com.daml.lf.language.{LookupError, Reference} import com.daml.lf.transaction.test.TransactionBuilder -import com.daml.lf.transaction.{GlobalKey, NodeId, ReplayNodeMismatch} +import com.daml.lf.transaction.{GlobalKey, NodeId, ReplayMismatch} import com.daml.lf.value.Value import com.daml.logging.LoggingContext import com.daml.metrics.Metrics @@ -253,7 +253,7 @@ class ApiSubmissionServiceSpec ) -> Status.ABORTED, ErrorCause.DamlLf( LfError.Validation( - LfError.Validation.ReplayMismatch(ReplayNodeMismatch(null, null, null, null)) + LfError.Validation.ReplayMismatch(ReplayMismatch(null, null)) ) ) -> Status.ABORTED, ErrorCause.DamlLf( diff --git a/ledger/participant-state/kvutils/src/test/suite/scala/com/daml/ledger/participant/state/kvutils/committer/transaction/validation/ModelConformanceValidatorSpec.scala b/ledger/participant-state/kvutils/src/test/suite/scala/com/daml/ledger/participant/state/kvutils/committer/transaction/validation/ModelConformanceValidatorSpec.scala index cf286d37ee8d..279ab8eb9f75 100644 --- a/ledger/participant-state/kvutils/src/test/suite/scala/com/daml/ledger/participant/state/kvutils/committer/transaction/validation/ModelConformanceValidatorSpec.scala +++ b/ledger/participant-state/kvutils/src/test/suite/scala/com/daml/ledger/participant/state/kvutils/committer/transaction/validation/ModelConformanceValidatorSpec.scala @@ -38,7 +38,7 @@ import com.daml.lf.transaction.test.TransactionBuilder import com.daml.lf.transaction.{ GlobalKey, GlobalKeyWithMaintainers, - ReplayedNodeMissing, + ReplayMismatch, SubmittedTransaction, TransactionVersion, } @@ -156,7 +156,7 @@ class ModelConformanceValidatorSpec ).thenReturn( ResultError( LfError.Validation.ReplayMismatch( - ReplayedNodeMissing(aTransaction._1, aTransaction._2, anotherTransaction._1) + ReplayMismatch(aTransaction._1, anotherTransaction._1) ) ) )