Check for duplicate contract IDs in the contract state machine (#17394)

* First stab at detecting double creates in the state machine

* Use subtyping to encode unions of errors instead of Either

* run formatter

* propagate duplicate contract ID error to clients

* remove redundant 'with TransactionError'

* also check fetches for double creation

* add test for create after fetch, found a bug!

* add one transaction test for create after fetch

* use sums instead of subtyping for classifying errors

* crash on double creation in SBuiltin rather than propagating the error as it should never happen

* remove unused parameter

* track all input contract IDs, not just fetches

* remove non-sensical test

* Fix the computation of input contract IDs
This commit is contained in:
Paul Brauner 2023-09-26 13:42:16 +02:00 committed by GitHub
parent 94edbbae2d
commit e8abfeb069
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 574 additions and 236 deletions

View File

@ -17,12 +17,17 @@ import com.daml.lf.speedy.Speedy._
import com.daml.lf.speedy.{SExpr0 => compileTime}
import com.daml.lf.speedy.{SExpr => runTime}
import com.daml.lf.speedy.SValue.{SValue => SV, _}
import com.daml.lf.transaction.TransactionErrors.{
AuthFailureDuringExecution,
DuplicateContractId,
DuplicateContractKey,
}
import com.daml.lf.transaction.{
ContractStateMachine,
GlobalKey,
GlobalKeyWithMaintainers,
Transaction => Tx,
TransactionVersion,
TransactionErrors => TxErr,
}
import com.daml.lf.value.{Value => V}
import com.daml.lf.value.Value.ValueArithmeticError
@ -2023,11 +2028,13 @@ private[lf] object SBuiltin {
}
}
private[speedy] def convTxError(err: Tx.TransactionError): IE = {
private[speedy] def convTxError(err: TxErr.TransactionError): IE = {
err match {
case Tx.AuthFailureDuringExecution(nid, fa) =>
case TxErr.AuthFailureDuringExecutionTxError(AuthFailureDuringExecution(nid, fa)) =>
IE.FailedAuthorization(nid, fa)
case Tx.DuplicateContractKey(key) =>
case TxErr.DuplicateContractIdTxError(DuplicateContractId(contractId)) =>
crash(s"Unexpected duplicate contract ID ${contractId}")
case TxErr.DuplicateContractKeyTxError(DuplicateContractKey(key)) =>
IE.DuplicateContractKey(key)
}
}

View File

@ -16,6 +16,7 @@ import com.daml.lf.transaction.{
NodeId,
SubmittedTransaction => SubmittedTx,
Transaction => Tx,
TransactionErrors => TxErr,
TransactionVersion => TxVersion,
}
import com.daml.lf.value.Value
@ -193,11 +194,11 @@ private[lf] object PartialTransaction {
)
@throws[SError.SErrorDamlException]
private def assertRightKey[X](either: Either[Tx.InconsistentContractKey, X]): X =
private def assertRightKey[X](either: Either[TxErr.InconsistentContractKey, X]): X =
either match {
case Right(value) =>
value
case Left(Tx.InconsistentContractKey(key)) =>
case Left(TxErr.InconsistentContractKey(key)) =>
throw SError.SErrorDamlException(interpretation.Error.InconsistentContractKey(key))
}
@ -339,7 +340,10 @@ private[speedy] case class PartialTransaction(
submissionTime: Time.Timestamp,
contract: ContractInfo,
optLocation: Option[Location],
): Either[(PartialTransaction, Tx.TransactionError), (Value.ContractId, PartialTransaction)] = {
): Either[
(PartialTransaction, TxErr.TransactionError),
(Value.ContractId, PartialTransaction),
] = {
val auth = Authorize(context.info.authorizers)
val actionNodeSeed = context.nextActionChildSeed
val discriminator =
@ -355,7 +359,8 @@ private[speedy] case class PartialTransaction(
actionNodeSeeds = actionNodeSeeds :+ actionNodeSeed,
)
authorizationChecker.authorizeCreate(optLocation, createNode)(auth) match {
case fa :: _ => Left((ptx, Tx.AuthFailureDuringExecution(nid, fa)))
case fa :: _ =>
Left((ptx, TxErr.TransactionError.inject(TxErr.AuthFailureDuringExecution(nid, fa))))
case Nil =>
ptx.contractState.visitCreate(
cid,
@ -365,7 +370,7 @@ private[speedy] case class PartialTransaction(
val nextPtx = ptx.copy(contractState = next)
Right((cid, nextPtx))
case Left(duplicate) =>
Left((ptx, duplicate))
Left((ptx, TxErr.TransactionError.from(duplicate)))
}
}
}
@ -376,7 +381,7 @@ private[speedy] case class PartialTransaction(
optLocation: Option[Location],
byKey: Boolean,
version: TxVersion,
): Either[Tx.TransactionError, PartialTransaction] = {
): Either[TxErr.TransactionError, PartialTransaction] = {
val contextActors = context.info.authorizers
val actingParties = contextActors intersect contract.stakeholders
val auth = Authorize(context.info.authorizers)
@ -401,7 +406,8 @@ private[speedy] case class PartialTransaction(
)
)
authorizationChecker.authorizeFetch(optLocation, node)(auth) match {
case fa :: _ => Left(Tx.AuthFailureDuringExecution(nid, fa))
case fa :: _ =>
Left(TxErr.TransactionError.inject(TxErr.AuthFailureDuringExecution(nid, fa)))
case Nil =>
Right(insertLeafNode(node, version, optLocation, newContractState))
}
@ -413,7 +419,7 @@ private[speedy] case class PartialTransaction(
key: CachedKey,
result: Option[Value.ContractId],
keyVersion: TxVersion,
): Either[Tx.TransactionError, PartialTransaction] = {
): Either[TxErr.TransactionError, PartialTransaction] = {
val auth = Authorize(context.info.authorizers)
val nid = NodeId(nextNodeIdx)
val node = Node.LookupByKey(
@ -428,7 +434,8 @@ private[speedy] case class PartialTransaction(
val newContractState =
assertRightKey(contractState.visitLookup(key.globalKey, keyInput.toKeyMapping, result))
authorizationChecker.authorizeLookupByKey(optLocation, node)(auth) match {
case fa :: _ => Left(Tx.AuthFailureDuringExecution(nid, fa))
case fa :: _ =>
Left(TxErr.TransactionError.inject(TxErr.AuthFailureDuringExecution(nid, fa)))
case Nil =>
Right(insertLeafNode(node, keyVersion, optLocation, newContractState))
}
@ -451,7 +458,7 @@ private[speedy] case class PartialTransaction(
byKey: Boolean,
chosenValue: Value,
version: TxVersion,
): Either[Tx.TransactionError, PartialTransaction] = {
): Either[TxErr.TransactionError, PartialTransaction] = {
val auth = Authorize(context.info.authorizers)
val nid = NodeId(nextNodeIdx)
val ec =
@ -488,7 +495,8 @@ private[speedy] case class PartialTransaction(
)
)
authorizationChecker.authorizeExercise(optLocation, makeExNode(ec))(auth) match {
case fa :: _ => Left(Tx.AuthFailureDuringExecution(nid, fa))
case fa :: _ =>
Left(TxErr.TransactionError.inject(TxErr.AuthFailureDuringExecution(nid, fa)))
case Nil =>
Right(
copy(

View File

@ -4,13 +4,13 @@
package com.daml.lf
package transaction
import com.daml.lf.transaction.Transaction.{
import com.daml.lf.transaction.Transaction.{KeyCreate, KeyInput, NegativeKeyLookup}
import com.daml.lf.transaction.TransactionErrors.{
CreateError,
DuplicateContractId,
DuplicateContractKey,
InconsistentContractKey,
KeyCreate,
KeyInput,
KeyInputError,
NegativeKeyLookup,
}
import com.daml.lf.value.Value
import com.daml.lf.value.Value.ContractId
@ -81,6 +81,7 @@ object ContractStateMachine {
*/
case class State[Nid] private[lf] (
locallyCreated: Set[ContractId],
inputContractIds: Set[ContractId],
globalKeyInputs: Map[GlobalKey, KeyInput],
activeState: ContractStateMachine.ActiveLedgerState[Nid],
rollbackStack: List[ContractStateMachine.ActiveLedgerState[Nid]],
@ -121,38 +122,41 @@ object ContractStateMachine {
/** Visit a create node */
def handleCreate(node: Node.Create): Either[KeyInputError, State[Nid]] =
visitCreate(node.coid, node.gkeyOpt).left.map(Right(_))
visitCreate(node.coid, node.gkeyOpt).left.map(KeyInputError.from)
private[lf] def visitCreate(
contractId: ContractId,
mbKey: Option[GlobalKey],
): Either[DuplicateContractKey, State[Nid]] = {
val me =
this.copy(
locallyCreated = locallyCreated + contractId,
activeState = this.activeState
.copy(locallyCreatedThisTimeline =
this.activeState.locallyCreatedThisTimeline + contractId
),
)
// if we have a contract key being added, include it in the list of
// active keys
mbKey match {
case None => Right(me)
case Some(gk) =>
val conflict = lookupActiveKey(gk).exists(_ != KeyInactive)
val newKeyInputs =
if (globalKeyInputs.contains(gk)) globalKeyInputs
else globalKeyInputs.updated(gk, KeyCreate)
Either.cond(
!conflict || mode == ContractKeyUniquenessMode.Off,
me.copy(
activeState = me.activeState.createKey(gk, contractId),
globalKeyInputs = newKeyInputs,
),
DuplicateContractKey(gk),
): Either[CreateError, State[Nid]] = {
if (locallyCreated.union(inputContractIds).contains(contractId)) {
Left(CreateError.inject(DuplicateContractId(contractId)))
} else {
val me =
this.copy(
locallyCreated = locallyCreated + contractId,
activeState = this.activeState
.copy(locallyCreatedThisTimeline =
this.activeState.locallyCreatedThisTimeline + contractId
),
)
// if we have a contract key being added, include it in the list of
// active keys
mbKey match {
case None => Right(me)
case Some(gk) =>
val conflict = lookupActiveKey(gk).exists(_ != KeyInactive)
val newKeyInputs =
if (globalKeyInputs.contains(gk)) globalKeyInputs
else globalKeyInputs.updated(gk, KeyCreate)
Either.cond(
!conflict || mode == ContractKeyUniquenessMode.Off,
me.copy(
activeState = me.activeState.createKey(gk, contractId),
globalKeyInputs = newKeyInputs,
),
CreateError.inject(DuplicateContractKey(gk)),
)
}
}
}
@ -163,8 +167,7 @@ object ContractStateMachine {
exe.gkeyOpt,
exe.byKey,
exe.consuming,
).left
.map(Left(_))
).left.map(KeyInputError.inject)
/** Omits the key lookup that are done in [[com.daml.lf.speedy.Compiler.compileChoiceByKey]] for by-bey nodes,
* which translates to a [[resolveKey]] below.
@ -177,12 +180,13 @@ object ContractStateMachine {
byKey: Boolean,
consuming: Boolean,
): Either[InconsistentContractKey, State[Nid]] = {
val state = witnessContractId(targetId)
for {
state <-
if (byKey || mode == ContractKeyUniquenessMode.Strict)
assertKeyMapping(targetId, mbKey)
state.assertKeyMapping(targetId, mbKey)
else
Right(this)
Right(state)
} yield {
if (consuming) {
val consumedState = state.activeState.consume(targetId, nodeId)
@ -200,7 +204,7 @@ object ContractStateMachine {
throw new UnsupportedOperationException(
"handleLookup can only be used if all key nodes are considered"
)
visitLookup(lookup.gkey, lookup.result, lookup.result).left.map(Left(_))
visitLookup(lookup.gkey, lookup.result, lookup.result).left.map(KeyInputError.inject)
}
/** Must be used to handle lookups iff in [[com.daml.lf.transaction.ContractKeyUniquenessMode.Off]] mode
@ -222,7 +226,7 @@ object ContractStateMachine {
throw new UnsupportedOperationException(
"handleLookupWith can only be used if only by-key nodes are considered"
)
visitLookup(lookup.gkey, keyInput, lookup.result).left.map(Left(_))
visitLookup(lookup.gkey, keyInput, lookup.result).left.map(KeyInputError.inject)
}
private[lf] def visitLookup(
@ -230,7 +234,11 @@ object ContractStateMachine {
keyInput: Option[ContractId],
keyResolution: Option[ContractId],
): Either[InconsistentContractKey, State[Nid]] = {
val (keyMapping, next) = resolveKey(gk) match {
val state = keyInput match {
case Some(contractId) => witnessContractId(contractId)
case None => this
}
val (keyMapping, next) = state.resolveKey(gk) match {
case Right(result) => result
case Left(handle) => handle(keyInput)
}
@ -270,17 +278,23 @@ object ContractStateMachine {
}
def handleFetch(node: Node.Fetch): Either[KeyInputError, State[Nid]] =
visitFetch(node.coid, node.gkeyOpt, node.byKey).left.map(Left(_))
visitFetch(node.coid, node.gkeyOpt, node.byKey).left.map(KeyInputError.inject)
private[lf] def visitFetch(
contractId: ContractId,
mbKey: Option[GlobalKey],
byKey: Boolean,
): Either[InconsistentContractKey, State[Nid]] =
): Either[InconsistentContractKey, State[Nid]] = {
val state = witnessContractId(contractId)
if (byKey || mode == ContractKeyUniquenessMode.Strict)
assertKeyMapping(contractId, mbKey)
state.assertKeyMapping(contractId, mbKey)
else
Right(this)
Right(state)
}
private[lf] def witnessContractId(contractId: ContractId): State[Nid] =
if (locallyCreated.contains(contractId)) this
else this.copy(inputContractIds = inputContractIds + contractId)
private[lf] def assertKeyMapping(
cid: Value.ContractId,
@ -371,25 +385,32 @@ object ContractStateMachine {
)
// We want consistent key lookups within an action in any contract key mode.
def consistentGlobalKeyInputs: Either[KeyInputError, Unit] =
substate.globalKeyInputs
.find {
case (key, KeyCreate) =>
lookupActiveKey(key).exists(
_ != KeyInactive
) && mode == ContractKeyUniquenessMode.Strict
case (key, NegativeKeyLookup) => lookupActiveKey(key).exists(_ != KeyInactive)
case (key, Transaction.KeyActive(cid)) =>
lookupActiveKey(key).exists(_ != KeyActive(cid))
case _ => false
} match {
case Some((key, KeyCreate)) => Left[KeyInputError, Unit](Right(DuplicateContractKey(key)))
case Some((key, NegativeKeyLookup)) =>
Left[KeyInputError, Unit](Left(InconsistentContractKey(key)))
case Some((key, Transaction.KeyActive(_))) =>
Left[KeyInputError, Unit](Left(InconsistentContractKey(key)))
case _ => Right[KeyInputError, Unit](())
def consistentGlobalKeyInputs: Either[KeyInputError, Unit] = {
substate.locallyCreated.find(locallyCreated.union(inputContractIds).contains) match {
case Some(contractId) =>
Left[KeyInputError, Unit](KeyInputError.inject(DuplicateContractId(contractId)))
case None =>
substate.globalKeyInputs
.find {
case (key, KeyCreate) =>
lookupActiveKey(key).exists(
_ != KeyInactive
) && mode == ContractKeyUniquenessMode.Strict
case (key, NegativeKeyLookup) => lookupActiveKey(key).exists(_ != KeyInactive)
case (key, Transaction.KeyActive(cid)) =>
lookupActiveKey(key).exists(_ != KeyActive(cid))
case _ => false
} match {
case Some((key, KeyCreate)) =>
Left[KeyInputError, Unit](KeyInputError.inject(DuplicateContractKey(key)))
case Some((key, NegativeKeyLookup)) =>
Left[KeyInputError, Unit](KeyInputError.inject(InconsistentContractKey(key)))
case Some((key, Transaction.KeyActive(_))) =>
Left[KeyInputError, Unit](KeyInputError.inject(InconsistentContractKey(key)))
case _ => Right[KeyInputError, Unit](())
}
}
}
for {
_ <- consistentGlobalKeyInputs
@ -426,6 +447,8 @@ object ContractStateMachine {
this.copy(
locallyCreated = this.locallyCreated.union(substate.locallyCreated),
inputContractIds =
this.inputContractIds.union(substate.inputContractIds.diff(this.locallyCreated)),
globalKeyInputs = globalKeyInputs,
activeState = next,
)
@ -447,6 +470,7 @@ object ContractStateMachine {
object State {
def empty[Nid](mode: ContractKeyUniquenessMode): State[Nid] = new State(
Set.empty,
Set.empty,
Map.empty,
ContractStateMachine.ActiveLedgerState.empty,

View File

@ -6,10 +6,10 @@ package transaction
import com.daml.lf.data.Ref._
import com.daml.lf.data._
import com.daml.lf.ledger.FailedAuthorization
import com.daml.lf.value.Value
import com.daml.lf.value.Value.ContractId
import com.daml.lf.transaction.ContractStateMachine.KeyMapping
import com.daml.lf.transaction.TransactionErrors.KeyInputError
import scala.annotation.tailrec
import scala.collection.immutable.HashMap
@ -249,7 +249,7 @@ final case class Transaction(
sealed abstract class HasTxNodes {
import Transaction.{KeyInput, KeyInputError, ChildrenRecursion}
import Transaction.{KeyInput, ChildrenRecursion}
def nodes: Map[NodeId, Node]
@ -701,34 +701,6 @@ object Transaction {
): Either[String, CommittedTransaction] =
submittedTransaction.suffixCid(f).map(CommittedTransaction(_))
/** Errors that can happen during building transactions. */
sealed abstract class TransactionError extends Product with Serializable
/** Signals that within the transaction we got to a point where
* two contracts with the same key were active.
*
* Note that speedy only detects duplicate key collisions
* if both contracts are used in the transaction in by-key operations
* meaning lookup, fetch or exercise-by-key or local creates.
*
* Two notable cases that will never produce duplicate key errors
* is a standalone create or a create and a fetch (but not fetch-by-key)
* with the same key.
*
* For ledger implementors this means that (for contract key uniqueness)
* it is sufficient to only look at the inputs and the outputs of the
* transaction while leaving all internal checks within the transaction
* to the engine.
*/
final case class DuplicateContractKey(
key: GlobalKey
) extends TransactionError
final case class AuthFailureDuringExecution(
nid: NodeId,
fa: FailedAuthorization,
) extends TransactionError
/** The state of a key at the beginning of the transaction.
*/
sealed trait KeyInput extends Product with Serializable {
@ -758,21 +730,9 @@ object Transaction {
override def isActive: Boolean = true
}
/** An exercise, fetch or lookupByKey failed because the mapping of key -> contract id
* was inconsistent with earlier nodes (in execution order). This can happened in case
* of a race condition between the contract and the contract keys queried to the ledger
* during an interpretation.
*/
final case class InconsistentContractKey(key: GlobalKey)
/** contractKeyInputs failed to produce an input due to an error for the given key.
*/
type KeyInputError = Either[InconsistentContractKey, DuplicateContractKey]
sealed abstract class ChildrenRecursion
object ChildrenRecursion {
case object DoRecurse extends ChildrenRecursion
case object DoNotRecurse extends ChildrenRecursion
}
}

View File

@ -0,0 +1,157 @@
// Copyright (c) 2023 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.ledger.FailedAuthorization
import com.daml.lf.value.Value.ContractId
/** Defines the errors raised by [[ContractStateMachine]] and its clients:
* - [[DuplicateContractId]]
* - [[DuplicateContractKey]]
* - [[InconsistentContractKey]]
* - [[AuthFailureDuringExecution]]
* , and classifies them into three overlapping categories:
* - [[CreateError]]
* - [[KeyInputError]]
* - [[TransactionError]]
*/
object TransactionErrors {
/** Signals that the transaction tried to create two contracts with the same
* contract ID or tried to create a contract whose contract ID has been
* previously successfully fetched.
*/
final case class DuplicateContractId(
contractId: ContractId
) extends Serializable
with Product
/** Signals that within the transaction we got to a point where
* two contracts with the same key were active.
*
* Note that speedy only detects duplicate key collisions
* if both contracts are used in the transaction in by-key operations
* meaning lookup, fetch or exercise-by-key or local creates.
*
* Two notable cases that will never produce duplicate key errors
* is a standalone create or a create and a fetch (but not fetch-by-key)
* with the same key.
*
* For ledger implementors this means that (for contract key uniqueness)
* it is sufficient to only look at the inputs and the outputs of the
* transaction while leaving all internal checks within the transaction
* to the engine.
*/
final case class DuplicateContractKey(
key: GlobalKey
) extends Serializable
with Product
/** An exercise, fetch or lookupByKey failed because the mapping of key -> contract id
* was inconsistent with earlier nodes (in execution order). This can happened in case
* of a race condition between the contract and the contract keys queried to the ledger
* during an interpretation.
*/
final case class InconsistentContractKey(key: GlobalKey) extends Serializable with Product
final case class AuthFailureDuringExecution(
nid: NodeId,
fa: FailedAuthorization,
) extends Serializable
with Product
/** Errors raised when building transactions with [[com.daml.lf.speedy.PartialTransaction]]:
* - [[DuplicateContractId]]
* - [[DuplicateContractKey]]
* - [[AuthFailureDuringExecution]]
*/
sealed trait TransactionError extends Serializable with Product
final case class DuplicateContractIdTxError(
duplicateContractId: DuplicateContractId
) extends TransactionError
final case class DuplicateContractKeyTxError(
duplicateContractKey: DuplicateContractKey
) extends TransactionError
final case class AuthFailureDuringExecutionTxError(
authFailureDuringExecution: AuthFailureDuringExecution
) extends TransactionError
object TransactionError {
def inject(error: DuplicateContractId): TransactionError =
DuplicateContractIdTxError(error)
def inject(error: DuplicateContractKey): TransactionError =
DuplicateContractKeyTxError(error)
def inject(error: AuthFailureDuringExecution): TransactionError =
AuthFailureDuringExecutionTxError(error)
def from(error: CreateError): TransactionError = error match {
case DuplicateContractIdCreateError(e) => inject(e)
case DuplicateContractKeyCreateError(e) => inject(e)
}
}
/** The errors returned by [[ContractStateMachine.State.handleNode]] and, as a consequence,
* [[HasTxNodes.contractKeyInputs]] (hence the name):
* - [[DuplicateContractId]]
* - [[DuplicateContractKey]]
* - [[InconsistentContractKey]]
*/
sealed trait KeyInputError extends Serializable with Product
final case class DuplicateContractIdKIError(
duplicateContractId: DuplicateContractId
) extends KeyInputError
final case class DuplicateContractKeyKIError(
duplicateContractKey: DuplicateContractKey
) extends KeyInputError
final case class InconsistentContractKeyKIError(
inconsistentContractKey: InconsistentContractKey
) extends KeyInputError
object KeyInputError {
def inject(error: DuplicateContractId): KeyInputError =
DuplicateContractIdKIError(error)
def inject(error: DuplicateContractKey): KeyInputError =
DuplicateContractKeyKIError(error)
def inject(error: InconsistentContractKey): KeyInputError =
InconsistentContractKeyKIError(error)
def from(error: CreateError): KeyInputError = error match {
case DuplicateContractIdCreateError(e) => inject(e)
case DuplicateContractKeyCreateError(e) => inject(e)
}
}
/** The errors returned by [[ContractStateMachine.State.visitCreate]]:
* - [[DuplicateContractId]]
* - [[DuplicateContractKey]]
*/
sealed trait CreateError extends Serializable with Product
final case class DuplicateContractIdCreateError(
duplicateContractId: DuplicateContractId
) extends CreateError
final case class DuplicateContractKeyCreateError(
duplicateContractKey: DuplicateContractKey
) extends CreateError
object CreateError {
def inject(error: DuplicateContractId): CreateError =
DuplicateContractIdCreateError(error)
def inject(error: DuplicateContractKey): CreateError =
DuplicateContractKeyCreateError(error)
}
}

View File

@ -16,13 +16,16 @@ import com.daml.lf.transaction.ContractStateMachine.{
import com.daml.lf.transaction.ContractStateMachineSpec._
import com.daml.lf.transaction.Transaction.{
ChildrenRecursion,
DuplicateContractKey,
InconsistentContractKey,
KeyCreate,
KeyInput,
KeyInputError,
NegativeKeyLookup,
}
import com.daml.lf.transaction.TransactionErrors.{
DuplicateContractId,
DuplicateContractKey,
InconsistentContractKey,
KeyInputError,
}
import com.daml.lf.transaction.test.{NodeIdTransactionBuilder, TestNodeBuilder}
import com.daml.lf.transaction.test.TransactionBuilder.Implicits.{
defaultPackageId,
@ -145,10 +148,13 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
)
def inconsistentContractKey[X](key: GlobalKey): Left[KeyInputError, X] =
Left(Left(InconsistentContractKey(key)))
Left(KeyInputError.inject(InconsistentContractKey(key)))
def duplicateContractKey[X](key: GlobalKey): Left[KeyInputError, X] =
Left(Right(DuplicateContractKey(key)))
Left(KeyInputError.inject(DuplicateContractKey(key)))
def duplicateContractId[X](contractId: ContractId): Left[KeyInputError, X] =
Left(KeyInputError.inject(DuplicateContractId(contractId)))
def createRbExLbkLbk: TestCase = {
// [ Create c1 (key=k1), Rollback [ Exe c1 [ LBK k1 -> None ]], LBK k1 -> c1 ]
@ -161,8 +167,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkLookupByKey("key1", Some(1)))
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> KeyCreate) ->
ActiveLedgerState[Unit](Set(1), Map.empty, Map(gkey("key1") -> 1))
(
Map(gkey("key1") -> KeyCreate),
ActiveLedgerState[Unit](Set(1), Map.empty, Map(gkey("key1") -> 1)),
Set.empty[ContractId],
)
)
TestCase(
"Create|Rb-Ex-LBK|LBK",
@ -191,12 +200,15 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkLookupByKey("key1", None), exercise0Nid)
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> Transaction.KeyActive(cid(1))) ->
(
Map(gkey("key1") -> Transaction.KeyActive(cid(1))),
ActiveLedgerState(
Set.empty,
Map(cid(0) -> (), cid(1) -> ()),
Map.empty,
)
),
Set(cid(0), cid(1)),
)
)
TestCase(
"multiple rollback",
@ -225,8 +237,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkExercise(1, consuming = true, "key1", byKey = true))
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> Transaction.KeyActive(1), gkey("key2") -> Transaction.KeyActive(2)) ->
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty)
(
Map(gkey("key1") -> Transaction.KeyActive(1), gkey("key2") -> Transaction.KeyActive(2)),
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty),
Set(cid(1), cid(2)),
)
)
TestCase(
"nested rollback",
@ -254,10 +269,14 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
builder.add(mkCreate(3, "key"), exerciseNid)
val tx = builder.build()
val expected: TestResult = Right(
Map(gkey("key") -> Transaction.KeyActive(2)) -> ActiveLedgerState(
Set(3),
Map(cid(1) -> (), cid(2) -> ()),
Map(gkey("key") -> cid(3)),
(
Map(gkey("key") -> Transaction.KeyActive(2)),
ActiveLedgerState(
Set(3),
Map(cid(1) -> (), cid(2) -> ()),
Map(gkey("key") -> cid(3)),
),
Set(cid(1), cid(2)),
)
)
TestCase(
@ -283,8 +302,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
// Custom resolver for visibility restriction due to divulgence
val resolver = Map(gkey("key1") -> None)
val expected = Right(
Map(gkey("key1") -> KeyCreate) ->
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty)
(
Map(gkey("key1") -> KeyCreate),
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty),
Set(cid(1), cid(2)),
)
)
TestCase(
"RbExeCreateLbkDivulged",
@ -307,8 +329,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkFetch(2, "key1", byKey = true), exercise1Nid)
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> Transaction.KeyActive(2)) ->
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty)
(
Map(gkey("key1") -> Transaction.KeyActive(2)),
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty),
Set(cid(1), cid(2)),
)
)
TestCase(
"RbExeCreateFbk",
@ -322,7 +347,96 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
}
def createAfterFetch: TestCase = {
// [ Fetch c1, Create c1 ]
val builder = new TxBuilder()
val _ = builder.add(mkFetch(1, "key1", byKey = false))
val _ = builder.add(mkCreate(1, "key2"))
val tx = builder.build()
val expected = duplicateContractId(1)
TestCase(
"CreateAfterFetch",
tx,
Map(
ContractKeyUniquenessMode.Strict -> expected,
ContractKeyUniquenessMode.Off -> expected,
),
)
}
def createAfterLookupByKey: TestCase = {
// [ LBK key c1, Create c1 ]
val builder = new TxBuilder()
val _ = builder.add(mkLookupByKey("key1", Some(1)))
val _ = builder.add(mkCreate(1, "key2"))
val tx = builder.build()
val expected = duplicateContractId(1)
TestCase(
"CreateAfterLookupByKey",
tx,
Map(
ContractKeyUniquenessMode.Strict -> expected,
ContractKeyUniquenessMode.Off -> expected,
),
)
}
def createAfterConsumingExercise: TestCase = {
// [ Exe c1, Create c1 ]
val builder = new TxBuilder()
val _ = builder.add(mkExercise(1, consuming = true))
val _ = builder.add(mkCreate(1, "key"))
val tx = builder.build()
val expected = duplicateContractId(1)
TestCase(
"CreateAfterConsumingExercise",
tx,
Map(
ContractKeyUniquenessMode.Strict -> expected,
ContractKeyUniquenessMode.Off -> expected,
),
)
}
def createAfterNonConsumingExercise: TestCase = {
// [ Exe c1, Create c1 ]
val builder = new TxBuilder()
val _ = builder.add(mkExercise(1, consuming = false))
val _ = builder.add(mkCreate(1, "key"))
val tx = builder.build()
val expected = duplicateContractId(1)
TestCase(
"CreateAfterNonConsumingExercise",
tx,
Map(
ContractKeyUniquenessMode.Strict -> expected,
ContractKeyUniquenessMode.Off -> expected,
),
)
}
def doubleCreate: TestCase = {
// [ Create c1, Create c1 ]
val createNode = mkCreate(1)
// We can't use TxBuilder to build this transaction because the builder ensures the unicity of
// of contract IDs.
val tx = VersionedTransaction(
txVersion,
nodes = Map(NodeId(0) -> createNode, NodeId(1) -> createNode),
roots = ImmArray(NodeId(0), NodeId(1)),
)
val expected = duplicateContractId(1)
TestCase(
"DoubleCreate",
tx,
Map(
ContractKeyUniquenessMode.Strict -> expected,
ContractKeyUniquenessMode.Off -> expected,
),
)
}
def doubleCreateWithKey: TestCase = {
// [ ExeN c1 [ Create c2 (key=k1), Create c3 (key=k1) ] ]
val builder = new TxBuilder()
val exerciseNid = builder.add(mkExercise(1, consuming = false))
@ -330,17 +444,20 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkCreate(3, "key1"), exerciseNid)
val tx = builder.build()
val expectedOff = Right(
Map(gkey("key1") -> KeyCreate) ->
(
Map(gkey("key1") -> KeyCreate),
ActiveLedgerState[Unit](
Set(2, 3),
Map.empty,
Map(
gkey("key1") -> cid(3) // Latest create wins
),
)
),
Set(cid(1)),
)
)
TestCase(
"DoubleCreate",
"DoubleCreateWithKey",
tx,
Map(
ContractKeyUniquenessMode.Strict -> duplicateContractKey(gkey("key1")),
@ -357,8 +474,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkLookupByKey("key1", None), exerciseNid)
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> NegativeKeyLookup) ->
ActiveLedgerState[Unit](Set.empty, Map.empty, Map.empty)
(
Map(gkey("key1") -> NegativeKeyLookup),
ActiveLedgerState[Unit](Set.empty, Map.empty, Map.empty),
Set(cid(1)),
)
)
TestCase(
"DivulgedLookup",
@ -381,8 +501,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkFetch(3, "key1"), exerciseNid)
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> Transaction.KeyActive(2)) ->
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty)
(
Map(gkey("key1") -> Transaction.KeyActive(2)),
ActiveLedgerState(Set.empty, Map(cid(1) -> ()), Map.empty),
Set(cid(1), cid(2), cid(3)),
)
)
TestCase(
"FetchByKey-then-Fetch",
@ -404,8 +527,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkLookupByKey("key1", Some(2)), exercise2Nid)
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> Transaction.KeyActive(2)) ->
ActiveLedgerState(Set.empty, Map(cid(3) -> ()), Map.empty)
(
Map(gkey("key1") -> Transaction.KeyActive(2)),
ActiveLedgerState(Set.empty, Map(cid(3) -> ()), Map.empty),
Set(cid(1), cid(2), cid(3)),
)
)
TestCase(
"Archive other contract with key",
@ -426,8 +552,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkCreate(3, "key1"))
val tx = builder.build()
val expected = Right(
Map(gkey("key1") -> KeyCreate) ->
ActiveLedgerState[Unit](Set(3), Map.empty, Map(gkey("key1") -> cid(3)))
(
Map(gkey("key1") -> KeyCreate),
ActiveLedgerState[Unit](Set(3), Map.empty, Map(gkey("key1") -> cid(3))),
Set(cid(1)),
)
)
TestCase(
"CreateAfterRbExercise",
@ -474,12 +603,15 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
val _ = builder.add(mkCreate(5, "key2"), exerciseNid)
val tx = builder.build()
val expectedOff = Right(
Map(gkey("key1") -> KeyCreate, gkey("key2") -> KeyCreate) ->
(
Map(gkey("key1") -> KeyCreate, gkey("key2") -> KeyCreate),
ActiveLedgerState[Unit](
Set(1, 3, 4, 5),
Map.empty,
Map(gkey("key1") -> cid(4), gkey("key2") -> cid(5)),
)
),
Set(cid(2)),
)
)
TestCase(
"differing cause 2",
@ -519,8 +651,11 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
builder.add(mkCreate(1), rollbackNid)
val tx = builder.build()
val expected = Right(
Map[GlobalKey, KeyInput]() ->
ActiveLedgerState(Set.empty, Map(cid(0) -> ()), Map.empty)
(
Map[GlobalKey, KeyInput](),
ActiveLedgerState(Set.empty, Map(cid(0) -> ()), Map.empty),
Set(cid(0)),
)
)
TestCase(
"rbCreate",
@ -543,7 +678,12 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
archiveRbLookupCreate,
rbExeCreateLbkDivulged,
rbExeCreateFbk,
createAfterFetch,
createAfterLookupByKey,
createAfterConsumingExercise,
createAfterNonConsumingExercise,
doubleCreate,
doubleCreateWithKey,
divulgedLookup,
rbFbkFetch,
archiveOtherKeyContract,
@ -572,13 +712,16 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
(result, expectedResult) match {
case (Left(err1), Left(err2)) => err1 shouldBe err2
case (Right(state), Right((gkI, activeState))) =>
case (Right(state), Right((gkI, activeState, contractsI))) =>
withClue("global key inputs") {
state.globalKeyInputs shouldBe gkI
}
withClue("active state") {
state.activeState shouldBe activeState
}
withClue("input contract IDs") {
state.inputContractIds shouldBe contractsI
}
state.rollbackStack shouldBe List.empty
case _ => fail(s"$result was not equal to $expectedResult")
}
@ -661,7 +804,7 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
root: NodeId,
resolver: KeyResolver,
state: ContractStateMachine.State[Unit],
): Either[Transaction.KeyInputError, ContractStateMachine.State[Unit]] = {
): Either[KeyInputError, ContractStateMachine.State[Unit]] = {
val node = nodes(root)
for {
next <- node match {
@ -691,7 +834,7 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
roots: Seq[NodeId],
resolver: KeyResolver,
state: ContractStateMachine.State[Unit],
): Either[Transaction.KeyInputError, ContractStateMachine.State[Unit]] = {
): Either[KeyInputError, ContractStateMachine.State[Unit]] = {
roots match {
case Seq() => Right(state)
case root +: tail =>
@ -716,7 +859,7 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
case (Left(_), Left(_)) =>
// We can't really make sure that we get the same errors.
// There may be multiple key conflicts and advancing non-deterministically picks one of them
case _ => fail(s"$directVisit was knot equal to $advanced")
case _ => fail(s"$directVisit was not equal to $advanced")
}
}
directVisit.flatMap(next => visitSubtrees(nodes, tail, resolver, next))
@ -725,7 +868,8 @@ class ContractStateMachineSpec extends AnyWordSpec with Matchers with TableDrive
}
object ContractStateMachineSpec {
type TestResult = Either[KeyInputError, (Map[GlobalKey, KeyInput], ActiveLedgerState[Unit])]
type TestResult =
Either[KeyInputError, (Map[GlobalKey, KeyInput], ActiveLedgerState[Unit], Set[ContractId])]
case class TestCase(
name: String,
transaction: HasTxNodes,

View File

@ -15,7 +15,15 @@ import com.daml.lf.transaction.Transaction.{
NotWellFormedError,
OrphanedNode,
}
import com.daml.lf.transaction.test.{NodeIdTransactionBuilder, TransactionBuilder, TestNodeBuilder}
import com.daml.lf.transaction.TransactionErrors.{
DuplicateContractId,
DuplicateContractIdKIError,
DuplicateContractKey,
DuplicateContractKeyKIError,
InconsistentContractKey,
KeyInputError,
}
import com.daml.lf.transaction.test.{NodeIdTransactionBuilder, TestNodeBuilder, TransactionBuilder}
import com.daml.lf.value.{Value => V}
import com.daml.lf.value.test.ValueGenerators.danglingRefGenNode
import org.scalacheck.Gen
@ -383,21 +391,21 @@ class TransactionSpec
val dummyBuilder = new TxBuilder()
val parties = List("Alice")
def keyValue(s: String) = V.ValueText(s)
def globalKey(s: V.ContractId) = GlobalKey.assertBuild("Mod:T", keyValue(s.coid))
def create(s: V.ContractId) = dummyBuilder
def globalKey(k: String) = GlobalKey.assertBuild("Mod:T", keyValue(k))
def create(s: V.ContractId, k: String) = dummyBuilder
.create(
id = s,
templateId = "Mod:T",
argument = V.ValueUnit,
signatories = parties,
observers = parties,
key = CreateKey.SignatoryMaintainerKey(keyValue(s.coid)),
key = CreateKey.SignatoryMaintainerKey(keyValue(k)),
)
def exe(s: V.ContractId, consuming: Boolean, byKey: Boolean) =
def exe(s: V.ContractId, k: String, consuming: Boolean, byKey: Boolean) =
dummyBuilder
.exercise(
contract = create(s),
contract = create(s, k),
choice = "Choice",
actingParties = parties.toSet,
consuming = consuming,
@ -405,69 +413,77 @@ class TransactionSpec
byKey = byKey,
)
def fetch(s: V.ContractId, byKey: Boolean) =
dummyBuilder.fetch(contract = create(s), byKey = byKey)
def fetch(s: V.ContractId, k: String, byKey: Boolean) =
dummyBuilder.fetch(contract = create(s, k), byKey = byKey)
def lookup(s: V.ContractId, found: Boolean) =
dummyBuilder.lookupByKey(contract = create(s), found = found)
def lookup(s: V.ContractId, k: String, found: Boolean) =
dummyBuilder.lookupByKey(contract = create(s, k), found = found)
"return None for create" in {
val builder = new TxBuilder()
val createNode = create(cid("#0"))
val createNode = create(cid("#0"), "k0")
builder.add(createNode)
builder.build().contractKeyInputs shouldBe Right(Map(globalKey(cid("#0")) -> KeyCreate))
builder.build().contractKeyInputs shouldBe Right(Map(globalKey("k0") -> KeyCreate))
}
"return Some(_) for fetch and fetch-by-key" in {
val builder = new TxBuilder()
val fetchNode0 = fetch(cid("#0"), byKey = false)
val fetchNode1 = fetch(cid("#1"), byKey = true)
val fetchNode0 = fetch(cid("#0"), "k0", byKey = false)
val fetchNode1 = fetch(cid("#1"), "k1", byKey = true)
builder.add(fetchNode0)
builder.add(fetchNode1)
builder.build().contractKeyInputs shouldBe Right(
Map(
globalKey(cid("#0")) -> KeyActive(cid("#0")),
globalKey(cid("#1")) -> KeyActive(cid("#1")),
globalKey("k0") -> KeyActive(cid("#0")),
globalKey("k1") -> KeyActive(cid("#1")),
)
)
}
"return Some(_) for consuming/non-consuming exercise and exercise-by-key" in {
val builder = new TxBuilder()
val exe0 = exe(cid("#0"), consuming = false, byKey = false)
val exe1 = exe(cid("#1"), consuming = true, byKey = false)
val exe2 = exe(cid("#2"), consuming = false, byKey = true)
val exe3 = exe(cid("#3"), consuming = true, byKey = true)
val exe0 = exe(cid("#0"), "k0", consuming = false, byKey = false)
val exe1 = exe(cid("#1"), "k1", consuming = true, byKey = false)
val exe2 = exe(cid("#2"), "k2", consuming = false, byKey = true)
val exe3 = exe(cid("#3"), "k3", consuming = true, byKey = true)
builder.add(exe0)
builder.add(exe1)
builder.add(exe2)
builder.add(exe3)
builder.build().contractKeyInputs shouldBe Right(
Seq(cid("#0"), cid("#1"), cid("#2"), cid("#3")).map(s => globalKey(s) -> KeyActive(s)).toMap
Map(
globalKey("k0") -> KeyActive(cid("#0")),
globalKey("k1") -> KeyActive(cid("#1")),
globalKey("k2") -> KeyActive(cid("#2")),
globalKey("k3") -> KeyActive(cid("#3")),
)
)
}
"return None for negative lookup by key" in {
val builder = new TxBuilder()
val lookupNode = lookup(cid("#0"), found = false)
val lookupNode = lookup(cid("#0"), "k0", found = false)
builder.add(lookupNode)
builder.build().contractKeyInputs shouldBe Right(
Map(globalKey(cid("#0")) -> NegativeKeyLookup)
Map(globalKey("k0") -> NegativeKeyLookup)
)
}
"return Some(_) for negative lookup by key" in {
"return Some(_) for positive lookup by key" in {
val builder = new TxBuilder()
val lookupNode = lookup(cid("#0"), found = true)
val lookupNode = lookup(cid("#0"), "k0", found = true)
builder.add(lookupNode)
inside(lookupNode.result) { case Some(cid) =>
builder.build().contractKeyInputs shouldBe Right(Map(globalKey(cid) -> KeyActive(cid)))
inside(lookupNode.result) { case Some(contractId) =>
contractId shouldBe cid("#0")
builder.build().contractKeyInputs shouldBe Right(
Map(globalKey("k0") -> KeyActive(contractId))
)
}
}
"returns keys used under rollback nodes" in {
val builder = new TxBuilder()
val createNode = create(cid("#0"))
val exerciseNode = exe(cid("#1"), consuming = false, byKey = false)
val fetchNode = fetch(cid("#2"), byKey = false)
val lookupNode = lookup(cid("#3"), found = false)
val createNode = create(cid("#0"), "k0")
val exerciseNode = exe(cid("#1"), "k1", consuming = false, byKey = false)
val fetchNode = fetch(cid("#2"), "k2", byKey = false)
val lookupNode = lookup(cid("#3"), "k3", found = false)
val rollback = builder.add(builder.rollback())
builder.add(createNode, rollback)
builder.add(exerciseNode, rollback)
@ -475,120 +491,142 @@ class TransactionSpec
builder.add(lookupNode, rollback)
builder.build().contractKeyInputs shouldBe Right(
Map(
globalKey(cid("#0")) -> KeyCreate,
globalKey(cid("#1")) -> KeyActive(exerciseNode.targetCoid),
globalKey(cid("#2")) -> KeyActive(fetchNode.coid),
globalKey(cid("#3")) -> NegativeKeyLookup,
globalKey("k0") -> KeyCreate,
globalKey("k1") -> KeyActive(exerciseNode.targetCoid),
globalKey("k2") -> KeyActive(fetchNode.coid),
globalKey("k3") -> NegativeKeyLookup,
)
)
}
"fetch and create conflict for the same contract ID" in {
val builder = new TxBuilder()
builder.add(fetch(cid("#0"), "k0", byKey = false))
builder.add(create(cid("#0"), "k1"))
builder.build().contractKeyInputs shouldBe Left(
DuplicateContractIdKIError(DuplicateContractId(cid("#0")))
)
}
"lookup by key and create conflict for the same contract ID" in {
val builder = new TxBuilder()
builder.add(lookup(cid("#0"), "k0", found = true))
builder.add(create(cid("#0"), "k1"))
builder.build().contractKeyInputs shouldBe Left(
DuplicateContractIdKIError(DuplicateContractId(cid("#0")))
)
}
"consuming exercise and create conflict for the same contract ID" in {
val builder = new TxBuilder()
builder.add(exe(cid("#0"), "k1", consuming = true, byKey = false))
builder.add(create(cid("#0"), "k2"))
builder.build().contractKeyInputs shouldBe Left(
DuplicateContractIdKIError(DuplicateContractId(cid("#0")))
)
}
"non-consuming exercise and create conflict for the same contract ID" in {
val builder = new TxBuilder()
builder.add(exe(cid("#0"), "k1", consuming = false, byKey = false))
builder.add(create(cid("#0"), "k2"))
builder.build().contractKeyInputs shouldBe Left(
DuplicateContractIdKIError(DuplicateContractId(cid("#0")))
)
}
"two creates conflict" in {
val builder = new TxBuilder()
builder.add(create(cid("#0")))
builder.add(create(cid("#0")))
builder.add(create(cid("#0"), "k0"))
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Left(
Right(DuplicateContractKey(globalKey(cid("#0"))))
DuplicateContractKeyKIError(DuplicateContractKey(globalKey("k0")))
)
}
"two creates do not conflict if interleaved with archive" in {
val builder = new TxBuilder()
builder.add(create(cid("#0")))
builder.add(exe(cid("#0"), consuming = true, byKey = false))
builder.add(create(cid("#0")))
builder.build().contractKeyInputs shouldBe Right(Map(globalKey(cid("#0")) -> KeyCreate))
builder.add(create(cid("#0"), "k0"))
builder.add(exe(cid("#0"), "k0", consuming = true, byKey = false))
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Right(Map(globalKey("k0") -> KeyCreate))
}
"two creates do not conflict if one is in rollback" in {
val builder = new TxBuilder()
val rollback = builder.add(builder.rollback())
builder.add(create(cid("#0")), rollback)
builder.add(create(cid("#0")))
builder.build().contractKeyInputs shouldBe Right(Map(globalKey(cid("#0")) -> KeyCreate))
builder.add(create(cid("#0"), "k0"), rollback)
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Right(Map(globalKey("k0") -> KeyCreate))
}
"negative lookup after create fails" in {
val builder = new TxBuilder()
builder.add(create(cid("#0")))
builder.add(lookup(cid("#0"), found = false))
builder.add(create(cid("#0"), "k0"))
builder.add(lookup(cid("#0"), "k0", found = false))
builder.build().contractKeyInputs shouldBe Left(
Left(InconsistentContractKey(globalKey(cid("#0"))))
KeyInputError.inject(InconsistentContractKey(globalKey("k0")))
)
}
"inconsistent lookups conflict" in {
val builder = new TxBuilder()
builder.add(lookup(cid("#0"), found = true))
builder.add(lookup(cid("#0"), found = false))
builder.add(lookup(cid("#0"), "k0", found = true))
builder.add(lookup(cid("#0"), "k0", found = false))
builder.build().contractKeyInputs shouldBe Left(
Left(InconsistentContractKey(globalKey(cid("#0"))))
KeyInputError.inject(InconsistentContractKey(globalKey("k0")))
)
}
"inconsistent lookups conflict across rollback" in {
val builder = new TxBuilder()
val rollback = builder.add(builder.rollback())
builder.add(lookup(cid("#0"), found = true), rollback)
builder.add(lookup(cid("#0"), found = false))
builder.add(lookup(cid("#0"), "k0", found = true), rollback)
builder.add(lookup(cid("#0"), "k0", found = false))
builder.build().contractKeyInputs shouldBe Left(
Left(InconsistentContractKey(globalKey(cid("#0"))))
KeyInputError.inject(InconsistentContractKey(globalKey("k0")))
)
}
"positive lookup conflicts with create" in {
val builder = new TxBuilder()
builder.add(lookup(cid("#0"), found = true))
builder.add(create(cid("#0")))
builder.add(lookup(cid("#0"), "k0", found = true))
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Left(
Right(DuplicateContractKey(globalKey(cid("#0"))))
KeyInputError.inject(DuplicateContractKey(globalKey("k0")))
)
}
"positive lookup in rollback conflicts with create" in {
val builder = new TxBuilder()
val rollback = builder.add(builder.rollback())
builder.add(lookup(cid("#0"), found = true), rollback)
builder.add(create(cid("#0")))
builder.add(lookup(cid("#0"), "k0", found = true), rollback)
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Left(
Right(DuplicateContractKey(globalKey(cid("#0"))))
KeyInputError.inject(DuplicateContractKey(globalKey("k0")))
)
}
"rolled back archive does not prevent conflict" in {
val builder = new TxBuilder()
builder.add(create(cid("#0")))
builder.add(create(cid("#0"), "k0"))
val rollback = builder.add(builder.rollback())
builder.add(exe(cid("#0"), consuming = true, byKey = true), rollback)
builder.add(create(cid("#0")))
builder.add(exe(cid("#0"), "k0", consuming = true, byKey = true), rollback)
builder.add(create(cid("#1"), "k0"))
builder.build().contractKeyInputs shouldBe Left(
Right(DuplicateContractKey(globalKey(cid("#0"))))
KeyInputError.inject(DuplicateContractKey(globalKey("k0")))
)
}
"successful, inconsistent lookups conflict" in {
val builder = new TxBuilder()
val create0 = create(cid("#0"))
val create1 = create(cid("#1")).copy(
keyOpt = Some(
GlobalKeyWithMaintainers.assertBuild(
templateId = "Mod:T",
value = keyValue(cid("#0").coid),
maintainers = Set.empty,
)
)
)
val create0 = create(cid("#0"), "k0")
val create1 = create(cid("#1"), "k0")
builder.add(builder.lookupByKey(create0, found = true))
builder.add(builder.lookupByKey(create1, found = true))
builder.build().contractKeyInputs shouldBe Left(
Left(
InconsistentContractKey(globalKey(cid("#0")))
)
KeyInputError.inject(InconsistentContractKey(globalKey("k0")))
)
}
"first negative input wins" in {
val builder = new TxBuilder()
val rollback = builder.add(builder.rollback())
val create0 = create(cid("#0"))
val create0 = create(cid("#0"), "k0")
val lookup0 = builder.lookupByKey(create0, found = false)
val create1 = create(cid("#1"))
val create1 = create(cid("#1"), "k1")
val lookup1 = builder.lookupByKey(create1, found = false)
builder.add(create0, rollback)
builder.add(lookup1, rollback)
builder.add(lookup0)
builder.add(create1)
builder.build().contractKeyInputs shouldBe Right(
Map(globalKey(cid("#0")) -> KeyCreate, globalKey(cid("#1")) -> NegativeKeyLookup)
Map(globalKey("k0") -> KeyCreate, globalKey("k1") -> NegativeKeyLookup)
)
}
}