Introduce locally-defined Rejection reasons instead of the participant-state rejection reason type. [KVL-1002] (#10376)

* participant-integration-api: Reduce usage of the state RejectionReason.

Use a local `Rejection` enumeration instead.

This also changes an implicit conversion to be a somewhat-explicit
conversion.

CHANGELOG_BEGIN
CHANGELOG_END

* sandbox-classic: Internalize rejection reasons.

Instead of using RejectionReasonV0, we can declare them locally.

* kvutils: Create internal rejection reasons in the committer.

These can be converted to state rejection reasons later.

By using internal rejection representations, we're not coupled to the
state API.

* sandbox-classic: Remove unused code from `Rejection`.

* participant-integration-api: Push the rejection reason conversion down.

Just a little bit further.

* participant-integration-api: Add tests for converting rejection reasons.

* participant-integration-api: Remove an unused test dependency.
This commit is contained in:
Samir Talwar 2021-07-23 15:20:00 +02:00 committed by GitHub
parent 96f048330a
commit d7077e154f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 480 additions and 228 deletions

View File

@ -219,6 +219,7 @@ da_scala_test_suite(
"@maven//:com_typesafe_akka_akka_stream",
"@maven//:com_typesafe_akka_akka_stream_testkit",
"@maven//:org_mockito_mockito_scala",
"@maven//:org_playframework_anorm_anorm",
"@maven//:org_scalacheck_scalacheck",
"@maven//:org_scalactic_scalactic",
"@maven//:org_scalatest_scalatest",
@ -260,7 +261,6 @@ da_scala_test_suite(
"//ledger/metrics:metrics-test-lib",
"//ledger/participant-state",
"//ledger/participant-state-index",
"//ledger/participant-state/kvutils",
"//ledger/test-common",
"//ledger/test-common:dar-files-stable-lib",
"//libs-scala/concurrent",

View File

@ -12,7 +12,7 @@ import com.daml.ledger.api.v1.ledger_offset.LedgerOffset
import com.daml.ledger.offset.Offset
import com.daml.lf.data.Ref
import com.daml.platform.ApiOffset.ApiOffsetConverter
import com.daml.platform.store.Conversions.domainRejectionReasonToErrorCode
import com.daml.platform.store.Conversions.RejectionReasonOps
import com.daml.platform.store.entries.LedgerEntry
import com.google.rpc.status.Status
@ -56,17 +56,14 @@ private[platform] object CompletionFromTransaction {
checkpoint = toApiCheckpoint(recordTime, offset),
Seq(Completion(commandId, Some(Status()), transactionId)),
)
case (offset, LedgerEntry.Rejection(recordTime, commandId, `appId`, actAs, reason))
if actAs.exists(parties) =>
val stateReason = reason.toParticipantStateRejectionReason
val status = Status(stateReason.code.value, stateReason.description)
offset -> CompletionStreamResponse(
checkpoint = toApiCheckpoint(recordTime, offset),
Seq(
Completion(
commandId,
Some(Status(reason.value, reason.description)),
)
),
Seq(Completion(commandId, Some(status))),
)
}
}

View File

@ -13,18 +13,14 @@ import anorm.Column.nonNull
import anorm._
import com.daml.ledger.api.domain
import com.daml.ledger.offset.Offset
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.ledger.participant.state.v1.RejectionReasonV0._
import com.daml.ledger.participant.state.{v1 => state}
import com.daml.lf.crypto.Hash
import com.daml.lf.data.Ref
import com.daml.lf.ledger.EventId
import com.daml.lf.value.Value
import io.grpc.Status.Code
import spray.json.DefaultJsonProtocol._
import spray.json._
import scala.language.implicitConversions
// TODO append-only: split this file on cleanup, and move anorm/db conversion related stuff to the right place
private[platform] object OracleArrayConversions {
@ -340,22 +336,21 @@ private[platform] object Conversions {
override val jdbcType: Int = ParameterMetaData.StringParameterMetaData.jdbcType
}
// RejectionReason
implicit def domainRejectionReasonToErrorCode(reason: domain.RejectionReason): Code =
domainRejectionReasonToParticipantRejectionReason(
reason
).code
implicit def domainRejectionReasonToParticipantRejectionReason(
reason: domain.RejectionReason
): RejectionReasonV0 =
reason match {
case r: domain.RejectionReason.Inconsistent => Inconsistent(r.description)
case r: domain.RejectionReason.Disputed => Disputed(r.description)
case r: domain.RejectionReason.OutOfQuota => ResourcesExhausted(r.description)
case r: domain.RejectionReason.PartyNotKnownOnLedger => PartyNotKnownOnLedger(r.description)
case r: domain.RejectionReason.SubmitterCannotActViaParticipant =>
SubmitterCannotActViaParticipant(r.description)
case r: domain.RejectionReason.InvalidLedgerTime => InvalidLedgerTime(r.description)
}
implicit class RejectionReasonOps(rejectionReason: domain.RejectionReason) {
def toParticipantStateRejectionReason: state.RejectionReason =
rejectionReason match {
case domain.RejectionReason.Inconsistent(reason) =>
state.RejectionReasonV0.Inconsistent(reason)
case domain.RejectionReason.Disputed(reason) =>
state.RejectionReasonV0.Disputed(reason)
case domain.RejectionReason.OutOfQuota(reason) =>
state.RejectionReasonV0.ResourcesExhausted(reason)
case domain.RejectionReason.PartyNotKnownOnLedger(reason) =>
state.RejectionReasonV0.PartyNotKnownOnLedger(reason)
case domain.RejectionReason.SubmitterCannotActViaParticipant(reason) =>
state.RejectionReasonV0.SubmitterCannotActViaParticipant(reason)
case domain.RejectionReason.InvalidLedgerTime(reason) =>
state.RejectionReasonV0.InvalidLedgerTime(reason)
}
}
}

View File

@ -345,7 +345,7 @@ private class JdbcLedgerDao(
ledgerEffectiveTime: Instant,
transaction: CommittedTransaction,
divulged: Iterable[DivulgedContract],
)(implicit connection: Connection): Option[RejectionReason] =
)(implicit connection: Connection): Option[PostCommitValidation.Rejection] =
Timed.value(
metrics.daml.index.db.storeTransactionDbMetrics.commitValidation,
postCommitValidation.validate(
@ -427,7 +427,7 @@ private class JdbcLedgerDao(
Update.CommandRejected(
recordTime = Time.Timestamp.assertFromInstant(recordTime),
submitterInfo = SubmitterInfo(actAs, applicationId, commandId, Instant.EPOCH),
reason = reason,
reason = reason.toParticipantStateRejectionReason,
)
),
)
@ -686,12 +686,12 @@ private class JdbcLedgerDao(
)
)
case Some(error) =>
case Some(reason) =>
submitterInfo.map(someSubmitterInfo =>
Update.CommandRejected(
recordTime = Time.Timestamp.assertFromInstant(recordTime),
submitterInfo = someSubmitterInfo,
reason = error,
reason = reason.toStateV1RejectionReason,
)
)
},

