Properly fill eventId for active contracts (#3126)

* Properly fill eventId for active contracts

This gets rid of the last remaining bit that assumes
contractId==eventId.

Fixes #65.
Contributes to #2068.

* Do not conflate eventId and contractId in the daml-lf interpreter

* Do not treat contractId as eventId in Ledger.scala

* Remember the transaction that divulged a contract.

* In this scope we can treat divulged contracts the same as disclosed ones

* revert a few more syntactical changes to make the overall diff smaller

* retain the same behavior on the scenario service api

* fix unreleased after rebase
This commit is contained in:
Gerolf Seitz 2019-10-16 19:02:04 +02:00 committed by mergify[bot]
parent 4815d6f9a9
commit a5921e0274
15 changed files with 195 additions and 120 deletions

View File

@ -4,15 +4,16 @@
package com.digitalasset.daml.lf.scenario
import scala.collection.JavaConverters._
import com.digitalasset.daml.lf.data.{Numeric, Ref}
import com.digitalasset.daml.lf.scenario.api.v1
import com.digitalasset.daml.lf.scenario.api.v1.{List => _, _}
import com.digitalasset.daml.lf.speedy.SError
import com.digitalasset.daml.lf.speedy.Speedy
import com.digitalasset.daml.lf.speedy.SValue
import com.digitalasset.daml.lf.transaction.{Transaction => Tx, Node => N}
import com.digitalasset.daml.lf.transaction.{Node => N, Transaction => Tx}
import com.digitalasset.daml.lf.types.Ledger
import com.digitalasset.daml.lf.types.Ledger.ScenarioNodeId
import com.digitalasset.daml.lf.value.Value.AbsoluteContractId
import com.digitalasset.daml.lf.value.{Value => V}
case class Conversions(homePackageId: Ref.PackageId) {
@ -180,7 +181,9 @@ case class Conversions(homePackageId: Ref.PackageId) {
ledger.ledgerData.nodeInfos.map(Function.tupled(convertNode))
// NOTE(JM): Iteration over IntMap is in key-order. The IntMap's Int is IntMapUtils.Int for some reason.
,
ledger.scenarioSteps.map { case (idx, step) => convertScenarioStep(idx.toInt, step) })
ledger.scenarioSteps.map {
case (idx, step) => convertScenarioStep(idx.toInt, step, ledger.ledgerData.coidToNodeId)
})
def convertFailedAuthorizations(fas: Ledger.FailedAuthorizations): FailedAuthorizations = {
val builder = FailedAuthorizations.newBuilder
@ -304,7 +307,10 @@ case class Conversions(homePackageId: Ref.PackageId) {
case V.RelativeContractId(txnid) => txnid.index.toString
}
def convertScenarioStep(stepId: Int, step: Ledger.ScenarioStep): ScenarioStep = {
def convertScenarioStep(
stepId: Int,
step: Ledger.ScenarioStep,
coidToNodeId: AbsoluteContractId => ScenarioNodeId): ScenarioStep = {
val builder = ScenarioStep.newBuilder
builder.setStepId(stepId)
step match {
@ -316,7 +322,7 @@ case class Conversions(homePackageId: Ref.PackageId) {
builder.setCommit(
commitBuilder
.setTxId(txId.index)
.setTx(convertTransaction(rtx))
.setTx(convertTransaction(rtx, coidToNodeId))
.build)
case Ledger.PassTime(dt) =>
builder.setPassTime(dt)
@ -336,13 +342,20 @@ case class Conversions(homePackageId: Ref.PackageId) {
builder.build
}
def convertTransaction(rtx: Ledger.RichTransaction): Transaction =
def convertTransaction(
rtx: Ledger.RichTransaction,
coidToNodeId: AbsoluteContractId => ScenarioNodeId): Transaction = {
val convertedGlobalImplicitDisclosure = rtx.globalImplicitDisclosure.map {
case (coid, parties) => coidToNodeId(coid) -> parties
}
Transaction.newBuilder
.setCommitter(convertParty(rtx.committer))
.setEffectiveAt(rtx.effectiveAt.micros)
.addAllRoots(rtx.roots.map(convertNodeId).toSeq.asJava)
.addAllNodes(rtx.nodes.keys.map(convertNodeId).asJava)
.addAllDisclosures(rtx.disclosures.toSeq.map {
// previously rtx.disclosures returned both global and local implicit disclosures, but this is not the case anymore
// therefore we need to explicitly add the contracts that are divulged directly (via ContractId rather than ScenarioNodeId)
.addAllDisclosures((rtx.disclosures ++ convertedGlobalImplicitDisclosure).toSeq.map {
case (nodeId, parties) =>
NodeAndParties.newBuilder
.setNodeId(convertNodeId(nodeId))
@ -351,6 +364,7 @@ case class Conversions(homePackageId: Ref.PackageId) {
}.asJava)
.setFailedAuthorizations(convertFailedAuthorizations(rtx.failedAuthorizations))
.build
}
def convertPartialTransaction(ptx: Tx.PartialTransaction): PartialTransaction = {
val builder = PartialTransaction.newBuilder

View File

@ -68,7 +68,7 @@ object Ledger {
case class ScenarioTransactionId(index: Int) extends Ordered[ScenarioTransactionId] {
def next: ScenarioTransactionId = ScenarioTransactionId(index + 1)
val id: TransactionIdString = TransactionIdString.assertFromString(index.toString)
val id: TransactionIdString = TransactionIdString.fromLong(index.toLong)
override def compare(that: ScenarioTransactionId): Int = index compare that.index
def makeCommitPrefix: LedgerString = LedgerString.concat(id, `:`)
}
@ -123,10 +123,11 @@ object Ledger {
roots: ImmArray[ScenarioNodeId],
nodes: Map[ScenarioNodeId, Node],
explicitDisclosure: Relation[ScenarioNodeId, Party],
implicitDisclosure: Relation[ScenarioNodeId, Party],
localImplicitDisclosure: Relation[ScenarioNodeId, Party],
globalImplicitDisclosure: Relation[AbsoluteContractId, Party],
failedAuthorizations: FailedAuthorizations
) {
def disclosures = Relation.union(explicitDisclosure, implicitDisclosure)
def disclosures = Relation.union(explicitDisclosure, localImplicitDisclosure)
}
final case class EnrichedTransaction(
@ -157,34 +158,25 @@ object Ledger {
commitPrefix: LedgerString,
committer: Party,
effectiveAt: Time.Timestamp,
enrichedTx: EnrichedTransaction): RichTransaction = {
RichTransaction(
committer = committer,
effectiveAt = effectiveAt,
roots = enrichedTx.roots.map(ScenarioNodeId(commitPrefix, _)),
nodes = enrichedTx.nodes.map {
case (nodeId, node) =>
(ScenarioNodeId(commitPrefix, nodeId), translateNode(commitPrefix, node))
},
explicitDisclosure = enrichedTx.explicitDisclosure.map {
case (nodeId, ps) =>
(ScenarioNodeId(commitPrefix, nodeId), ps)
},
implicitDisclosure = {
val localDiv: Relation[ScenarioNodeId, Party] =
enrichedTx.localImplicitDisclosure.map {
case (nodeId, ps) =>
(ScenarioNodeId(commitPrefix, nodeId), ps)
}
val globalDiv: Relation[ScenarioNodeId, Party] =
enrichedTx.globalImplicitDisclosure.map {
case (absCoid, ps) => (ScenarioNodeId(absCoid), ps)
}
Relation.union(localDiv, globalDiv)
},
failedAuthorizations = enrichedTx.failedAuthorizations
)
}
enrichedTx: EnrichedTransaction): RichTransaction = RichTransaction(
committer = committer,
effectiveAt = effectiveAt,
roots = enrichedTx.roots.map(ScenarioNodeId(commitPrefix, _)),
nodes = enrichedTx.nodes.map {
case (nodeId, node) =>
(ScenarioNodeId(commitPrefix, nodeId), translateNode(commitPrefix, node))
},
explicitDisclosure = enrichedTx.explicitDisclosure.map {
case (nodeId, ps) =>
(ScenarioNodeId(commitPrefix, nodeId), ps)
},
localImplicitDisclosure = enrichedTx.localImplicitDisclosure.map {
case (nodeId, ps) =>
(ScenarioNodeId(commitPrefix, nodeId), ps)
},
globalImplicitDisclosure = enrichedTx.globalImplicitDisclosure,
failedAuthorizations = enrichedTx.failedAuthorizations
)
/**
* Translate a node of the update interpreter transaction to a node of the sandbox ledger
@ -392,8 +384,7 @@ object Ledger {
effectiveAt: Time.Timestamp,
coid: AbsoluteContractId
): LookupResult = {
val i = ScenarioNodeId(coid)
ledgerData.nodeInfos.get(i) match {
ledgerData.coidToNodeId.get(coid).flatMap(ledgerData.nodeInfos.get) match {
case None => LookupContractNotFound(coid)
case Some(info) =>
info.node match {
@ -1047,7 +1038,7 @@ object Ledger {
// ----------------------------------------------------------------
object LedgerData {
lazy val empty = LedgerData(Set.empty, Map.empty, Map.empty)
lazy val empty = LedgerData(Set.empty, Map.empty, Map.empty, Map.empty)
}
/**
@ -1059,8 +1050,14 @@ object Ledger {
final case class LedgerData(
activeContracts: Set[AbsoluteContractId],
nodeInfos: NodeInfos,
activeKeys: Map[GlobalKey, AbsoluteContractId]
activeKeys: Map[GlobalKey, AbsoluteContractId],
coidToNodeId: Map[AbsoluteContractId, ScenarioNodeId]
) {
def nodeInfoByCoid(coid: AbsoluteContractId): NodeInfo = nodeInfos(coidToNodeId(coid))
def updateNodeInfo(coid: AbsoluteContractId)(f: (NodeInfo) => NodeInfo): LedgerData =
coidToNodeId.get(coid).map(updateNodeInfo(_)(f)).getOrElse(this)
def updateNodeInfo(nodeId: ScenarioNodeId)(f: (NodeInfo) => NodeInfo): LedgerData =
copy(
nodeInfos = nodeInfos
@ -1068,17 +1065,21 @@ object Ledger {
.map(ni => nodeInfos.updated(nodeId, f(ni)))
.getOrElse(nodeInfos))
def markAsActive(nodeId: AbsoluteContractId): LedgerData =
copy(activeContracts = activeContracts + nodeId)
def markAsActive(coid: AbsoluteContractId): LedgerData =
copy(activeContracts = activeContracts + coid)
def markAsInactive(nid: AbsoluteContractId): LedgerData =
copy(activeContracts = activeContracts - nid)
def markAsInactive(coid: AbsoluteContractId): LedgerData =
copy(activeContracts = activeContracts - coid)
def createdIn(coid: AbsoluteContractId, nodeId: ScenarioNodeId): LedgerData =
copy(coidToNodeId = coidToNodeId + (coid -> nodeId))
def addKey(key: GlobalKey, acoid: AbsoluteContractId): LedgerData =
copy(activeKeys = activeKeys + (key -> acoid))
def removeKey(key: GlobalKey): LedgerData =
copy(activeKeys = activeKeys - key)
}
case class UniqueKeyViolation(gk: GlobalKey)
@ -1119,7 +1120,9 @@ object Ledger {
node match {
case nc: NodeCreate.WithTxValue[AbsoluteContractId] =>
val newCache1 =
newCache.markAsActive(nc.coid)
newCache
.markAsActive(nc.coid)
.createdIn(nc.coid, nodeId)
val mbNewCache2 = nc.key match {
case None => Right(newCache1)
case Some(keyWithMaintainers) =>
@ -1133,14 +1136,14 @@ object Ledger {
case NodeFetch(referencedCoid, templateId @ _, optLoc @ _, _, _, _) =>
val newCacheP =
newCache.updateNodeInfo(ScenarioNodeId(referencedCoid))(info =>
newCache.updateNodeInfo(referencedCoid)(info =>
info.copy(referencedBy = info.referencedBy + nodeId))
processNodes(Right(newCacheP), idsToProcess)
case ex: NodeExercises.WithTxValue[ScenarioNodeId, AbsoluteContractId] =>
val newCache0 =
newCache.updateNodeInfo(ScenarioNodeId(ex.targetCoid))(
newCache.updateNodeInfo(ex.targetCoid)(
info =>
info.copy(
referencedBy = info.referencedBy + nodeId,
@ -1150,7 +1153,7 @@ object Ledger {
if (ex.consuming) {
val newCache0_1 = newCache0.markAsInactive(ex.targetCoid)
val nc = newCache0_1
.nodeInfos(ScenarioNodeId(ex.targetCoid))
.nodeInfoByCoid(ex.targetCoid)
.node
.asInstanceOf[NodeCreate[
AbsoluteContractId,
@ -1172,7 +1175,7 @@ object Ledger {
processNodes(Right(newCache), idsToProcess)
case Some(referencedCoid) =>
val newCacheP =
newCache.updateNodeInfo(ScenarioNodeId(referencedCoid))(info =>
newCache.updateNodeInfo(referencedCoid)(info =>
info.copy(referencedBy = info.referencedBy + nodeId))
processNodes(Right(newCacheP), idsToProcess)
@ -1187,14 +1190,19 @@ object Ledger {
val mbCacheAfterProcess =
processNodes(Right(ledgerData), List(None -> richTr.roots.toList))
mbCacheAfterProcess.map(
cacheAfterProcess =>
Relation
.union(richTr.implicitDisclosure, richTr.explicitDisclosure)
.foldLeft(cacheAfterProcess) {
case (cacheP, (nodeId, witnesses)) =>
cacheP.updateNodeInfo(nodeId)(_.addObservers(witnesses.map(_ -> trId).toMap))
})
mbCacheAfterProcess.map { cacheAfterProcess =>
val globalImplicitDisclosure = richTr.globalImplicitDisclosure.map {
case (cid, parties) => ledgerData.coidToNodeId(cid) -> parties
}
Relation
.union(
Relation
.union(richTr.localImplicitDisclosure, richTr.explicitDisclosure),
globalImplicitDisclosure)
.foldLeft(cacheAfterProcess) {
case (cacheP, (nodeId, witnesses)) =>
cacheP.updateNodeInfo(nodeId)(_.addObservers(witnesses.map(_ -> trId).toMap))
}
}
}
}

View File

@ -300,6 +300,24 @@ class ActiveContractsService(session: LedgerSession) extends LedgerTestSuite(ses
}
}
private val eventId =
LedgerTest("ACSeventId", "The ActiveContractService should properly fill the eventId field") {
context =>
for {
ledger <- context.participant()
party <- ledger.allocateParty()
_ <- ledger.create(party, Dummy(party))
Vector(dummyEvent) <- ledger.activeContracts(party)
flatTransaction <- ledger.flatTransactionByEventId(dummyEvent.eventId, party)
transactionTree <- ledger.transactionTreeByEventId(dummyEvent.eventId, party)
} yield {
assert(
flatTransaction.transactionId == transactionTree.transactionId,
s"EventId ${dummyEvent.eventId} did not resolve to the same flat transaction (${flatTransaction.transactionId}) and transaction tree (${transactionTree.transactionId})."
)
}
}
override val tests: Vector[LedgerTest] = Vector(
invalidLedgerId,
emptyResponse,
@ -309,6 +327,7 @@ class ActiveContractsService(session: LedgerSession) extends LedgerTestSuite(ses
usableOffset,
verbosityFlag,
multiPartyRequests,
agreementText
agreementText,
eventId
)
}

View File

@ -11,7 +11,7 @@ import com.digitalasset.daml.lf.data.Relation.Relation
import com.digitalasset.daml.lf.transaction.Node.{GlobalKey, KeyWithMaintainers}
import com.digitalasset.daml.lf.value.Value
import com.digitalasset.daml.lf.value.Value.{AbsoluteContractId, ContractInst, VersionedValue}
import com.digitalasset.ledger.WorkflowId
import com.digitalasset.ledger.{EventId, WorkflowId}
import com.digitalasset.platform.sandbox.stores.ActiveLedgerState._
sealed abstract class LetLookup
@ -110,6 +110,7 @@ object ActiveLedgerState {
id: Value.AbsoluteContractId,
let: Instant, // time when the contract was committed
transactionId: TransactionIdString, // transaction id where the contract originates
eventId: EventId,
workflowId: Option[WorkflowId], // workflow id from where the contract originates
contract: ContractInst[VersionedValue[AbsoluteContractId]],
witnesses: Set[Party],

View File

@ -12,7 +12,7 @@ import com.digitalasset.daml.lf.transaction.Node.GlobalKey
import com.digitalasset.daml.lf.transaction.{GenTransaction, Node => N}
import com.digitalasset.daml.lf.value.Value
import com.digitalasset.daml.lf.value.Value.AbsoluteContractId
import com.digitalasset.ledger.WorkflowId
import com.digitalasset.ledger.{EventId, WorkflowId}
import com.digitalasset.platform.sandbox.services.transaction.SandboxEventIdFormatter
import com.digitalasset.platform.sandbox.stores.ActiveLedgerState._
import com.digitalasset.platform.sandbox.stores.ledger.SequencingError
@ -69,13 +69,13 @@ class ActiveLedgerStateManager[ALS](initialState: => ALS)(
* A higher order function to update an abstract active ledger state (ALS) with the effects of the given transaction.
* Makes sure that there are no double spends or timing errors.
*/
def addTransaction[Nid](
def addTransaction(
let: Instant,
transactionId: TransactionIdString,
workflowId: Option[WorkflowId],
transaction: GenTransaction.WithTxValue[Nid, AbsoluteContractId],
disclosure: Relation[Nid, Party],
localDivulgence: Relation[Nid, Party],
transaction: GenTransaction.WithTxValue[EventId, AbsoluteContractId],
disclosure: Relation[EventId, Party],
localDivulgence: Relation[EventId, Party],
globalDivulgence: Relation[AbsoluteContractId, Party],
divulgedContracts: List[(Value.AbsoluteContractId, AbsoluteContractInst)])
: Either[Set[SequencingError], ALS] = {
@ -147,6 +147,7 @@ class ActiveLedgerStateManager[ALS](initialState: => ALS)(
id = absCoid,
let = let,
transactionId = transactionId,
eventId = nodeId,
workflowId = workflowId,
contract = nc.coinst.mapValue(
_.mapContractId(SandboxEventIdFormatter.makeAbsCoid(transactionId))),
@ -182,7 +183,7 @@ class ActiveLedgerStateManager[ALS](initialState: => ALS)(
)
}
}
case ne: N.NodeExercises.WithTxValue[Nid, AbsoluteContractId] =>
case ne: N.NodeExercises.WithTxValue[EventId, AbsoluteContractId] =>
val nodeParties = ne.signatories
.union(ne.stakeholders)
.union(ne.actingParties)

View File

@ -12,7 +12,7 @@ import com.digitalasset.daml.lf.transaction.Node.GlobalKey
import com.digitalasset.daml.lf.transaction.GenTransaction
import com.digitalasset.daml.lf.value.Value
import com.digitalasset.daml.lf.value.Value.AbsoluteContractId
import com.digitalasset.ledger.WorkflowId
import com.digitalasset.ledger.{EventId, WorkflowId}
import com.digitalasset.ledger.api.domain.PartyDetails
import com.digitalasset.platform.sandbox.stores.ActiveLedgerState._
import com.digitalasset.platform.sandbox.stores.ledger.SequencingError
@ -130,13 +130,13 @@ case class InMemoryActiveLedgerState(
/** adds a transaction to the ActiveContracts, make sure that there are no double spends or
* timing errors. this check is leveraged to achieve higher concurrency, see LedgerState
*/
def addTransaction[Nid](
def addTransaction(
let: Instant,
transactionId: TransactionIdString,
workflowId: Option[WorkflowId],
transaction: GenTransaction.WithTxValue[Nid, AbsoluteContractId],
disclosure: Relation[Nid, Party],
localDivulgence: Relation[Nid, Party],
transaction: GenTransaction.WithTxValue[EventId, AbsoluteContractId],
disclosure: Relation[EventId, Party],
localDivulgence: Relation[EventId, Party],
glovalDivulgence: Relation[AbsoluteContractId, Party],
referencedContracts: List[(Value.AbsoluteContractId, AbsoluteContractInst)]
): Either[Set[SequencingError], InMemoryActiveLedgerState] =

View File

@ -192,7 +192,7 @@ abstract class LedgerBackedIndexService(
AcsUpdateEvent.Create(
// we use absolute contract ids as event ids throughout the sandbox
domain.TransactionId(ac.transactionId),
EventId(cId.coid),
domain.EventId(ac.eventId),
cId,
ac.contract.template,
ac.contract.arg,

View File

@ -147,7 +147,6 @@ object Ledger {
*
* @param jdbcUrl the jdbc url string containing the username and password as well
* @param ledgerId the id to be used for the ledger
* @param timeProvider the provider of time
* @return a jdbc backed Ledger
*/
def jdbcBackedReadOnly(

View File

@ -22,7 +22,6 @@ import com.digitalasset.daml.lf.transaction.VersionTimeline
import scala.collection.breakOut
import scala.collection.mutable.ArrayBuffer
import scalaz.syntax.std.map._
import scala.annotation.tailrec
@ -232,34 +231,32 @@ object ScenarioLoader {
}
}
val transactionId = Ref.LedgerString.concat(transactionIdPrefix, txId.id)
val transactionId = txId.id
val workflowId =
Some(Ref.LedgerString.concat(workflowIdPrefix, Ref.LedgerString.fromInt(stepId)))
// note that it's important that we keep the event ids in line with the contract ids, since
// the sandbox code assumes that in TransactionConversion.
val txNoHash = GenTransaction(richTransaction.nodes, richTransaction.roots, Set.empty)
val tx = txNoHash.mapContractIdAndValue(absCidWithHash, _.mapContractId(absCidWithHash))
import richTransaction.{explicitDisclosure, implicitDisclosure}
val tx = txNoHash
.mapContractIdAndValue(absCidWithHash, _.mapContractId(absCidWithHash))
.mapNodeId(nodeIdWithHash)
val mappedExplicitDisclosure = richTransaction.explicitDisclosure.map {
case (nid, parties) => nodeIdWithHash(nid) -> parties
}
val mappedLocalImplicitDisclosure = richTransaction.localImplicitDisclosure.map {
case (nid, parties) => nodeIdWithHash(nid) -> parties
}
// copies non-absolute-able node IDs, but IDs that don't match
// get intersected away later
val globalizedImplicitDisclosure = richTransaction.implicitDisclosure mapKeys { nid =>
absCidWithHash(AbsoluteContractId(nid))
}
acs.addTransaction[L.ScenarioNodeId](
acs.addTransaction(
time.toInstant,
transactionId,
workflowId,
tx,
explicitDisclosure,
implicitDisclosure,
globalizedImplicitDisclosure,
List.empty) match {
mappedExplicitDisclosure,
mappedLocalImplicitDisclosure,
richTransaction.globalImplicitDisclosure,
List.empty
) match {
case Right(newAcs) =>
val recordTx = tx.mapNodeId(nodeIdWithHash)
val recordDisclosure = explicitDisclosure.map {
case (nid, parties) => (nodeIdWithHash(nid), parties)
}
ledger +=
(
(
@ -272,8 +269,8 @@ object ScenarioLoader {
workflowId,
time.toInstant,
time.toInstant,
recordTx,
recordDisclosure
tx,
mappedExplicitDisclosure
)))
(newAcs, time, Some(txId))
case Left(err) =>

View File

@ -160,8 +160,17 @@ class InMemoryLedger(
SandboxEventIdFormatter.makeAbsCoid(trId)
val blindingInfo = Blinding.blind(transaction)
val mappedDisclosure = blindingInfo.disclosure.map {
case (nodeId, v) => SandboxEventIdFormatter.fromTransactionId(trId, nodeId) -> v
}
val mappedLocalDivulgence = blindingInfo.localDivulgence.map {
case (nodeId, v) => SandboxEventIdFormatter.fromTransactionId(trId, nodeId) -> v
}
val mappedGlobalDivulgence = blindingInfo.globalDivulgence
val mappedTx = transaction.mapContractIdAndValue(toAbsCoid, _.mapContractId(toAbsCoid))
val mappedTx = transaction
.mapContractIdAndValue(toAbsCoid, _.mapContractId(toAbsCoid))
.mapNodeId(SandboxEventIdFormatter.fromTransactionId(trId, _))
// 5b. modify the ActiveContracts, while checking that we do not have double
// spends or timing issues
val acsRes = acs.addTransaction(
@ -169,9 +178,9 @@ class InMemoryLedger(
trId,
transactionMeta.workflowId,
mappedTx,
blindingInfo.disclosure,
blindingInfo.localDivulgence,
blindingInfo.globalDivulgence,
mappedDisclosure,
mappedLocalDivulgence,
mappedGlobalDivulgence,
List.empty
)
acsRes match {
@ -181,8 +190,6 @@ class InMemoryLedger(
RejectionReason.Inconsistent(s"Reason: ${err.mkString("[", ", ", "]")}"))
case Right(newAcs) =>
acs = newAcs
val recordTx = mappedTx
.mapNodeId(SandboxEventIdFormatter.fromTransactionId(trId, _))
val recordBlinding =
blindingInfo.disclosure.map {
case (nid, parties) =>
@ -197,7 +204,7 @@ class InMemoryLedger(
transactionMeta.workflowId,
transactionMeta.ledgerEffectiveTime.toInstant,
recordTime,
recordTx,
mappedTx,
recordBlinding
)
entries.publish(entry)

View File

@ -481,7 +481,7 @@ private class JdbcLedgerDao(
// Note: ACS is typed as Unit here, as the ACS is given implicitly by the current database state
// within the current SQL transaction. All of the given functions perform side effects to update the database.
val atr = acsManager.addTransaction[EventId](
val atr = acsManager.addTransaction(
ledgerEffectiveTime,
transactionId,
workflowId,
@ -949,21 +949,22 @@ private class JdbcLedgerDao(
.deserializeContractInstance(ByteStreams.toByteArray(contractStream))
.getOrElse(sys.error(s"failed to deserialize contract! cid:$coid"))
val (signatories, observers) =
val (signatories, observers, eventId) =
transactionSerializer
.deserializeTransaction(ByteStreams.toByteArray(tx))
.getOrElse(sys.error(s"failed to deserialize transaction! cid:$coid"))
.nodes
.collectFirst {
case (_, NodeCreate(coid, _, _, signatories, stakeholders, _))
case (eventId, NodeCreate(coid, _, _, signatories, stakeholders, _))
if coid == absoluteCoid =>
(signatories, stakeholders diff signatories)
(signatories, stakeholders diff signatories, eventId)
} getOrElse sys.error(s"no create node in contract creating transaction! cid:$coid")
ActiveContract(
absoluteCoid,
ledgerEffectiveTime.toInstant,
transactionId,
eventId,
workflowId,
contractInstance,
witnesses,
@ -1075,7 +1076,7 @@ private class JdbcLedgerDao(
def orEmptyStringList(xs: Seq[String]) = if (xs.nonEmpty) xs else List("")
def contractStream(conn: Connection, offset: Long) = {
def contractStream(conn: Connection, offset: Long): Source[ActiveContract, Future[Done]] = {
//TODO: investigate where Akka Streams is actually iterating on the JDBC ResultSet (because, that is blocking IO!)
AkkaStream
.source(

View File

@ -396,7 +396,7 @@ class V2_1__Rebuild_Acs extends BaseJavaMigration {
// Note: ACS is typed as Unit here, as the ACS is given implicitly by the current database state
// within the current SQL transaction. All of the given functions perform side effects to update the database.
val atr = acsManager.addTransaction[EventId](
val atr = acsManager.addTransaction(
ledgerEffectiveTime,
transactionId,
workflowId,

View File

@ -27,12 +27,15 @@ import com.digitalasset.ledger.api.v1.value.Identifier
import com.digitalasset.ledger.client.services.acs.ActiveContractSetClient
import com.digitalasset.ledger.client.services.commands.SynchronousCommandClient
import com.digitalasset.ledger.client.services.transactions.TransactionClient
import com.digitalasset.platform.common.util.DirectExecutionContext
import com.digitalasset.platform.sandbox.services.{SandboxFixture, TestCommands}
import com.google.protobuf.timestamp.Timestamp
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.time.{Millis, Span}
import org.scalatest.{Matchers, Suite, WordSpec}
import scala.concurrent.Future
@SuppressWarnings(
Array(
"org.wartremover.warts.Any",
@ -107,6 +110,8 @@ abstract class ScenarioLoadingITBase
)
}
implicit val ec = DirectExecutionContext
"ScenarioLoading" when {
"contracts have been created" should {
@ -163,13 +168,20 @@ abstract class ScenarioLoadingITBase
}
}
"event ids are the same as contract ids (ACS)" in {
"event ids can be used to load transactions (ACS)" in {
val client = newTransactionClient(ledgerIdOnServer)
whenReady(submitRequest(SubmitAndWaitRequest(commands = dummyRequest.commands))) { _ =>
whenReady(getSnapshot()) { resp =>
val responses = resp.init // last response is just ledger offset
val events = responses.flatMap(extractEvents)
events.foreach { event =>
event.eventId shouldBe event.contractId
val eventIds = responses.flatMap(_.activeContracts).map(_.eventId)
val txByEventIdF = Future
.sequence(eventIds.map(evId =>
client.getFlatTransactionByEventId(evId, Seq(M.party)).map(evId -> _)))
.map(_.toMap)
whenReady(txByEventIdF) { txByEventId =>
eventIds.foreach { evId =>
txByEventId.keySet should contain(evId)
}
}
}
}
@ -178,18 +190,31 @@ abstract class ScenarioLoadingITBase
"event ids are the same as contract ids (transaction service)" in {
val beginOffset =
LedgerOffset(LedgerOffset.Value.Boundary(LedgerOffset.LedgerBoundary.LEDGER_BEGIN))
val resultsF =
newTransactionClient(ledgerIdOnServer)
.getTransactions(beginOffset, None, transactionFilter)
.take(4)
.runWith(Sink.seq)
val client = newTransactionClient(ledgerIdOnServer)
val resultsF = client
.getTransactions(beginOffset, None, transactionFilter)
.take(4)
.runWith(Sink.seq)
whenReady(resultsF) { txs =>
val events = txs.flatMap(_.events).map(_.getCreated)
events.length shouldBe 4
events.foreach { event =>
event.eventId shouldBe event.contractId
val txByEventIdF = Future
.sequence(
events.map(e =>
client
.getFlatTransactionByEventId(e.eventId, Seq(M.party))
.map(e.eventId -> _)))
.map(_.toMap)
whenReady(txByEventIdF) { txByEventId =>
events.foreach { event =>
txByEventId.keys should contain(event.eventId)
}
}
}
}
}

View File

@ -136,6 +136,7 @@ class JdbcLedgerDaoSpec
absCid,
let,
txId,
event1,
Some(workflowId),
contractInstance,
Set(alice, bob),

View File

@ -11,3 +11,5 @@ HEAD — ongoing
- [DAML Ledger Integration Kit] Skew/LET/MRT/Config tests consolidated in a single suite.
- [JSON API - Experimental] Add ``/parties`` endpoint.
- [Sandbox] Party management fix, see `issue #3177 <https://github.com/digital-asset/daml/issues/3177>`_.
+ [Ledger] Fixed a bug where ``CreatedEvent#event_id`` field is not properly filled by ``ActiveContractsService``.
See `issue #65 <https://github.com/digital-asset/daml/issues/65>`__.