View File

@ -6,8 +6,9 @@ package com.daml.platform.store.appendonlydao.events
import java.sql.Connection
import java.time.Instant
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.ledger.participant.state.v1
import com.daml.lf.transaction.CommittedTransaction
import com.daml.platform.store.appendonlydao.events.PostCommitValidation._
import com.daml.platform.store.backend.{ContractStorageBackend, PartyStorageBackend}
/** Performs post-commit validation on transactions for Sandbox Classic.
@ -31,7 +32,7 @@ private[appendonlydao] sealed trait PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0]
)(implicit connection: Connection): Option[Rejection]
}
@ -47,7 +48,7 @@ private[appendonlydao] object PostCommitValidation {
committedTransaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] =
)(implicit connection: Connection): Option[Rejection] =
None
}
@ -60,7 +61,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val causalMonotonicityViolation =
validateCausalMonotonicity(transaction, transactionLedgerEffectiveTime, divulged)
@ -84,7 +85,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val referredContracts = collectReferredContracts(transaction, divulged)
if (referredContracts.isEmpty) {
None
@ -92,19 +93,19 @@ private[appendonlydao] object PostCommitValidation {
dao
.maximumLedgerTime(referredContracts)(connection)
.map(validateCausalMonotonicity(_, transactionLedgerEffectiveTime))
.getOrElse(Some(UnknownContract))
.getOrElse(Some(Rejection.UnknownContract))
}
}
private def validateCausalMonotonicity(
maximumLedgerEffectiveTime: Option[Instant],
transactionLedgerEffectiveTime: Instant,
): Option[RejectionReasonV0] =
): Option[Rejection] =
maximumLedgerEffectiveTime
.filter(_.isAfter(transactionLedgerEffectiveTime))
.fold(Option.empty[RejectionReasonV0])(contractLedgerEffectiveTime => {
.fold(Option.empty[Rejection])(contractLedgerEffectiveTime => {
Some(
CausalMonotonicityViolation(
Rejection.CausalMonotonicityViolation(
contractLedgerEffectiveTime = contractLedgerEffectiveTime,
transactionLedgerEffectiveTime = transactionLedgerEffectiveTime,
)
@ -113,13 +114,13 @@ private[appendonlydao] object PostCommitValidation {
private def validateParties(
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val informees = transaction.informees
val allocatedInformees = dao.parties(informees.toSeq)(connection).map(_.party)
if (allocatedInformees.toSet == informees)
None
else
Some(RejectionReasonV0.PartyNotKnownOnLedger("Some parties are unallocated"))
Some(Rejection.UnallocatedParties)
}
private def collectReferredContracts(
@ -131,7 +132,7 @@ private[appendonlydao] object PostCommitValidation {
private def validateKeyUsages(
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] =
)(implicit connection: Connection): Option[Rejection] =
transaction
.foldInExecutionOrder[Result](Right(State.empty(dao)))(
exerciseBegin = (acc, _, exe) => {
@ -148,7 +149,7 @@ private[appendonlydao] object PostCommitValidation {
private def validateKeyUsages(
node: Node,
state: State,
)(implicit connection: Connection): Either[RejectionReasonV0, State] =
)(implicit connection: Connection): Result =
node match {
case c: Create =>
state.validateCreate(c.versionedKey.map(convert(c.versionedCoinst.template, _)), c.coid)
@ -166,7 +167,7 @@ private[appendonlydao] object PostCommitValidation {
}
private type Result = Either[RejectionReasonV0, State]
private type Result = Either[Rejection, State]
/** The active ledger key state during validation.
* After a rollback node, we restore the state at the
@ -214,23 +215,25 @@ private[appendonlydao] object PostCommitValidation {
private val dao: PartyStorageBackend with ContractStorageBackend,
) {
def validateCreate(maybeKey: Option[Key], id: ContractId)(implicit
connection: Connection
): Either[RejectionReasonV0, State] =
def validateCreate(
maybeKey: Option[Key],
id: ContractId,
)(implicit connection: Connection): Result =
maybeKey.fold[Result](Right(this)) { key =>
lookup(key).fold[Result](Right(add(key, id)))(_ => Left(DuplicateKey))
lookup(key).fold[Result](Right(add(key, id)))(_ => Left(Rejection.DuplicateKey))
}
// `causalMonotonicity` already reports unknown contracts, no need to check it here
def removeKeyIfDefined(maybeKey: Option[Key]): Right[RejectionReasonV0, State] =
def removeKeyIfDefined(maybeKey: Option[Key]): Result =
Right(maybeKey.fold(this)(remove))
def validateLookupByKey(key: Key, expectation: Option[ContractId])(implicit
connection: Connection
): Either[RejectionReasonV0, State] = {
def validateLookupByKey(
key: Key,
expectation: Option[ContractId],
)(implicit connection: Connection): Result = {
val result = lookup(key)
if (result == expectation) Right(this)
else Left(MismatchingLookup(expectation, result))
else Left(Rejection.MismatchingLookup(expectation, result))
}
def beginRollback(): State =
@ -271,25 +274,57 @@ private[appendonlydao] object PostCommitValidation {
State(ActiveState(Map.empty, Set.empty), Nil, dao)
}
private[events] val DuplicateKey: RejectionReasonV0 =
RejectionReasonV0.Inconsistent("DuplicateKey: contract key is not unique")
sealed trait Rejection {
def description: String
private[events] def MismatchingLookup(
expectation: Option[ContractId],
result: Option[ContractId],
): RejectionReasonV0 =
RejectionReasonV0.Inconsistent(
s"Contract key lookup with different results: expected [$expectation], actual [$result]"
)
def toStateV1RejectionReason: v1.RejectionReason
}
private[events] val UnknownContract: RejectionReasonV0 =
RejectionReasonV0.Inconsistent("Unknown contract")
object Rejection {
object UnknownContract extends Rejection {
override val description =
"Unknown contract"
private[events] def CausalMonotonicityViolation(
contractLedgerEffectiveTime: Instant,
transactionLedgerEffectiveTime: Instant,
): RejectionReasonV0 =
RejectionReasonV0.InvalidLedgerTime(
s"Encountered contract with LET [$contractLedgerEffectiveTime] greater than the LET of the transaction [$transactionLedgerEffectiveTime]"
)
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
object DuplicateKey extends Rejection {
override val description =
"DuplicateKey: contract key is not unique"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
final case class MismatchingLookup(
expectation: Option[ContractId],
result: Option[ContractId],
) extends Rejection {
override lazy val description: String =
s"Contract key lookup with different results: expected [$expectation], actual [$result]"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
final case class CausalMonotonicityViolation(
contractLedgerEffectiveTime: Instant,
transactionLedgerEffectiveTime: Instant,
) extends Rejection {
override lazy val description: String =
s"Encountered contract with LET [$contractLedgerEffectiveTime] greater than the LET of the transaction [$transactionLedgerEffectiveTime]"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.InvalidLedgerTime(description)
}
object UnallocatedParties extends Rejection {
override def description: String =
"Some parties are unallocated"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.PartyNotKnownOnLedger(description)
}
}
}

View File

@ -33,7 +33,7 @@ import com.daml.ledger.participant.state.index.v2.{
CommandDeduplicationResult,
PackageDetails,
}
import com.daml.ledger.participant.state.v1._
import com.daml.ledger.participant.state.v1.{DivulgedContract, RejectionReason, SubmitterInfo}
import com.daml.ledger.resources.ResourceOwner
import com.daml.lf.archive.ArchiveParser
import com.daml.lf.data.Ref
@ -510,7 +510,9 @@ private class JdbcLedgerDao(
preparedInsert.writeEvents(metrics)
insertCompletions(submitterInfo, transactionId, recordTime, offsetStep)
case Some(error) =>
submitterInfo.foreach(handleError(offsetStep.offset, _, recordTime, error))
submitterInfo.foreach(
handleError(offsetStep.offset, _, recordTime, error.toStateV1RejectionReason)
)
}
updateLedgerEnd(offsetStep)
@ -522,7 +524,7 @@ private class JdbcLedgerDao(
ledgerEffectiveTime: Instant,
transaction: CommittedTransaction,
divulged: Iterable[DivulgedContract],
)(implicit connection: Connection) =
)(implicit connection: Connection): Option[PostCommitValidation.Rejection] =
Timed.value(
metrics.daml.index.db.storeTransactionDbMetrics.commitValidation,
postCommitValidation.validate(
@ -604,7 +606,7 @@ private class JdbcLedgerDao(
submitterInfo = SubmitterInfo(actAs, applicationId, commandId, Instant.EPOCH),
offset = offset,
recordTime = recordTime,
reason = reason,
reason = reason.toParticipantStateRejectionReason,
)
.execute()
}

View File

@ -6,8 +6,9 @@ package com.daml.platform.store.dao.events
import java.sql.Connection
import java.time.Instant
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.ledger.participant.state.v1
import com.daml.lf.transaction.CommittedTransaction
import com.daml.platform.store.dao.events.PostCommitValidation._
/** Performs post-commit validation on transactions for Sandbox Classic.
* This is intended exclusively as a temporary replacement for
@ -30,7 +31,7 @@ private[dao] sealed trait PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0]
)(implicit connection: Connection): Option[Rejection]
}
@ -46,7 +47,7 @@ private[dao] object PostCommitValidation {
committedTransaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] =
)(implicit connection: Connection): Option[Rejection] =
None
}
@ -57,7 +58,7 @@ private[dao] object PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val causalMonotonicityViolation =
validateCausalMonotonicity(transaction, transactionLedgerEffectiveTime, divulged)
@ -81,7 +82,7 @@ private[dao] object PostCommitValidation {
transaction: CommittedTransaction,
transactionLedgerEffectiveTime: Instant,
divulged: Set[ContractId],
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val referredContracts = collectReferredContracts(transaction, divulged)
if (referredContracts.isEmpty) {
None
@ -89,19 +90,19 @@ private[dao] object PostCommitValidation {
data
.lookupMaximumLedgerTime(referredContracts)
.map(validateCausalMonotonicity(_, transactionLedgerEffectiveTime))
.getOrElse(Some(UnknownContract))
.getOrElse(Some(Rejection.UnknownContract))
}
}
private def validateCausalMonotonicity(
maximumLedgerEffectiveTime: Option[Instant],
transactionLedgerEffectiveTime: Instant,
): Option[RejectionReasonV0] =
): Option[Rejection] =
maximumLedgerEffectiveTime
.filter(_.isAfter(transactionLedgerEffectiveTime))
.fold(Option.empty[RejectionReasonV0])(contractLedgerEffectiveTime => {
.fold(Option.empty[Rejection])(contractLedgerEffectiveTime => {
Some(
CausalMonotonicityViolation(
Rejection.CausalMonotonicityViolation(
contractLedgerEffectiveTime = contractLedgerEffectiveTime,
transactionLedgerEffectiveTime = transactionLedgerEffectiveTime,
)
@ -110,13 +111,13 @@ private[dao] object PostCommitValidation {
private def validateParties(
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] = {
)(implicit connection: Connection): Option[Rejection] = {
val informees = transaction.informees
val allocatedInformees = data.lookupParties(informees.toSeq).map(_.party)
if (allocatedInformees.toSet == informees)
None
else
Some(RejectionReasonV0.PartyNotKnownOnLedger("Some parties are unallocated"))
Some(Rejection.UnallocatedParties)
}
private def collectReferredContracts(
@ -128,7 +129,7 @@ private[dao] object PostCommitValidation {
private def validateKeyUsages(
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] =
)(implicit connection: Connection): Option[Rejection] =
transaction
.foldInExecutionOrder[Result](Right(State.empty(data)))(
exerciseBegin = (acc, _, exe) => {
@ -145,7 +146,7 @@ private[dao] object PostCommitValidation {
private def validateKeyUsages(
node: Node,
state: State,
)(implicit connection: Connection): Either[RejectionReasonV0, State] =
)(implicit connection: Connection): Result =
node match {
case c: Create =>
state.validateCreate(c.versionedKey.map(convert(c.versionedCoinst.template, _)), c.coid)
@ -163,7 +164,7 @@ private[dao] object PostCommitValidation {
}
private type Result = Either[RejectionReasonV0, State]
private type Result = Either[Rejection, State]
/** The active ledger key state during validation.
* After a rollback node, we restore the state at the
@ -213,21 +214,21 @@ private[dao] object PostCommitValidation {
def validateCreate(maybeKey: Option[Key], id: ContractId)(implicit
connection: Connection
): Either[RejectionReasonV0, State] =
): Result =
maybeKey.fold[Result](Right(this)) { key =>
lookup(key).fold[Result](Right(add(key, id)))(_ => Left(DuplicateKey))
lookup(key).fold[Result](Right(add(key, id)))(_ => Left(Rejection.DuplicateKey))
}
// `causalMonotonicity` already reports unknown contracts, no need to check it here
def removeKeyIfDefined(maybeKey: Option[Key]): Right[RejectionReasonV0, State] =
def removeKeyIfDefined(maybeKey: Option[Key]): Result =
Right(maybeKey.fold(this)(remove))
def validateLookupByKey(key: Key, expectation: Option[ContractId])(implicit
connection: Connection
): Either[RejectionReasonV0, State] = {
): Result = {
val result = lookup(key)
if (result == expectation) Right(this)
else Left(MismatchingLookup(expectation, result))
else Left(Rejection.MismatchingLookup(expectation, result))
}
def beginRollback(): State =
@ -266,25 +267,58 @@ private[dao] object PostCommitValidation {
State(ActiveState(Map.empty, Set.empty), Nil, data)
}
private[events] val DuplicateKey: RejectionReasonV0 =
RejectionReasonV0.Inconsistent("DuplicateKey: contract key is not unique")
sealed trait Rejection {
def description: String
private[events] def MismatchingLookup(
expectation: Option[ContractId],
result: Option[ContractId],
): RejectionReasonV0 =
RejectionReasonV0.Inconsistent(
s"Contract key lookup with different results: expected [$expectation], actual [$result]"
)
def toStateV1RejectionReason: v1.RejectionReason
}
private[events] val UnknownContract: RejectionReasonV0 =
RejectionReasonV0.Inconsistent("Unknown contract")
object Rejection {
object UnknownContract extends Rejection {
override val description =
"Unknown contract"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
object DuplicateKey extends Rejection {
override val description =
"DuplicateKey: contract key is not unique"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
final case class MismatchingLookup(
expectation: Option[ContractId],
result: Option[ContractId],
) extends Rejection {
override lazy val description: String =
s"Contract key lookup with different results: expected [$expectation], actual [$result]"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
final case class CausalMonotonicityViolation(
contractLedgerEffectiveTime: Instant,
transactionLedgerEffectiveTime: Instant,
) extends Rejection {
override lazy val description: String =
s"Encountered contract with LET [$contractLedgerEffectiveTime] greater than the LET of the transaction [$transactionLedgerEffectiveTime]"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.InvalidLedgerTime(description)
}
object UnallocatedParties extends Rejection {
override def description: String =
"Some parties are unallocated"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.PartyNotKnownOnLedger(description)
}
}
private[events] def CausalMonotonicityViolation(
contractLedgerEffectiveTime: Instant,
transactionLedgerEffectiveTime: Instant,
): RejectionReasonV0 =
RejectionReasonV0.InvalidLedgerTime(
s"Encountered contract with LET [$contractLedgerEffectiveTime] greater than the LET of the transaction [$transactionLedgerEffectiveTime]"
)
}

View File

@ -0,0 +1,50 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.platform.store
import com.daml.ledger.api.domain
import com.daml.ledger.participant.state.{v1 => state}
import com.daml.platform.store.Conversions._
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AsyncWordSpec
class ConversionsSpec extends AsyncWordSpec with Matchers {
"converting rejection reasons" should {
"convert an 'Inconsistent' rejection reason" in {
val reason = domain.RejectionReason.Inconsistent("This was not very consistent.")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.Inconsistent("This was not very consistent."))
}
"convert an 'Disputed' rejection reason" in {
val reason = domain.RejectionReason.Disputed("I dispute that.")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.Disputed("I dispute that."))
}
"convert an 'OutOfQuota' rejection reason" in {
val reason = domain.RejectionReason.OutOfQuota("Insert coins to continue.")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.ResourcesExhausted("Insert coins to continue."))
}
"convert an 'PartyNotKnownOnLedger' rejection reason" in {
val reason = domain.RejectionReason.PartyNotKnownOnLedger("Who on earth is Alice?")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.PartyNotKnownOnLedger("Who on earth is Alice?"))
}
"convert an 'SubmitterCannotActViaParticipant' rejection reason" in {
val reason = domain.RejectionReason.SubmitterCannotActViaParticipant("Wrong box.")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.SubmitterCannotActViaParticipant("Wrong box."))
}
"convert an 'InvalidLedgerTime' rejection reason" in {
val reason = domain.RejectionReason.InvalidLedgerTime("Too late.")
val converted = reason.toParticipantStateRejectionReason
converted should be(state.RejectionReasonV0.InvalidLedgerTime("Too late."))
}
}
}

View File

@ -8,7 +8,6 @@ import java.time.Instant
import java.util.UUID
import com.daml.ledger.api.domain.PartyDetails
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.lf.transaction.GlobalKey
import com.daml.lf.transaction.test.{TransactionBuilder => TxBuilder}
import com.daml.lf.value.Value.ValueText
@ -88,7 +87,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(UnknownContract)
error shouldBe Some(Rejection.UnknownContract)
}
"accept a fetch of a contract created within the transaction" in {
@ -124,7 +123,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(UnknownContract)
error shouldBe Some(Rejection.UnknownContract)
}
"accept a successful lookup of a contract created in this transaction" in {
@ -150,7 +149,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
)
error shouldBe Some(
MismatchingLookup(expectation = Some(missingCreate.coid), result = None)
Rejection.MismatchingLookup(expectation = Some(missingCreate.coid), result = None)
)
}
@ -211,7 +210,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(DuplicateKey)
error shouldBe Some(Rejection.DuplicateKey)
}
"reject a create after a rolled back archive of a contract with the same key" in {
@ -228,7 +227,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(DuplicateKey)
error shouldBe Some(Rejection.DuplicateKey)
}
"accept a failed lookup in a rollback" in {
@ -273,7 +272,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(DuplicateKey)
error shouldBe Some(Rejection.DuplicateKey)
}
"accept an exercise on the committed contract" in {
@ -294,7 +293,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
)
error shouldBe Some(
CausalMonotonicityViolation(
Rejection.CausalMonotonicityViolation(
contractLedgerEffectiveTime = committedContractLedgerEffectiveTime,
transactionLedgerEffectiveTime = committedContractLedgerEffectiveTime.minusNanos(1),
)
@ -319,7 +318,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
)
error shouldBe Some(
CausalMonotonicityViolation(
Rejection.CausalMonotonicityViolation(
contractLedgerEffectiveTime = committedContractLedgerEffectiveTime,
transactionLedgerEffectiveTime = committedContractLedgerEffectiveTime.minusNanos(1),
)
@ -346,7 +345,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
)
error shouldBe Some(
MismatchingLookup(result = Some(committedContract.coid), expectation = None)
Rejection.MismatchingLookup(result = Some(committedContract.coid), expectation = None)
)
}
@ -361,7 +360,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(DuplicateKey)
error shouldBe Some(Rejection.DuplicateKey)
}
"reject a failed lookup in a rollback" in {
@ -376,7 +375,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
)
error shouldBe Some(
MismatchingLookup(
Rejection.MismatchingLookup(
result = Some(committedContract.coid),
expectation = None,
)
@ -409,7 +408,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(DuplicateKey)
error shouldBe Some(Rejection.DuplicateKey)
}
}
@ -460,7 +459,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(RejectionReasonV0.PartyNotKnownOnLedger("Some parties are unallocated"))
error shouldBe Some(Rejection.UnallocatedParties)
}
"reject if party is used in rollback" in {
@ -475,7 +474,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
divulged = Set.empty,
)
error shouldBe Some(RejectionReasonV0.PartyNotKnownOnLedger("Some parties are unallocated"))
error shouldBe Some(Rejection.UnallocatedParties)
}
}
}

View File

@ -0,0 +1,145 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.ledger.participant.state.kvutils.committer.transaction
import java.time.Instant
import com.daml.ledger.configuration.LedgerTimeModel
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlStateKey
import com.daml.ledger.participant.state.kvutils.Err
import com.daml.ledger.participant.state.v1
import com.daml.lf
import com.daml.lf.data.Ref
sealed trait Rejection {
def description: String
def toStateV1RejectionReason: v1.RejectionReason
}
object Rejection {
final case class ValidationFailure(error: lf.engine.Error) extends Rejection {
override lazy val description: String =
error.msg
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Disputed(description)
}
object InternallyInconsistentTransaction {
object DuplicateKeys extends Rejection {
override val description: String =
"DuplicateKeys: the transaction contains a duplicate key"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Disputed(description)
}
object InconsistentKeys extends Rejection {
override val description: String =
"InconsistentKeys: the transaction is internally inconsistent"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Disputed(description)
}
}
object ExternallyInconsistentTransaction {
object InconsistentContracts extends Rejection {
override def description: String =
"InconsistentContracts: at least one contract has been archived since the submission"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
object DuplicateKeys extends Rejection {
override val description: String =
"DuplicateKeys: at least one contract key is not unique"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
object InconsistentKeys extends Rejection {
override val description: String =
"InconsistentKeys: at least one contract key has changed since the submission"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
}
final case class MissingInputState(key: DamlStateKey) extends Rejection {
override lazy val description: String =
s"Missing input state for key $key"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Inconsistent(description)
}
final case class InvalidParticipantState(error: Err) extends Rejection {
override lazy val description: String =
error.getMessage
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.Disputed(description)
}
final case class LedgerTimeOutOfRange(
outOfRange: LedgerTimeModel.OutOfRange
) extends Rejection {
override lazy val description: String =
outOfRange.message
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.InvalidLedgerTime(description)
}
final case class RecordTimeOutOfRange(
minimumRecordTime: Instant,
maximumRecordTime: Instant,
) extends Rejection {
override lazy val description: String =
s"Record time is outside of valid range [$minimumRecordTime, $maximumRecordTime]"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.InvalidLedgerTime(description)
}
object CausalMonotonicityViolated extends Rejection {
override val description: String =
"Causal monotonicity violated"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.InvalidLedgerTime(description)
}
final case class SubmittingPartyNotKnownOnLedger(submitter: Ref.Party) extends Rejection {
override lazy val description: String =
s"Submitting party '$submitter' not known"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.PartyNotKnownOnLedger(description)
}
final case class PartiesNotKnownOnLedger(parties: Iterable[Ref.Party]) extends Rejection {
override lazy val description: String =
s"Parties not known on ledger: ${parties.mkString("[", ", ", "]")}"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.PartyNotKnownOnLedger(description)
}
final case class SubmitterCannotActViaParticipant(
submitter: Ref.Party,
participantId: Ref.ParticipantId,
) extends Rejection {
override lazy val description: String =
s"Party '$submitter' not hosted by participant $participantId"
override def toStateV1RejectionReason: v1.RejectionReason =
v1.RejectionReasonV0.SubmitterCannotActViaParticipant(description)
}
}

View File

@ -26,11 +26,11 @@ private[transaction] class Rejections(metrics: Metrics) {
def buildRejectionStep[A](
transactionEntry: DamlTransactionEntrySummary,
reason: RejectionReasonV0,
rejection: Rejection,
recordTime: Option[Timestamp],
)(implicit loggingContext: LoggingContext): StepResult[A] = {
buildRejectionStep(
buildRejectionEntry(transactionEntry, reason),
buildRejectionEntry(transactionEntry, rejection),
recordTime,
)
}
@ -50,15 +50,15 @@ private[transaction] class Rejections(metrics: Metrics) {
def buildRejectionEntry(
transactionEntry: DamlTransactionEntrySummary,
reason: RejectionReasonV0,
rejection: Rejection,
)(implicit loggingContext: LoggingContext): DamlTransactionRejectionEntry.Builder = {
logger.trace(s"Transaction rejected, ${reason.description}.")
logger.trace(s"Transaction rejected, ${rejection.description}.")
val builder = DamlTransactionRejectionEntry.newBuilder
builder
.setSubmitterInfo(transactionEntry.submitterInfo)
reason match {
rejection.toStateV1RejectionReason match {
case RejectionReasonV0.Inconsistent(reason) =>
builder.setInconsistent(Inconsistent.newBuilder.setDetails(reason))
case RejectionReasonV0.Disputed(reason) =>
@ -69,8 +69,7 @@ private[transaction] class Rejections(metrics: Metrics) {
builder.setPartyNotKnownOnLedger(PartyNotKnownOnLedger.newBuilder.setDetails(reason))
case RejectionReasonV0.SubmitterCannotActViaParticipant(details) =>
builder.setSubmitterCannotActViaParticipant(
SubmitterCannotActViaParticipant.newBuilder
.setDetails(details)
SubmitterCannotActViaParticipant.newBuilder.setDetails(details)
)
case RejectionReasonV0.InvalidLedgerTime(reason) =>
builder.setInvalidLedgerTime(InvalidLedgerTime.newBuilder.setDetails(reason))

View File

@ -16,7 +16,6 @@ import com.daml.ledger.participant.state.kvutils.committer.transaction.validatio
TransactionConsistencyValidator,
}
import com.daml.ledger.participant.state.kvutils.{Conversions, Err}
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.lf.data.Ref.Party
import com.daml.lf.engine.{Blinding, Engine}
import com.daml.lf.transaction.{BlindingInfo, TransactionOuterClass}
@ -156,20 +155,14 @@ private[kvutils] class TransactionCommitter(
case Some(_) =>
Some(
rejection(
RejectionReasonV0.SubmitterCannotActViaParticipant(
s"Party '$submitter' not hosted by participant ${commitContext.participantId}"
)
Rejection.SubmitterCannotActViaParticipant(submitter, commitContext.participantId)
)
)
case None =>
Some(
rejection(
RejectionReasonV0.PartyNotKnownOnLedger(s"Submitting party '$submitter' not known")
)
)
Some(rejection(Rejection.SubmittingPartyNotKnownOnLedger(submitter)))
}
def rejection(reason: RejectionReasonV0): StepResult[DamlTransactionEntrySummary] =
def rejection(reason: Rejection): StepResult[DamlTransactionEntrySummary] =
rejections.buildRejectionStep(
transactionEntry,
reason,
@ -288,12 +281,13 @@ private[kvutils] class TransactionCommitter(
transactionEntry: DamlTransactionEntrySummary,
)(implicit loggingContext: LoggingContext): StepResult[DamlTransactionEntrySummary] = {
val parties = transactionEntry.transaction.informees
if (parties.forall(party => commitContext.get(partyStateKey(party)).isDefined))
val missingParties = parties.filter(party => commitContext.get(partyStateKey(party)).isEmpty)
if (missingParties.isEmpty)
StepContinue(transactionEntry)
else
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.PartyNotKnownOnLedger("Not all parties known"),
Rejection.PartiesNotKnownOnLedger(missingParties),
commitContext.recordTime,
)
}

View File

@ -11,11 +11,11 @@ import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.committer.Committer.getCurrentConfiguration
import com.daml.ledger.participant.state.kvutils.committer.transaction.{
DamlTransactionEntrySummary,
Rejection,
Rejections,
Step,
}
import com.daml.ledger.participant.state.kvutils.committer.{CommitContext, StepContinue, StepResult}
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.LoggingContext
@ -42,7 +42,7 @@ private[transaction] class LedgerTimeValidator(defaultConfig: Configuration)
outOfRange =>
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.InvalidLedgerTime(outOfRange.message),
Rejection.LedgerTimeOutOfRange(outOfRange),
commitContext.recordTime,
),
_ => StepContinue(transactionEntry),
@ -68,9 +68,7 @@ private[transaction] class LedgerTimeValidator(defaultConfig: Configuration)
.setTransactionRejectionEntry(
rejections.buildRejectionEntry(
transactionEntry,
RejectionReasonV0.InvalidLedgerTime(
s"Record time is outside of valid range [$minimumRecordTime, $maximumRecordTime]"
),
Rejection.RecordTimeOutOfRange(minimumRecordTime, maximumRecordTime),
)
)
.build

View File

@ -11,12 +11,12 @@ import com.daml.ledger.participant.state.kvutils.Conversions.{
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlContractState, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.committer.transaction.{
DamlTransactionEntrySummary,
Rejection,
Rejections,
Step,
}
import com.daml.ledger.participant.state.kvutils.committer.{CommitContext, StepContinue, StepResult}
import com.daml.ledger.participant.state.kvutils.{Conversions, Err}
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.lf.archive
import com.daml.lf.data.Ref.PackageId
import com.daml.lf.data.Time.Timestamp
@ -101,21 +101,21 @@ private[transaction] class ModelConformanceValidator(engine: Engine, metrics: Me
.map(error =>
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Disputed(error.msg),
Rejection.ValidationFailure(error),
commitContext.recordTime,
)
)
} yield ()
stepResult.fold(identity, _ => StepContinue(transactionEntry))
} catch {
case missingInputErr: Err.MissingInputState =>
case missingInputErr @ Err.MissingInputState(key) =>
logger.error(
"Model conformance validation failed due to a missing input state (most likely due to invalid state on the participant).",
missingInputErr,
)
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Inconsistent(missingInputErr.getMessage),
Rejection.MissingInputState(key),
commitContext.recordTime,
)
case err: Err =>
@ -125,7 +125,7 @@ private[transaction] class ModelConformanceValidator(engine: Engine, metrics: Me
)
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Disputed(err.getMessage),
Rejection.InvalidParticipantState(err),
commitContext.recordTime,
)
}
@ -216,7 +216,7 @@ private[transaction] class ModelConformanceValidator(engine: Engine, metrics: Me
else
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.InvalidLedgerTime("Causal monotonicity violated"),
Rejection.CausalMonotonicityViolated,
commitContext.recordTime,
)
}
@ -231,16 +231,12 @@ private[transaction] object ModelConformanceValidator {
)(
error: KeyInputError
)(implicit loggingContext: LoggingContext): StepResult[DamlTransactionEntrySummary] = {
val description = error match {
val rejection = error match {
case DuplicateKeys(_) =>
"DuplicateKeys: the transaction contains a duplicate key"
Rejection.InternallyInconsistentTransaction.DuplicateKeys
case InconsistentKeys(_) =>
"InconsistentKeys: the transaction is internally inconsistent"
Rejection.InternallyInconsistentTransaction.InconsistentKeys
}
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Disputed(description),
recordTime,
)
rejections.buildRejectionStep(transactionEntry, rejection, recordTime)
}
}

View File

@ -12,11 +12,11 @@ import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
}
import com.daml.ledger.participant.state.kvutils.committer.transaction.{
DamlTransactionEntrySummary,
Rejection,
Rejections,
Step,
}
import com.daml.ledger.participant.state.kvutils.committer.{CommitContext, StepContinue, StepResult}
import com.daml.ledger.participant.state.v1.RejectionReasonV0
import com.daml.lf.transaction.Transaction.{
DuplicateKeys,
InconsistentKeys,
@ -100,17 +100,13 @@ private[transaction] object TransactionConsistencyValidator extends TransactionV
case Right(_) =>
StepContinue(transactionEntry)
case Left(error) =>
val message = error match {
val rejection = error match {
case Duplicate =>
"DuplicateKeys: at least one contract key is not unique"
Rejection.ExternallyInconsistentTransaction.DuplicateKeys
case Inconsistent =>
"InconsistentKeys: at least one contract key has changed since the submission"
Rejection.ExternallyInconsistentTransaction.InconsistentKeys
}
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Inconsistent(message),
commitContext.recordTime,
)
rejections.buildRejectionStep(transactionEntry, rejection, commitContext.recordTime)
}
}
@ -134,9 +130,7 @@ private[transaction] object TransactionConsistencyValidator extends TransactionV
else
rejections.buildRejectionStep(
transactionEntry,
RejectionReasonV0.Inconsistent(
"InconsistentContracts: at least one contract has been archived since the submission"
),
Rejection.ExternallyInconsistentTransaction.InconsistentContracts,
commitContext.recordTime,
)
}

View File

@ -0,0 +1,48 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.platform.sandbox.stores.ledger
import com.daml.ledger.api.domain
import com.daml.ledger.configuration.LedgerTimeModel
import com.daml.ledger.participant.state.v1
import com.daml.platform.store.Conversions.RejectionReasonOps
import io.grpc.Status
sealed trait Rejection {
val reason: String
val code: Status.Code
def description: String
def toDomainRejectionReason: domain.RejectionReason
def toStateV1RejectionReason: v1.RejectionReason =
toDomainRejectionReason.toParticipantStateRejectionReason
}
object Rejection {
object NoLedgerConfiguration extends Rejection {
override val reason: String = "NO_LEDGER_CONFIGURATION"
override val description: String =
"No ledger configuration available, cannot validate ledger time"
override val code: Status.Code = Status.Code.ABORTED
override lazy val toDomainRejectionReason: domain.RejectionReason =
domain.RejectionReason.InvalidLedgerTime(description)
}
final case class InvalidLedgerTime(outOfRange: LedgerTimeModel.OutOfRange) extends Rejection {
override val reason: String = "INVALID_LEDGER_TIME"
override val code: Status.Code = Status.Code.ABORTED
override lazy val description: String = outOfRange.message
override lazy val toDomainRejectionReason: domain.RejectionReason =
domain.RejectionReason.InvalidLedgerTime(description)
}
}

View File

@ -1,23 +0,0 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.platform.sandbox.stores.ledger
import com.daml.ledger.configuration.LedgerTimeModel
private sealed trait TimeModelError {
def message: String
}
private object TimeModelError {
object NoLedgerConfiguration extends TimeModelError {
val message: String =
"No ledger configuration available, cannot validate ledger time"
}
final case class InvalidLedgerTime(outOfRange: LedgerTimeModel.OutOfRange)
extends TimeModelError {
lazy val message: String =
outOfRange.message
}
}

View File

@ -53,7 +53,7 @@ import com.daml.platform.participant.util.LfEngineToApi
import com.daml.platform.sandbox.stores.InMemoryActiveLedgerState
import com.daml.platform.sandbox.stores.ledger.ScenarioLoader.LedgerEntryOrBump
import com.daml.platform.sandbox.stores.ledger.inmemory.InMemoryLedger._
import com.daml.platform.sandbox.stores.ledger.{Ledger, TimeModelError}
import com.daml.platform.sandbox.stores.ledger.{Ledger, Rejection}
import com.daml.platform.store.CompletionFromTransaction
import com.daml.platform.store.Contract.ActiveContract
import com.daml.platform.store.entries.{
@ -289,14 +289,11 @@ private[sandbox] final class InMemoryLedger(
private def checkTimeModel(
ledgerTime: Instant,
recordTime: Instant,
): Either[TimeModelError, Unit] =
): Either[Rejection, Unit] =
ledgerConfiguration
.toRight(TimeModelError.NoLedgerConfiguration)
.toRight(Rejection.NoLedgerConfiguration)
.flatMap(config =>
config.timeModel
.checkTime(ledgerTime, recordTime)
.left
.map(TimeModelError.InvalidLedgerTime)
config.timeModel.checkTime(ledgerTime, recordTime).left.map(Rejection.InvalidLedgerTime)
)
private def handleSuccessfulTx(
@ -309,7 +306,7 @@ private[sandbox] final class InMemoryLedger(
val recordTime = timeProvider.getCurrentTime
checkTimeModel(ledgerTime, recordTime)
.fold(
error => handleError(submitterInfo, RejectionReason.InvalidLedgerTime(error.message)),
rejection => handleError(submitterInfo, rejection.toDomainRejectionReason),
_ => {
val (committedTransaction, disclosureForIndex, divulgence) =
Ledger

View File

@ -19,12 +19,7 @@ import com.daml.ledger.api.health.HealthStatus
import com.daml.ledger.configuration.Configuration
import com.daml.ledger.offset.Offset
import com.daml.ledger.participant.state.index.v2.{ContractStore, PackageDetails}
import com.daml.ledger.participant.state.v1.{
RejectionReasonV0,
SubmissionResult,
SubmitterInfo,
TransactionMeta,
}
import com.daml.ledger.participant.state.v1.{SubmissionResult, SubmitterInfo, TransactionMeta}
import com.daml.ledger.resources.{Resource, ResourceContext, ResourceOwner}
import com.daml.lf.data.{ImmArray, Ref, Time}
import com.daml.lf.engine.{Engine, ValueEnricher}
@ -42,7 +37,7 @@ import com.daml.platform.sandbox.LedgerIdGenerator
import com.daml.platform.sandbox.config.LedgerName
import com.daml.platform.sandbox.stores.ledger.ScenarioLoader.LedgerEntryOrBump
import com.daml.platform.sandbox.stores.ledger.sql.SqlLedger._
import com.daml.platform.sandbox.stores.ledger.{Ledger, SandboxOffset, TimeModelError}
import com.daml.platform.sandbox.stores.ledger.{Ledger, Rejection, SandboxOffset}
import com.daml.platform.store.appendonlydao.events.CompressionStrategy
import com.daml.platform.store.cache.TranslationCacheBackedContractStore
import com.daml.platform.store.dao.{LedgerDao, LedgerWriteDao}
@ -395,16 +390,13 @@ private final class SqlLedger(
private def checkTimeModel(
ledgerTime: Instant,
recordTime: Instant,
): Either[RejectionReasonV0, Unit] = {
): Either[Rejection, Unit] =
currentConfiguration
.get()
.toRight(TimeModelError.NoLedgerConfiguration)
.toRight(Rejection.NoLedgerConfiguration)
.flatMap(
_.timeModel.checkTime(ledgerTime, recordTime).left.map(TimeModelError.InvalidLedgerTime)
_.timeModel.checkTime(ledgerTime, recordTime).left.map(Rejection.InvalidLedgerTime)
)
.left
.map(error => RejectionReasonV0.InvalidLedgerTime(error.message))
}
override def publishTransaction(
submitterInfo: SubmitterInfo,
@ -424,7 +416,7 @@ private final class SqlLedger(
Some(submitterInfo),
recordTime,
CurrentOffset(offset),
reason,
reason.toStateV1RejectionReason,
),
_ => {
val divulgedContracts = Nil