Storage query abstraction first sketch (#9959)

changelog_begin
changelog_end
This commit is contained in:
Marton Nagy 2021-06-14 18:05:57 +02:00 committed by GitHub
parent c75013622c
commit 66d38cd24d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 3120 additions and 2376 deletions

View File

@ -11,7 +11,6 @@ import akka.{Done, NotUsed}
import com.daml.ledger.participant.state.v1.Offset
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.index.BuffersUpdater._
import com.daml.platform.store.appendonlydao.EventSequentialId
import com.daml.platform.store.appendonlydao.events.{Contract, Key, Party}
import com.daml.platform.store.dao.events.ContractStateEvent
import com.daml.platform.store.interfaces.TransactionLogUpdate
@ -48,7 +47,7 @@ private[index] class BuffersUpdater(
private val logger = ContextualizedLogger.get(getClass)
private[index] val updaterIndex = new AtomicReference(
Offset.beforeBegin -> EventSequentialId.beforeBegin
Offset.beforeBegin -> 0L // TODO: append-only: FIXME consolidating parameters table
)
private val (transactionLogUpdatesKillSwitch, transactionLogUpdatesDone) =

View File

@ -17,7 +17,6 @@ import com.daml.metrics.Metrics
import com.daml.platform.akkastreams.dispatcher.Dispatcher
import com.daml.platform.akkastreams.dispatcher.SubSource.RangeSource
import com.daml.platform.index.ReadOnlySqlLedgerWithMutableCache.DispatcherLagMeter
import com.daml.platform.store.appendonlydao.EventSequentialId
import com.daml.platform.store.cache.MutableCacheBackedContractStore.SignalNewLedgerHead
import com.daml.platform.store.cache.{EventsBuffer, MutableCacheBackedContractStore}
import com.daml.platform.store.dao.LedgerReadDao
@ -67,7 +66,8 @@ private[index] object ReadOnlySqlLedgerWithMutableCache {
private def dispatcherOffsetSeqIdOwner(ledgerEnd: Offset, evtSeqId: Long) = {
Dispatcher.owner(
name = "transaction-log-updates",
zeroIndex = (Offset.beforeBegin, EventSequentialId.beforeBegin),
zeroIndex =
(Offset.beforeBegin, 0L), // TODO: append-only: FIXME consolidating parameters table
headAtInitialization = (ledgerEnd, evtSeqId),
)
}

View File

@ -257,7 +257,7 @@ object ParallelIndexerFactory {
)
def ingestTail[DB_BATCH](
ingestTailFunction: (Connection, StorageBackend.Params) => Unit,
ingestTailFunction: StorageBackend.Params => Connection => Unit,
dbDispatcher: DbDispatcher,
metrics: Metrics,
)(implicit loggingContext: LoggingContext): Batch[DB_BATCH] => Future[Batch[DB_BATCH]] =
@ -267,12 +267,11 @@ object ParallelIndexerFactory {
) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.parallelIndexer.tailIngestion) { connection =>
ingestTailFunction(
connection,
StorageBackend.Params(
ledgerEnd = batch.lastOffset,
eventSeqId = batch.lastSeqEventId,
),
)
)
)(connection)
metrics.daml.indexer.ledgerEndSequentialId.updateValue(batch.lastSeqEventId)
metrics.daml.indexer.lastReceivedRecordTime.updateValue(batch.lastRecordTime)
metrics.daml.indexer.lastReceivedOffset.updateValue(batch.lastOffset.toHexString)

View File

@ -23,6 +23,8 @@ import java.io.BufferedReader
import scala.language.implicitConversions
import java.util.stream.Collectors
// TODO append-only: split this file on cleanup, and move anorm/db conversion related stuff to the right place
private[platform] object OracleArrayConversions {
implicit object PartyJsonFormat extends RootJsonFormat[Party] {
def write(c: Party) =

View File

@ -12,21 +12,20 @@ import com.daml.ledger.api.v1.command_completion_service.CompletionStreamRespons
import com.daml.logging.LoggingContext
import com.daml.metrics.Metrics
import com.daml.platform.ApiOffset
import com.daml.platform.store.DbType
import com.daml.platform.store.appendonlydao.events.{QueryNonPruned, SqlFunctions}
import com.daml.platform.store.appendonlydao.events.QueryNonPruned
import com.daml.platform.store.backend.CompletionStorageBackend
import com.daml.platform.store.dao.LedgerDaoCommandCompletionsReader
import scala.concurrent.{ExecutionContext, Future}
private[appendonlydao] final class CommandCompletionsReader(
dispatcher: DbDispatcher,
dbType: DbType,
storageBackend: CompletionStorageBackend,
queryNonPruned: QueryNonPruned,
metrics: Metrics,
executionContext: ExecutionContext,
) extends LedgerDaoCommandCompletionsReader {
private val sqlFunctions = SqlFunctions(dbType)
private def offsetFor(response: CompletionStreamResponse): Offset =
ApiOffset.assertFromString(response.checkpoint.get.offset.get.getAbsolute)
@ -38,19 +37,17 @@ private[appendonlydao] final class CommandCompletionsReader(
)(implicit
loggingContext: LoggingContext
): Source[(Offset, CompletionStreamResponse), NotUsed] = {
val query = CommandCompletionsTable.prepareGet(
startExclusive = startExclusive,
endInclusive = endInclusive,
applicationId = applicationId,
parties = parties,
sqlFunctions = sqlFunctions,
)
Source
.future(
dispatcher
.executeSql(metrics.daml.index.db.getCompletions) { implicit connection =>
QueryNonPruned.executeSql[List[CompletionStreamResponse]](
query.as(CommandCompletionsTable.parser.*),
queryNonPruned.executeSql[List[CompletionStreamResponse]](
storageBackend.commandCompletions(
startExclusive = startExclusive,
endInclusive = endInclusive,
applicationId = applicationId,
parties = parties,
)(connection),
startExclusive,
pruned =>
s"Command completions request from ${startExclusive.toHexString} to ${endInclusive.toHexString} overlaps with pruned offset ${pruned.toHexString}",

View File

@ -1,61 +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.store.appendonlydao
import java.time.Instant
import anorm.{Row, RowParser, SimpleSql, SqlParser, SqlStringInterpolation, ~}
import com.daml.ledger.ApplicationId
import com.daml.ledger.api.v1.command_completion_service.CompletionStreamResponse
import com.daml.ledger.api.v1.completion.Completion
import com.daml.ledger.participant.state.v1.Offset
import com.daml.lf.data.Ref
import com.daml.platform.store.CompletionFromTransaction.toApiCheckpoint
import com.daml.platform.store.Conversions._
import com.daml.platform.store.appendonlydao.events.SqlFunctions
import com.google.rpc.status.Status
private[platform] object CommandCompletionsTable {
import SqlParser.{int, str}
private val sharedColumns: RowParser[Offset ~ Instant ~ String] =
offset("completion_offset") ~ instant("record_time") ~ str("command_id")
private val acceptedCommandParser: RowParser[CompletionStreamResponse] =
sharedColumns ~ str("transaction_id") map {
case offset ~ recordTime ~ commandId ~ transactionId =>
CompletionStreamResponse(
checkpoint = toApiCheckpoint(recordTime, offset),
completions = Seq(Completion(commandId, Some(Status()), transactionId)),
)
}
private val rejectedCommandParser: RowParser[CompletionStreamResponse] =
sharedColumns ~ int("status_code") ~ str("status_message") map {
case offset ~ recordTime ~ commandId ~ statusCode ~ statusMessage =>
CompletionStreamResponse(
checkpoint = toApiCheckpoint(recordTime, offset),
completions = Seq(Completion(commandId, Some(Status(statusCode, statusMessage)))),
)
}
val parser: RowParser[CompletionStreamResponse] = acceptedCommandParser | rejectedCommandParser
def prepareGet(
startExclusive: Offset,
endInclusive: Offset,
applicationId: ApplicationId,
parties: Set[Ref.Party],
sqlFunctions: SqlFunctions,
): SimpleSql[Row] = {
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", parties)
SQL"select completion_offset, record_time, command_id, transaction_id, status_code, status_message from participant_command_completions where completion_offset > $startExclusive and completion_offset <= $endInclusive and application_id = $applicationId and #$submittersInPartiesClause order by completion_offset asc"
}
def prepareCompletionsDelete(endInclusive: Offset): SimpleSql[Row] =
SQL"delete from participant_command_completions where completion_offset <= $endInclusive"
}

View File

@ -4,12 +4,9 @@ package com.daml.platform.store.appendonlydao
import java.sql.Connection
import java.time.Instant
import java.util.Date
import akka.NotUsed
import akka.stream.scaladsl.Source
import anorm.SqlParser._
import anorm.{Macro, RowParser, SQL, SqlParser}
import com.daml.daml_lf_dev.DamlLf.Archive
import com.daml.ledger.api.domain
import com.daml.ledger.api.domain.{LedgerId, ParticipantId, PartyDetails}
@ -35,15 +32,13 @@ import com.daml.metrics.{Metrics, Timed}
import com.daml.platform.configuration.ServerRole
import com.daml.platform.indexer.{CurrentOffset, IncrementalOffsetStep, OffsetStep}
import com.daml.platform.store.Conversions._
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import com.daml.platform.store._
import com.daml.platform.store.appendonlydao.CommandCompletionsTable.prepareCompletionsDelete
import com.daml.platform.store.appendonlydao.events.{
CompressionStrategy,
ContractsReader,
EventsTableDelete,
LfValueTranslation,
PostCommitValidation,
QueryNonPrunedImpl,
TransactionsReader,
}
import com.daml.platform.store.backend.{StorageBackend, UpdateToDbDto}
@ -70,26 +65,8 @@ import scala.concurrent.{ExecutionContext, Future}
import scala.util.Try
import scala.util.control.NonFatal
private final case class ParsedPartyData(
party: String,
displayName: Option[String],
ledgerOffset: Offset,
explicit: Boolean,
isLocal: Boolean,
)
private final case class ParsedPackageData(
packageId: String,
sourceDescription: Option[String],
size: Long,
knownSince: Date,
)
private final case class ParsedCommandData(deduplicateUntil: Instant)
private class JdbcLedgerDao(
dbDispatcher: DbDispatcher,
dbType: DbType,
servicesExecutionContext: ExecutionContext,
eventsPageSize: Int,
eventsProcessingParallelism: Int,
@ -100,47 +77,40 @@ private class JdbcLedgerDao(
enricher: Option[ValueEnricher],
sequentialIndexer: SequentialWriteDao,
participantId: v1.ParticipantId,
storageBackend: StorageBackend[_],
) extends LedgerDao {
import JdbcLedgerDao._
private val queries = dbType match {
case DbType.Postgres => PostgresQueries
case DbType.H2Database => H2DatabaseQueries
case DbType.Oracle => throw new NotImplementedError("not yet supported")
}
private val logger = ContextualizedLogger.get(this.getClass)
override def currentHealth(): HealthStatus = dbDispatcher.currentHealth()
override def lookupLedgerId()(implicit loggingContext: LoggingContext): Future[Option[LedgerId]] =
dbDispatcher.executeSql(metrics.daml.index.db.getLedgerId)(ParametersTable.getLedgerId)
dbDispatcher.executeSql(metrics.daml.index.db.getLedgerId)(storageBackend.ledgerId)
override def lookupParticipantId()(implicit
loggingContext: LoggingContext
): Future[Option[ParticipantId]] =
dbDispatcher.executeSql(metrics.daml.index.db.getParticipantId)(
ParametersTable.getParticipantId
)
dbDispatcher.executeSql(metrics.daml.index.db.getParticipantId)(storageBackend.participantId)
/** Defaults to Offset.begin if ledger_end is unset
*/
override def lookupLedgerEnd()(implicit loggingContext: LoggingContext): Future[Offset] =
dbDispatcher.executeSql(metrics.daml.index.db.getLedgerEnd)(ParametersTable.getLedgerEnd)
dbDispatcher.executeSql(metrics.daml.index.db.getLedgerEnd)(storageBackend.ledgerEndOffset)
override def lookupLedgerEndOffsetAndSequentialId()(implicit
loggingContext: LoggingContext
): Future[(Offset, Long)] =
dbDispatcher.executeSql(metrics.daml.index.db.getLedgerEndOffsetAndSequentialId)(
ParametersTable.getLedgerEndOffsetAndSequentialId
storageBackend.ledgerEndOffsetAndSequentialId
)
override def lookupInitialLedgerEnd()(implicit
loggingContext: LoggingContext
): Future[Option[Offset]] =
dbDispatcher.executeSql(metrics.daml.index.db.getInitialLedgerEnd)(
ParametersTable.getInitialLedgerEnd
storageBackend.initialLedgerEnd
)
override def initializeLedger(
@ -148,98 +118,24 @@ private class JdbcLedgerDao(
)(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.initializeLedgerParameters) {
implicit connection =>
queries.enforceSynchronousCommit
ParametersTable.setLedgerId(ledgerId.unwrap)(connection)
storageBackend.enforceSynchronousCommit(connection)
storageBackend.updateLedgerId(ledgerId.unwrap)(connection)
}
override def initializeParticipantId(
participantId: ParticipantId
)(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.initializeParticipantId) { implicit connection =>
queries.enforceSynchronousCommit
ParametersTable.setParticipantId(participantId.unwrap)(connection)
storageBackend.enforceSynchronousCommit(connection)
storageBackend.updateParticipantId(participantId.unwrap)(connection)
}
private val SQL_GET_CONFIGURATION_ENTRIES = SQL(
"""select
| configuration_entries.ledger_offset,
| configuration_entries.recorded_at,
| configuration_entries.submission_id,
| configuration_entries.typ,
| configuration_entries.configuration,
| configuration_entries.rejection_reason
| from
| configuration_entries,
| parameters
| where
| ledger_offset > {startExclusive} and
| ledger_offset <= {endInclusive} and
| parameters.ledger_end >= ledger_offset
| order by ledger_offset asc
| limit {pageSize}
| offset {queryOffset}""".stripMargin
)
private val SQL_GET_LATEST_CONFIGURATION_ENTRY = SQL(
s"""select
| configuration_entries.ledger_offset,
| configuration_entries.recorded_at,
| configuration_entries.submission_id,
| configuration_entries.typ,
| configuration_entries.configuration,
| configuration_entries.rejection_reason
| from
| configuration_entries,
| parameters
| where
| configuration_entries.typ = '$acceptType' and
| parameters.ledger_end >= ledger_offset
| order by ledger_offset desc
| limit 1""".stripMargin
)
private def lookupLedgerConfiguration(connection: Connection): Option[(Offset, Configuration)] =
SQL_GET_LATEST_CONFIGURATION_ENTRY
.on()
.asVectorOf(configurationEntryParser)(connection)
.collectFirst { case (offset, ConfigurationEntry.Accepted(_, configuration)) =>
offset -> configuration
}
override def lookupLedgerConfiguration()(implicit
loggingContext: LoggingContext
): Future[Option[(Offset, Configuration)]] =
dbDispatcher.executeSql(metrics.daml.index.db.lookupConfiguration)(lookupLedgerConfiguration)
private val configurationEntryParser: RowParser[(Offset, ConfigurationEntry)] =
(offset("ledger_offset") ~
str("typ") ~
str("submission_id") ~
str("rejection_reason").map(s => if (s.isEmpty) null else s).? ~
byteArray("configuration"))
.map(flatten)
.map { case (offset, typ, submissionId, rejectionReason, configBytes) =>
val config = Configuration
.decode(configBytes)
.fold(err => sys.error(s"Failed to decode configuration: $err"), identity)
offset ->
(typ match {
case `acceptType` =>
ConfigurationEntry.Accepted(
submissionId = submissionId,
configuration = config,
)
case `rejectType` =>
ConfigurationEntry.Rejected(
submissionId = submissionId,
rejectionReason = rejectionReason.getOrElse("<missing reason>"),
proposedConfiguration = config,
)
case _ =>
sys.error(s"getConfigurationEntries: Unknown configuration entry type: $typ")
})
}
dbDispatcher.executeSql(metrics.daml.index.db.lookupConfiguration)(
storageBackend.ledgerConfiguration
)
override def getConfigurationEntries(
startExclusive: Offset,
@ -248,15 +144,12 @@ private class JdbcLedgerDao(
PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset.toString) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadConfigurationEntries) {
implicit connection =>
SQL_GET_CONFIGURATION_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> PageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(configurationEntryParser)
storageBackend.configurationEntries(
startExclusive = startExclusive,
endInclusive = endInclusive,
pageSize = PageSize,
queryOffset = queryOffset,
)
}
}
}
@ -273,7 +166,7 @@ private class JdbcLedgerDao(
dbDispatcher.executeSql(
metrics.daml.index.db.storeConfigurationEntryDbMetrics
) { implicit conn =>
val optCurrentConfig = lookupLedgerConfiguration(conn)
val optCurrentConfig = storageBackend.ledgerConfiguration(conn)
val optExpectedGeneration: Option[Long] =
optCurrentConfig.map { case (_, c) => c.generation + 1 }
val finalRejectionReason: Option[String] =
@ -321,7 +214,7 @@ private class JdbcLedgerDao(
sequentialIndexer.store(conn, offset, Some(update))
PersistenceResponse.Ok
}).recover {
case NonFatal(e) if e.getMessage.contains(queries.DUPLICATE_KEY_ERROR) =>
case NonFatal(e) if e.getMessage.contains(storageBackend.duplicateKeyError) =>
logger.warn(s"Ignoring duplicate configuration submission, submissionId=$submissionId")
conn.rollback(savepoint)
sequentialIndexer.store(
@ -360,7 +253,7 @@ private class JdbcLedgerDao(
)
PersistenceResponse.Ok
}).recover {
case NonFatal(e) if e.getMessage.contains(queries.DUPLICATE_KEY_ERROR) =>
case NonFatal(e) if e.getMessage.contains(storageBackend.duplicateKeyError) =>
logger.warn(
s"Ignoring duplicate party submission with ID ${partyDetails.party} for submissionId $submissionIdOpt"
)
@ -387,72 +280,20 @@ private class JdbcLedgerDao(
}
}
private val SQL_GET_PARTY_ENTRIES = SQL(
"select * from party_entries where ledger_offset>{startExclusive} and ledger_offset<={endInclusive} order by ledger_offset asc limit {pageSize} offset {queryOffset}"
)
private val partyEntryParser: RowParser[(Offset, PartyLedgerEntry)] =
(offset("ledger_offset") ~
date("recorded_at") ~
ledgerString("submission_id").? ~
party("party").? ~
str("display_name").? ~
str("typ") ~
str("rejection_reason").? ~
bool("is_local").?)
.map(flatten)
.map {
case (
offset,
recordTime,
submissionIdOpt,
Some(party),
displayNameOpt,
`acceptType`,
None,
Some(isLocal),
) =>
offset ->
PartyLedgerEntry.AllocationAccepted(
submissionIdOpt,
recordTime.toInstant,
PartyDetails(party, displayNameOpt, isLocal),
)
case (
offset,
recordTime,
Some(submissionId),
None,
None,
`rejectType`,
Some(reason),
None,
) =>
offset -> PartyLedgerEntry.AllocationRejected(
submissionId,
recordTime.toInstant,
reason,
)
case invalidRow =>
sys.error(s"getPartyEntries: invalid party entry row: $invalidRow")
}
override def getPartyEntries(
startExclusive: Offset,
endInclusive: Offset,
)(implicit loggingContext: LoggingContext): Source[(Offset, PartyLedgerEntry), NotUsed] = {
PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset.toString) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadPartyEntries) { implicit connection =>
SQL_GET_PARTY_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> PageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(partyEntryParser)
}
dbDispatcher.executeSql(metrics.daml.index.db.loadPartyEntries)(
storageBackend.partyEntries(
startExclusive = startExclusive,
endInclusive = endInclusive,
pageSize = PageSize,
queryOffset = queryOffset,
)
)
}
}
}
@ -554,7 +395,7 @@ private class JdbcLedgerDao(
logger.info("Storing initial state")
dbDispatcher.executeSql(metrics.daml.index.db.storeInitialStateFromScenario) {
implicit connection =>
queries.enforceSynchronousCommit
storageBackend.enforceSynchronousCommit(connection)
ledgerEntries.foreach { case (offset, entry) =>
entry match {
case tx: LedgerEntry.Transaction =>
@ -609,11 +450,6 @@ private class JdbcLedgerDao(
private val PageSize = 100
private val SQL_SELECT_ALL_PARTIES =
SQL(
"select parties.party, parties.display_name, parties.ledger_offset, parties.explicit, parties.is_local from parties, parameters where parameters.ledger_end >= parties.ledger_offset"
)
override def getParties(
parties: Seq[Party]
)(implicit loggingContext: LoggingContext): Future[List[PartyDetails]] =
@ -621,73 +457,25 @@ private class JdbcLedgerDao(
Future.successful(List.empty)
else
dbDispatcher
.executeSql(metrics.daml.index.db.loadParties) { implicit conn =>
selectParties(parties)
}
.map(_.map(constructPartyDetails))(servicesExecutionContext)
.executeSql(metrics.daml.index.db.loadParties)(storageBackend.parties(parties))
override def listKnownParties()(implicit
loggingContext: LoggingContext
): Future[List[PartyDetails]] =
dbDispatcher
.executeSql(metrics.daml.index.db.loadAllParties) { implicit conn =>
SQL_SELECT_ALL_PARTIES
.as(PartyDataParser.*)
}
.map(_.map(constructPartyDetails))(servicesExecutionContext)
private val SQL_SELECT_PACKAGES =
SQL(
"""select packages.package_id, packages.source_description, packages.known_since, packages.size
|from packages, parameters
|where packages.ledger_offset <= parameters.ledger_end
|""".stripMargin
)
private val SQL_SELECT_PACKAGE =
SQL("""select packages.package
|from packages, parameters
|where package_id = {package_id}
|and packages.ledger_offset <= parameters.ledger_end
|""".stripMargin)
private val PackageDataParser: RowParser[ParsedPackageData] =
Macro.parser[ParsedPackageData](
"package_id",
"source_description",
"size",
"known_since",
)
.executeSql(metrics.daml.index.db.loadAllParties)(storageBackend.knownParties)
override def listLfPackages()(implicit
loggingContext: LoggingContext
): Future[Map[PackageId, PackageDetails]] =
dbDispatcher
.executeSql(metrics.daml.index.db.loadPackages) { implicit conn =>
SQL_SELECT_PACKAGES
.as(PackageDataParser.*)
}
.map(
_.map(d =>
PackageId.assertFromString(d.packageId) -> PackageDetails(
d.size,
d.knownSince.toInstant,
d.sourceDescription,
)
).toMap
)(servicesExecutionContext)
.executeSql(metrics.daml.index.db.loadPackages)(storageBackend.lfPackages)
override def getLfArchive(
packageId: PackageId
)(implicit loggingContext: LoggingContext): Future[Option[Archive]] =
dbDispatcher
.executeSql(metrics.daml.index.db.loadArchive) { implicit conn =>
SQL_SELECT_PACKAGE
.on(
"package_id" -> packageId
)
.as[Option[Array[Byte]]](SqlParser.byteArray("package").singleOpt)
}
.executeSql(metrics.daml.index.db.loadArchive)(storageBackend.lfArchive(packageId))
.map(_.map(data => Archive.parseFrom(Decode.damlLfCodedInputStreamFromBytes(data))))(
servicesExecutionContext
)
@ -754,157 +542,76 @@ private class JdbcLedgerDao(
}
}
private val SQL_GET_PACKAGE_ENTRIES = SQL(
"select * from package_entries where ledger_offset>{startExclusive} and ledger_offset<={endInclusive} order by ledger_offset asc limit {pageSize} offset {queryOffset}"
)
private val packageEntryParser: RowParser[(Offset, PackageLedgerEntry)] =
(offset("ledger_offset") ~
date("recorded_at") ~
ledgerString("submission_id").? ~
str("typ") ~
str("rejection_reason").?)
.map(flatten)
.map {
case (offset, recordTime, Some(submissionId), `acceptType`, None) =>
offset ->
PackageLedgerEntry.PackageUploadAccepted(submissionId, recordTime.toInstant)
case (offset, recordTime, Some(submissionId), `rejectType`, Some(reason)) =>
offset ->
PackageLedgerEntry.PackageUploadRejected(submissionId, recordTime.toInstant, reason)
case invalidRow =>
sys.error(s"packageEntryParser: invalid party entry row: $invalidRow")
}
override def getPackageEntries(
startExclusive: Offset,
endInclusive: Offset,
)(implicit loggingContext: LoggingContext): Source[(Offset, PackageLedgerEntry), NotUsed] =
PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset.toString) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadPackageEntries) { implicit connection =>
SQL_GET_PACKAGE_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> PageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(packageEntryParser)
}
dbDispatcher.executeSql(metrics.daml.index.db.loadPackageEntries)(
storageBackend.packageEntries(
startExclusive = startExclusive,
endInclusive = endInclusive,
pageSize = PageSize,
queryOffset = queryOffset,
)
)
}
}
private val SQL_SELECT_COMMAND = SQL("""
|select deduplicate_until
|from participant_command_submissions
|where deduplication_key = {deduplicationKey}
""".stripMargin)
private val CommandDataParser: RowParser[ParsedCommandData] =
Macro.parser[ParsedCommandData](
"deduplicate_until"
)
override def deduplicateCommand(
commandId: domain.CommandId,
submitters: List[Ref.Party],
submittedAt: Instant,
deduplicateUntil: Instant,
)(implicit loggingContext: LoggingContext): Future[CommandDeduplicationResult] =
dbDispatcher.executeSql(metrics.daml.index.db.deduplicateCommandDbMetrics) { implicit conn =>
dbDispatcher.executeSql(metrics.daml.index.db.deduplicateCommandDbMetrics) { conn =>
val key = DeduplicationKeyMaker.make(commandId, submitters)
// Insert a new deduplication entry, or update an expired entry
val updated = SQL(queries.SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt,
"deduplicateUntil" -> deduplicateUntil,
)
.executeUpdate()
val updated = storageBackend.upsertDeduplicationEntry(
key = key,
submittedAt = submittedAt,
deduplicateUntil = deduplicateUntil,
)(conn)
if (updated == 1) {
// New row inserted, this is the first time the command is submitted
CommandDeduplicationNew
} else {
// Deduplication row already exists
val result = SQL_SELECT_COMMAND
.on("deduplicationKey" -> key)
.as(CommandDataParser.single)
CommandDeduplicationDuplicate(result.deduplicateUntil)
CommandDeduplicationDuplicate(storageBackend.deduplicatedUntil(key)(conn))
}
}
private val SQL_DELETE_EXPIRED_COMMANDS = SQL("""
|delete from participant_command_submissions
|where deduplicate_until < {currentTime}
""".stripMargin)
override def removeExpiredDeduplicationData(
currentTime: Instant
)(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.removeExpiredDeduplicationDataDbMetrics) {
implicit conn =>
SQL_DELETE_EXPIRED_COMMANDS
.on("currentTime" -> currentTime)
.execute()
()
}
private val SQL_DELETE_COMMAND = SQL("""
|delete from participant_command_submissions
|where deduplication_key = {deduplicationKey}
""".stripMargin)
private[this] def stopDeduplicatingCommandSync(
commandId: domain.CommandId,
submitters: List[Party],
)(implicit conn: Connection): Unit = {
val key = DeduplicationKeyMaker.make(commandId, submitters)
SQL_DELETE_COMMAND
.on("deduplicationKey" -> key)
.execute()
()
}
dbDispatcher.executeSql(metrics.daml.index.db.removeExpiredDeduplicationDataDbMetrics)(
storageBackend.removeExpiredDeduplicationData(currentTime)
)
override def stopDeduplicatingCommand(
commandId: domain.CommandId,
submitters: List[Party],
)(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.stopDeduplicatingCommandDbMetrics) {
implicit conn =>
stopDeduplicatingCommandSync(commandId, submitters)
}
private val SQL_UPDATE_MOST_RECENT_PRUNING = SQL("""
|update parameters set participant_pruned_up_to_inclusive={pruned_up_to_inclusive}
|where participant_pruned_up_to_inclusive < {pruned_up_to_inclusive} or participant_pruned_up_to_inclusive is null
|""".stripMargin)
private def updateMostRecentPruning(
prunedUpToInclusive: Offset
)(implicit conn: Connection): Unit = {
SQL_UPDATE_MOST_RECENT_PRUNING
.on("pruned_up_to_inclusive" -> prunedUpToInclusive)
.execute()
()
)(implicit loggingContext: LoggingContext): Future[Unit] = {
val key = DeduplicationKeyMaker.make(commandId, submitters)
dbDispatcher.executeSql(metrics.daml.index.db.stopDeduplicatingCommandDbMetrics)(
storageBackend.stopDeduplicatingCommand(key)
)
}
override def prune(
pruneUpToInclusive: Offset
)(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.pruneDbMetrics) { implicit conn =>
EventsTableDelete.prepareEventsDelete(pruneUpToInclusive).execute()
prepareCompletionsDelete(pruneUpToInclusive).execute()
updateMostRecentPruning(pruneUpToInclusive)
dbDispatcher.executeSql(metrics.daml.index.db.pruneDbMetrics) { conn =>
storageBackend.pruneEvents(pruneUpToInclusive)(conn)
storageBackend.pruneCompletions(pruneUpToInclusive)(conn)
storageBackend.updatePrunedUptoInclusive(pruneUpToInclusive)(conn)
logger.info(s"Pruned ledger api server index db up to ${pruneUpToInclusive.toHexString}")
}
override def reset()(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.truncateAllTables) { implicit conn =>
val _ = SQL(queries.SQL_TRUNCATE_TABLES).execute()
}
dbDispatcher.executeSql(metrics.daml.index.db.truncateAllTables)(storageBackend.reset)
private val translation: LfValueTranslation =
new LfValueTranslation(
@ -914,10 +621,13 @@ private class JdbcLedgerDao(
loadPackage = (packageId, loggingContext) => this.getLfArchive(packageId)(loggingContext),
)
private val queryNonPruned = QueryNonPrunedImpl(storageBackend)
override val transactionsReader: TransactionsReader =
new TransactionsReader(
dispatcher = dbDispatcher,
dbType = dbType,
queryNonPruned = queryNonPruned,
storageBackend = storageBackend,
pageSize = eventsPageSize,
eventProcessingParallelism = eventsProcessingParallelism,
metrics = metrics,
@ -927,16 +637,22 @@ private class JdbcLedgerDao(
)
override val contractsReader: ContractsReader =
ContractsReader(dbDispatcher, dbType, metrics)(
ContractsReader(dbDispatcher, metrics, storageBackend)(
servicesExecutionContext
)
override val completions: CommandCompletionsReader =
new CommandCompletionsReader(dbDispatcher, dbType, metrics, servicesExecutionContext)
new CommandCompletionsReader(
dbDispatcher,
storageBackend,
queryNonPruned,
metrics,
servicesExecutionContext,
)
private val postCommitValidation =
if (performPostCommitValidation)
new PostCommitValidation.BackedBy(contractsReader.committedContracts, validatePartyAllocation)
new PostCommitValidation.BackedBy(storageBackend, validatePartyAllocation)
else
PostCommitValidation.Skip
@ -1004,7 +720,7 @@ private class JdbcLedgerDao(
private[this] def validateOffsetStep(offsetStep: OffsetStep, conn: Connection): Offset = {
offsetStep match {
case IncrementalOffsetStep(p, o) =>
val actualEnd = ParametersTable.getLedgerEnd(conn)
val actualEnd = storageBackend.ledgerEndOffset(conn)
if (actualEnd.compareTo(p) != 0) throw LedgerEndUpdateError(p) else o
case CurrentOffset(o) => o
}
@ -1122,14 +838,14 @@ private[platform] object JdbcLedgerDao {
}
private def sequentialWriteDao(
dbType: DbType,
participantId: v1.ParticipantId,
lfValueTranslationCache: LfValueTranslationCache.Cache,
metrics: Metrics,
compressionStrategy: CompressionStrategy,
storageBackend: StorageBackend[_],
): SequentialWriteDao =
SequentialWriteDaoImpl(
storageBackend = StorageBackend.of(dbType),
storageBackend = storageBackend,
updateToDbDtos = UpdateToDbDto(
participantId = participantId,
translation = new LfValueTranslation(
@ -1142,30 +858,6 @@ private[platform] object JdbcLedgerDao {
),
)
private[appendonlydao] def selectParties(
parties: Seq[Party]
)(implicit connection: Connection): List[ParsedPartyData] =
SQL_SELECT_MULTIPLE_PARTIES
.on("parties" -> parties)
.as(PartyDataParser.*)
private[appendonlydao] def constructPartyDetails(data: ParsedPartyData): PartyDetails =
PartyDetails(Party.assertFromString(data.party), data.displayName, data.isLocal)
private val SQL_SELECT_MULTIPLE_PARTIES =
SQL(
"select parties.party, parties.display_name, parties.ledger_offset, parties.explicit, parties.is_local from parties, parameters where party in ({parties}) and parties.ledger_offset <= parameters.ledger_end"
)
private val PartyDataParser: RowParser[ParsedPartyData] =
Macro.parser[ParsedPartyData](
"party",
"display_name",
"ledger_offset",
"explicit",
"is_local",
)
private def owner(
serverRole: ServerRole,
jdbcUrl: String,
@ -1193,9 +885,9 @@ private[platform] object JdbcLedgerDao {
jdbcAsyncCommitMode,
)
dbType = DbType.jdbcType(jdbcUrl)
storageBackend = StorageBackend.of(dbType)
} yield new JdbcLedgerDao(
dbDispatcher,
dbType,
servicesExecutionContext,
eventsPageSize,
eventsProcessingParallelism,
@ -1205,101 +897,50 @@ private[platform] object JdbcLedgerDao {
validatePartyAllocation,
enricher,
sequentialWriteDao(
dbType,
participantId,
lfValueTranslationCache,
metrics,
compressionStrategy,
storageBackend,
),
participantId,
storageBackend,
)
sealed trait Queries {
protected[JdbcLedgerDao] def enforceSynchronousCommit(implicit conn: Connection): Unit
protected[JdbcLedgerDao] def SQL_INSERT_COMMAND: String
protected[JdbcLedgerDao] def SQL_TRUNCATE_TABLES: String
// TODO: Avoid brittleness of error message checks
protected[JdbcLedgerDao] def DUPLICATE_KEY_ERROR: String
}
object PostgresQueries extends Queries {
override protected[JdbcLedgerDao] val SQL_INSERT_COMMAND: String =
"""insert into participant_command_submissions as pcs (deduplication_key, deduplicate_until)
|values ({deduplicationKey}, {deduplicateUntil})
|on conflict (deduplication_key)
| do update
| set deduplicate_until={deduplicateUntil}
| where pcs.deduplicate_until < {submittedAt}""".stripMargin
override protected[JdbcLedgerDao] val DUPLICATE_KEY_ERROR: String =
"duplicate key"
override protected[JdbcLedgerDao] val SQL_TRUNCATE_TABLES: String =
"""truncate table configuration_entries cascade;
|truncate table package_entries cascade;
|truncate table parameters cascade;
|truncate table participant_command_completions cascade;
|truncate table participant_command_submissions cascade;
|truncate table participant_events_divulgence cascade;
|truncate table participant_events_create cascade;
|truncate table participant_events_consuming_exercise cascade;
|truncate table participant_events_non_consuming_exercise cascade;
|truncate table parties cascade;
|truncate table party_entries cascade;
""".stripMargin
override protected[JdbcLedgerDao] def enforceSynchronousCommit(implicit
conn: Connection
): Unit = {
val statement =
conn.prepareStatement("SET LOCAL synchronous_commit = 'on'")
try {
statement.execute()
()
} finally {
statement.close()
}
}
}
// TODO H2 support
object H2DatabaseQueries extends Queries {
override protected[JdbcLedgerDao] val SQL_INSERT_COMMAND: String =
"""merge into participant_command_submissions pcs
|using dual on deduplication_key = {deduplicationKey}
|when not matched then
| insert (deduplication_key, deduplicate_until)
| values ({deduplicationKey}, {deduplicateUntil})
|when matched and pcs.deduplicate_until < {submittedAt} then
| update set deduplicate_until={deduplicateUntil}""".stripMargin
override protected[JdbcLedgerDao] val DUPLICATE_KEY_ERROR: String =
"Unique index or primary key violation"
override protected[JdbcLedgerDao] val SQL_TRUNCATE_TABLES: String =
"""set referential_integrity false;
|truncate table configuration_entries;
|truncate table package_entries;
|truncate table parameters;
|truncate table participant_command_completions;
|truncate table participant_command_submissions;
|truncate table participant_events;
|truncate table participant_contracts;
|truncate table participant_contract_witnesses;
|truncate table parties;
|truncate table party_entries;
|set referential_integrity true;
""".stripMargin
/** H2 does not support asynchronous commits */
override protected[JdbcLedgerDao] def enforceSynchronousCommit(implicit
conn: Connection
): Unit = ()
}
// object H2DatabaseQueries extends Queries {
// override protected[JdbcLedgerDao] val SQL_INSERT_COMMAND: String =
// """merge into participant_command_submissions pcs
// |using dual on deduplication_key = {deduplicationKey}
// |when not matched then
// | insert (deduplication_key, deduplicate_until)
// | values ({deduplicationKey}, {deduplicateUntil})
// |when matched and pcs.deduplicate_until < {submittedAt} then
// | update set deduplicate_until={deduplicateUntil}""".stripMargin
//
// override protected[JdbcLedgerDao] val DUPLICATE_KEY_ERROR: String =
// "Unique index or primary key violation"
//
// override protected[JdbcLedgerDao] val SQL_TRUNCATE_TABLES: String =
// """set referential_integrity false;
// |truncate table configuration_entries;
// |truncate table package_entries;
// |truncate table parameters;
// |truncate table participant_command_completions;
// |truncate table participant_command_submissions;
// |truncate table participant_events;
// |truncate table participant_contracts;
// |truncate table participant_contract_witnesses;
// |truncate table parties;
// |truncate table party_entries;
// |set referential_integrity true;
// """.stripMargin
//
// /** H2 does not support asynchronous commits */
// override protected[JdbcLedgerDao] def enforceSynchronousCommit(implicit
// conn: Connection
// ): Unit = ()
// }
val acceptType = "accept"
val rejectType = "reject"

View File

@ -1,86 +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.store.appendonlydao
import java.sql.Connection
import anorm.SqlParser.long
import anorm.{Row, RowParser, SimpleSql, SqlParser, SqlStringInterpolation}
import com.daml.ledger.api.domain.{LedgerId, ParticipantId}
import com.daml.ledger.participant.state.v1.Offset
import com.daml.platform.store.Conversions.{ledgerString, offset, participantId}
import com.daml.scalautil.Statement.discard
private[appendonlydao] object ParametersTable {
private val TableName: String = "parameters"
private val LedgerIdColumnName: String = "ledger_id"
private val ParticipantIdColumnName: String = "participant_id"
private val LedgerEndColumnName: String = "ledger_end"
private val LedgerEndSequentialIdColumnName: String = "ledger_end_sequential_id"
private val LedgerIdParser: RowParser[LedgerId] =
ledgerString(LedgerIdColumnName).map(LedgerId(_))
private val ParticipantIdParser: RowParser[Option[ParticipantId]] =
participantId(ParticipantIdColumnName).map(ParticipantId(_)).?
private val LedgerEndParser: RowParser[Option[Offset]] =
offset(LedgerEndColumnName).?
private val LedgerEndOrBeforeBeginParser: RowParser[Offset] =
LedgerEndParser.map(_.getOrElse(Offset.beforeBegin))
private val LedgerEndOffsetAndSequentialIdParser =
(offset(LedgerEndColumnName).? ~ long(LedgerEndSequentialIdColumnName).?)
.map(SqlParser.flatten)
.map {
case (Some(offset), Some(seqId)) => (offset, seqId)
case (Some(offset), None) => (offset, EventSequentialId.beforeBegin)
case (None, None) => (Offset.beforeBegin, EventSequentialId.beforeBegin)
case (None, Some(_)) =>
throw InvalidLedgerEnd("Parameters table in invalid state: ledger_end is not set")
}
private val SelectLedgerEnd: SimpleSql[Row] = SQL"select #$LedgerEndColumnName from #$TableName"
private val SelectLedgerEndOffsetAndSequentialId =
SQL"select #$LedgerEndColumnName, #$LedgerEndSequentialIdColumnName from #$TableName"
def getLedgerId(connection: Connection): Option[LedgerId] =
SQL"select #$LedgerIdColumnName from #$TableName".as(LedgerIdParser.singleOpt)(connection)
def setLedgerId(ledgerId: String)(connection: Connection): Unit =
discard(
SQL"insert into #$TableName(#$LedgerIdColumnName) values($ledgerId)".execute()(connection)
)
def getParticipantId(connection: Connection): Option[ParticipantId] =
SQL"select #$ParticipantIdColumnName from #$TableName".as(ParticipantIdParser.single)(
connection
)
def setParticipantId(participantId: String)(connection: Connection): Unit =
discard(
SQL"update #$TableName set #$ParticipantIdColumnName = $participantId".execute()(
connection
)
)
def getLedgerEnd(connection: Connection): Offset =
SelectLedgerEnd.as(LedgerEndOrBeforeBeginParser.single)(connection)
// TODO mutable contract state cache - use only one getLedgerEnd method
def getLedgerEndOffsetAndSequentialId(connection: Connection): (Offset, Long) =
SelectLedgerEndOffsetAndSequentialId.as(LedgerEndOffsetAndSequentialIdParser.single)(connection)
def getInitialLedgerEnd(connection: Connection): Option[Offset] =
SelectLedgerEnd.as(LedgerEndParser.single)(connection)
case class InvalidLedgerEnd(msg: String) extends RuntimeException(msg)
}
private[platform] object EventSequentialId {
val beforeBegin = 0L
}

View File

@ -6,7 +6,12 @@ package com.daml.platform.store.appendonlydao
import java.sql.Connection
import com.daml.ledger.participant.state.v1.{Offset, Update}
import com.daml.platform.store.backend.{DbDto, StorageBackend}
import com.daml.platform.store.backend.{
DbDto,
IngestionStorageBackend,
ParameterStorageBackend,
StorageBackend,
}
import scala.util.chaining.scalaUtilChainingOps
@ -15,7 +20,7 @@ trait SequentialWriteDao {
}
case class SequentialWriteDaoImpl[DB_BATCH](
storageBackend: StorageBackend[DB_BATCH],
storageBackend: IngestionStorageBackend[DB_BATCH] with ParameterStorageBackend,
updateToDbDtos: Offset => Update => Iterator[DbDto],
) extends SequentialWriteDao {
@ -55,11 +60,10 @@ case class SequentialWriteDaoImpl[DB_BATCH](
.pipe(storageBackend.insertBatch(connection, _))
storageBackend.updateParams(
connection,
StorageBackend.Params(
ledgerEnd = offset,
eventSeqId = lastEventSeqId,
),
)
)
)(connection)
}
}

View File

@ -4,59 +4,16 @@
package com.daml.platform.store.appendonlydao.events
import java.io.InputStream
import java.sql.Connection
import java.time.Instant
import anorm.SqlParser.{binaryStream, int, long}
import anorm._
import com.daml.ledger.participant.state.v1.Offset
import com.daml.lf.data.Ref
import com.daml.platform.store.Conversions.{contractId, offset, _}
import com.daml.platform.store.appendonlydao.events
import com.daml.platform.store.dao.events.ContractStateEvent
import com.daml.platform.store.serialization.{Compression, ValueSerializer}
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import com.daml.platform.store.LfValueTranslationCache
import com.daml.platform.store.backend.StorageBackend.RawContractStateEvent
import scala.util.control.NoStackTrace
object ContractStateEventsReader {
val contractStateRowParser: RowParser[RawContractStateEvent] =
(int("event_kind") ~
contractId("contract_id") ~
identifier("template_id").? ~
instant("ledger_effective_time").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").? ~
binaryStream("create_argument").? ~
int("create_argument_compression").? ~
long("event_sequential_id") ~
flatEventWitnessesColumn("flat_event_witnesses") ~
offset("event_offset")).map {
case eventKind ~ contractId ~ templateId ~ ledgerEffectiveTime ~ createKeyValue ~ createKeyCompression ~ createArgument ~ createArgumentCompression ~ eventSequentialId ~ flatEventWitnesses ~ offset =>
RawContractStateEvent(
eventKind,
contractId,
templateId,
ledgerEffectiveTime,
createKeyValue,
createKeyCompression,
createArgument,
createArgumentCompression,
flatEventWitnesses,
eventSequentialId,
offset,
)
}
/** This method intentionally produces a generic DTO to perform as much work as possible outside of the db thread pool
* (specifically the translation to the `ContractStateEvent`)
*/
def readRawEvents(range: EventsRange[(Offset, Long)])(implicit
conn: Connection
): Vector[RawContractStateEvent] =
createsAndArchives(EventsRange(range.startExclusive._2, range.endInclusive._2))
.asVectorOf(contractStateRowParser)
def toContractStateEvent(
raw: RawContractStateEvent,
@ -145,29 +102,6 @@ object ContractStateEventsReader {
private def decompressAndDeserialize(algorithm: Compression.Algorithm, value: InputStream) =
ValueSerializer.deserializeValue(algorithm.decompress(value))
private val createsAndArchives: EventsRange[Long] => SimpleSql[Row] =
(range: EventsRange[Long]) => SQL(s"""
|SELECT
| event_kind,
| contract_id,
| template_id,
| create_key_value,
| create_key_value_compression,
| create_argument,
| create_argument_compression,
| flat_event_witnesses,
| ledger_effective_time,
| event_sequential_id,
| event_offset
|FROM
| participant_events
|WHERE
| event_sequential_id > ${range.startExclusive}
| and event_sequential_id <= ${range.endInclusive}
| and (event_kind = 10 or event_kind = 20)
|ORDER BY event_sequential_id ASC
|""".stripMargin)
case class CreateMissingError(field: String) extends NoStackTrace {
override def getMessage: String =
s"Create events should not be missing $field"
@ -178,20 +112,6 @@ object ContractStateEventsReader {
s"Invalid event kind: $eventKind"
}
private[events] case class RawContractStateEvent(
eventKind: Int,
contractId: ContractId,
templateId: Option[Ref.Identifier],
ledgerEffectiveTime: Option[Instant],
createKeyValue: Option[InputStream],
createKeyCompression: Option[Int],
createArgument: Option[InputStream],
createArgumentCompression: Option[Int],
flatEventWitnesses: Set[Party],
eventSequentialId: Long,
offset: Offset,
)
private object EventKind {
val Create = 10
val ConsumingExercise = 20

View File

@ -6,30 +6,22 @@ package com.daml.platform.store.appendonlydao.events
import java.io.InputStream
import java.time.Instant
import anorm.SqlParser._
import anorm.{RowParser, SqlParser, SqlStringInterpolation, _}
import com.codahale.metrics.Timer
import com.daml.logging.LoggingContext
import com.daml.metrics.{Metrics, Timed}
import com.daml.platform.store.Conversions._
import com.daml.platform.store.DbType
import com.daml.platform.store.interfaces.LedgerDaoContractsReader._
import com.daml.platform.store.appendonlydao.events.ContractsReader._
import com.daml.platform.store.appendonlydao.events.SqlFunctions.{
H2SqlFunctions,
PostgresSqlFunctions,
}
import com.daml.platform.store.appendonlydao.DbDispatcher
import com.daml.platform.store.backend.ContractStorageBackend
import com.daml.platform.store.interfaces.LedgerDaoContractsReader
import com.daml.platform.store.serialization.{Compression, ValueSerializer}
import scala.concurrent.{ExecutionContext, Future}
private[appendonlydao] sealed class ContractsReader(
val committedContracts: PostCommitValidationData,
storageBackend: ContractStorageBackend,
dispatcher: DbDispatcher,
metrics: Metrics,
sqlFunctions: SqlFunctions,
)(implicit ec: ExecutionContext)
extends LedgerDaoContractsReader {
@ -39,9 +31,9 @@ private[appendonlydao] sealed class ContractsReader(
Timed.future(
metrics.daml.index.db.lookupMaximumLedgerTime,
dispatcher
.executeSql(metrics.daml.index.db.lookupMaximumLedgerTimeDbMetrics) { implicit connection =>
committedContracts.lookupMaximumLedgerTime(ids)
}
.executeSql(metrics.daml.index.db.lookupMaximumLedgerTimeDbMetrics)(
storageBackend.maximumLedgerTime(ids)
)
.map(_.get),
)
@ -56,35 +48,9 @@ private[appendonlydao] sealed class ContractsReader(
): Future[KeyState] =
Timed.future(
metrics.daml.index.db.lookupKey,
dispatcher.executeSql(metrics.daml.index.db.lookupContractByKeyDbMetrics) {
implicit connection =>
SQL"""
WITH last_contract_key_create AS (
SELECT contract_id, flat_event_witnesses
FROM participant_events
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= $validAt
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id, flat_event_witnesses
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE NOT EXISTS -- check no archival visible
(SELECT 1
FROM participant_events
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= $validAt
AND contract_id = last_contract_key_create.contract_id
);
"""
.as(
(contractId("contract_id") ~ flatEventWitnessesColumn("flat_event_witnesses")).map {
case cId ~ stakeholders => KeyAssigned(cId, stakeholders)
}.singleOpt
)
.getOrElse(KeyUnassigned)
},
dispatcher.executeSql(metrics.daml.index.db.lookupContractByKeyDbMetrics)(
storageBackend.keyState(key, validAt)
),
)
override def lookupContractState(contractId: ContractId, before: Long)(implicit
@ -93,42 +59,20 @@ private[appendonlydao] sealed class ContractsReader(
Timed.future(
metrics.daml.index.db.lookupActiveContract,
dispatcher
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics) { implicit connection =>
SQL"""
SELECT
template_id,
flat_event_witnesses,
create_argument,
create_argument_compression,
event_kind,
ledger_effective_time
FROM participant_events
WHERE
contract_id = $contractId
AND event_sequential_id <= $before
AND (event_kind = 10 OR event_kind = 20)
ORDER BY event_sequential_id DESC
LIMIT 1;
"""
.as(fullDetailsContractRowParser.singleOpt)
}
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics)(
storageBackend.contractState(contractId, before)
)
.map(_.map {
case (
templateId,
stakeholders,
createArgument,
createArgumentCompression,
10,
maybeCreateLedgerEffectiveTime,
) =>
case raw if raw.eventKind == 10 =>
val contract = toContract(
contractId = contractId,
templateId =
assertPresent(templateId)("template_id must be present for a create event"),
createArgument =
assertPresent(createArgument)("create_argument must be present for a create event"),
assertPresent(raw.templateId)("template_id must be present for a create event"),
createArgument = assertPresent(raw.createArgument)(
"create_argument must be present for a create event"
),
createArgumentCompression =
Compression.Algorithm.assertLookup(createArgumentCompression),
Compression.Algorithm.assertLookup(raw.createArgumentCompression),
decompressionTimer =
metrics.daml.index.db.lookupActiveContractDbMetrics.compressionTimer,
deserializationTimer =
@ -136,13 +80,13 @@ private[appendonlydao] sealed class ContractsReader(
)
ActiveContract(
contract,
stakeholders,
assertPresent(maybeCreateLedgerEffectiveTime)(
raw.flatEventWitnesses,
assertPresent(raw.ledgerEffectiveTime)(
"ledger_effective_time must be present for a create event"
),
)
case (_, stakeholders, _, _, 20, _) => ArchivedContract(stakeholders)
case (_, _, _, _, kind, _) => throw ContractsReaderError(s"Unexpected event kind $kind")
case raw if raw.eventKind == 20 => ArchivedContract(raw.flatEventWitnesses)
case raw => throw ContractsReaderError(s"Unexpected event kind ${raw.eventKind}")
}),
)
@ -155,17 +99,16 @@ private[appendonlydao] sealed class ContractsReader(
Timed.future(
metrics.daml.index.db.lookupActiveContract,
dispatcher
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics) { implicit connection =>
lookupActiveContractAndLoadArgumentQuery(contractId, readers)
.as(contractRowParser.singleOpt)
}
.map(_.map { case (templateId, createArgument, createArgumentCompression) =>
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics)(
storageBackend.activeContractWithArgument(readers, contractId)
)
.map(_.map { raw =>
toContract(
contractId = contractId,
templateId = templateId,
createArgument = createArgument,
templateId = raw.templateId,
createArgument = raw.createArgument,
createArgumentCompression =
Compression.Algorithm.assertLookup(createArgumentCompression),
Compression.Algorithm.assertLookup(raw.createArgumentCompression),
decompressionTimer =
metrics.daml.index.db.lookupActiveContractDbMetrics.compressionTimer,
deserializationTimer =
@ -185,10 +128,9 @@ private[appendonlydao] sealed class ContractsReader(
Timed.future(
metrics.daml.index.db.lookupActiveContract,
dispatcher
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics) { implicit connection =>
lookupActiveContractWithCachedArgumentQuery(contractId, readers)
.as(contractWithoutValueRowParser.singleOpt)
}
.executeSql(metrics.daml.index.db.lookupActiveContractDbMetrics)(
storageBackend.activeContractWithoutArgument(readers, contractId)
)
.map(
_.map(templateId =>
toContract(
@ -206,208 +148,23 @@ private[appendonlydao] sealed class ContractsReader(
)(implicit loggingContext: LoggingContext): Future[Option[ContractId]] =
Timed.future(
metrics.daml.index.db.lookupKey,
dispatcher.executeSql(metrics.daml.index.db.lookupContractByKeyDbMetrics) {
implicit connection =>
lookupContractKeyQuery(readers, key).as(contractId("contract_id").singleOpt)
},
dispatcher.executeSql(metrics.daml.index.db.lookupContractByKeyDbMetrics)(
storageBackend.contractKey(readers, key)
),
)
private def lookupActiveContractAndLoadArgumentQuery(
contractId: ContractId,
readers: Set[Party],
) = {
val tree_event_witnessesWhere =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", readers)
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere -- only use visible archivals
LIMIT 1
),
create_event AS (
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
-- no visibility check, as it is used to backfill missing template_id and create_arguments for divulged contracts
create_event_unrestricted AS (
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgance_event.template_id and .create_argument can be NULL
-- for certain integrations. For example, the KV integration exploits that
-- every participant node knows about all create events. The integration
-- therefore only communicates the change in visibility to the IndexDB, but
-- does not include a full divulgence event.
COALESCE(divulgence_events.template_id, create_event_unrestricted.template_id),
COALESCE(divulgence_events.create_argument, create_event_unrestricted.create_argument),
COALESCE(divulgence_events.create_argument_compression, create_event_unrestricted.create_argument_compression)
FROM participant_events AS divulgence_events LEFT OUTER JOIN create_event_unrestricted USING (contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_kind = 0 -- divulgence
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere
ORDER BY divulgence_events.event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgance events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;"""
}
private def lookupActiveContractWithCachedArgumentQuery(
contractId: ContractId,
readers: Set[Party],
) = {
val tree_event_witnessesWhere =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", readers)
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere -- only use visible archivals
LIMIT 1
),
create_event AS (
SELECT contract_id, template_id
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
-- no visibility check, as it is used to backfill missing template_id and create_arguments for divulged contracts
create_event_unrestricted AS (
SELECT contract_id, template_id
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgance_event.template_id can be NULL
-- for certain integrations. For example, the KV integration exploits that
-- every participant node knows about all create events. The integration
-- therefore only communicates the change in visibility to the IndexDB, but
-- does not include a full divulgence event.
COALESCE(divulgence_events.template_id, create_event_unrestricted.template_id)
FROM participant_events AS divulgence_events LEFT OUTER JOIN create_event_unrestricted USING (contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_kind = 0 -- divulgence
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND #$tree_event_witnessesWhere
ORDER BY divulgence_events.event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, template_id
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;
"""
}
private def lookupContractKeyQuery(readers: Set[Party], key: Key): SimpleSql[Row] = {
def flat_event_witnessesWhere(columnPrefix: String) =
sqlFunctions.arrayIntersectionWhereClause(s"$columnPrefix.flat_event_witnesses", readers)
SQL"""
WITH last_contract_key_create AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= parameters.ledger_end_sequential_id
-- do NOT check visibility here, as otherwise we do not abort the scan early
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE #${flat_event_witnessesWhere("last_contract_key_create")} -- check visibility only here
AND NOT EXISTS -- check no archival visible
(SELECT 1
FROM participant_events, parameters
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #${flat_event_witnessesWhere("participant_events")}
AND contract_id = last_contract_key_create.contract_id
);
"""
}
}
private[appendonlydao] object ContractsReader {
private val contractWithoutValueRowParser: RowParser[String] =
str("template_id")
private val fullDetailsContractRowParser: RowParser[
(Option[String], Set[Party], Option[InputStream], Option[Int], Int, Option[Instant])
] =
str("template_id").? ~ flatEventWitnessesColumn("flat_event_witnesses") ~ binaryStream(
"create_argument"
).? ~ int(
"create_argument_compression"
).? ~ int("event_kind") ~ get[Instant](
"ledger_effective_time"
)(anorm.Column.columnToInstant).? map SqlParser.flatten
private val contractRowParser: RowParser[(String, InputStream, Option[Int])] =
str("template_id") ~ binaryStream("create_argument") ~ int(
"create_argument_compression"
).? map SqlParser.flatten
private[appendonlydao] def apply(
dispatcher: DbDispatcher,
dbType: DbType,
metrics: Metrics,
storageBackend: ContractStorageBackend,
)(implicit ec: ExecutionContext): ContractsReader = {
def sqlFunctions = dbType match {
case DbType.Postgres => PostgresSqlFunctions
case DbType.H2Database => H2SqlFunctions
case DbType.Oracle => throw new NotImplementedError("not yet supported")
}
new ContractsReader(
committedContracts = ContractsTable,
storageBackend = storageBackend,
dispatcher = dispatcher,
metrics = metrics,
sqlFunctions = sqlFunctions,
)
}

View File

@ -1,119 +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.store.appendonlydao.events
import java.sql.Connection
import java.time.Instant
import anorm.SqlStringInterpolation
import com.daml.ledger.api.domain.PartyDetails
import com.daml.platform.store.Conversions._
import com.daml.platform.store.appendonlydao.JdbcLedgerDao
import scala.util.{Failure, Success, Try}
private[events] object ContractsTable extends PostCommitValidationData {
override final def lookupContractKeyGlobally(
key: Key
)(implicit connection: Connection): Option[ContractId] =
SQL"""
WITH last_contract_key_create AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= parameters.ledger_end_sequential_id
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE NOT EXISTS
(SELECT 1
FROM participant_events, parameters
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND contract_id = last_contract_key_create.contract_id
);
"""
.as(contractId("contract_id").singleOpt)
// TODO append-only: revisit this approach when doing cleanup, so we can decide if it is enough or not.
override final def lookupMaximumLedgerTime(
ids: Set[ContractId]
)(implicit connection: Connection): Try[Option[Instant]] = {
if (ids.isEmpty) {
Failure(ContractsTable.emptyContractIds)
} else {
def lookup(id: ContractId): Option[Option[Instant]] =
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1
),
create_event AS (
SELECT ledger_effective_time
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT ledger_effective_time
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 0 -- divulgence
AND event_sequential_id <= parameters.ledger_end_sequential_id
ORDER BY event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgance events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT ledger_effective_time
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;
""".as(instant("ledger_effective_time").?.singleOpt)
val queriedIds: List[(ContractId, Option[Option[Instant]])] = ids.toList
.map(id => id -> lookup(id))
val foundLedgerEffectiveTimes: List[Option[Instant]] = queriedIds
.collect { case (_, Some(found)) =>
found
}
if (foundLedgerEffectiveTimes.size != ids.size) {
val missingIds = queriedIds.collect { case (missingId, None) =>
missingId
}
Failure(ContractsTable.notFound(missingIds.toSet))
} else Success(foundLedgerEffectiveTimes.max)
}
}
override final def lookupParties(parties: Seq[Party])(implicit
connection: Connection
): List[PartyDetails] =
JdbcLedgerDao.selectParties(parties).map(JdbcLedgerDao.constructPartyDetails)
private def emptyContractIds: Throwable =
new IllegalArgumentException(
"Cannot lookup the maximum ledger time for an empty set of contract identifiers"
)
private def notFound(missingContractIds: Set[ContractId]): Throwable =
new IllegalArgumentException(
s"The following contracts have not been found: ${missingContractIds.map(_.coid).mkString(", ")}"
)
}

View File

@ -2,8 +2,8 @@
// SPDX-License-Identifier: Apache-2.0
package com.daml.platform.store.appendonlydao.events
import anorm.SqlParser.get
import anorm.{Row, RowParser, SimpleSql, SqlStringInterpolation}
import java.sql.Connection
import com.daml.ledger.participant.state.v1.Offset
// (startExclusive, endInclusive]
@ -23,71 +23,68 @@ private[events] object EventsRange {
A.gteq(range.startExclusive, range.endInclusive)
}
/** Converts Offset range to Event Sequential ID range.
*
* @param range offset range
* @param connection SQL connection
* @return Event Sequential ID range
*/
def readEventSeqIdRange(range: EventsRange[Offset])(
connection: java.sql.Connection
): EventsRange[Long] =
if (isEmpty(range))
EmptyEventSeqIdRange
else
EventsRange(
startExclusive = readEventSeqId(range.startExclusive)(connection),
endInclusive = readEventSeqId(range.endInclusive)(connection),
)
final class EventSeqIdReader(lookupEventSeqId: Offset => Connection => Option[Long]) {
/** Converts ledger end offset to Event Sequential ID.
*
* @param endInclusive ledger end offset
* @param connection SQL connection
* @return Event Sequential ID range.
*/
def readEventSeqIdRange(endInclusive: Offset)(
connection: java.sql.Connection
): EventsRange[Long] = {
if (endInclusive == Offset.beforeBegin) EmptyEventSeqIdRange
else EmptyEventSeqIdRange.copy(endInclusive = readEventSeqId(endInclusive)(connection))
}
/** Converts Offset range to Event Sequential ID range.
*
* @param range offset range
* @param connection SQL connection
* @return Event Sequential ID range
*/
def readEventSeqIdRange(range: EventsRange[Offset])(
connection: java.sql.Connection
): EventsRange[Long] =
if (isEmpty(range))
EmptyEventSeqIdRange
else
EventsRange(
startExclusive = readEventSeqId(range.startExclusive)(connection),
endInclusive = readEventSeqId(range.endInclusive)(connection),
)
private def readEventSeqId(offset: Offset)(connection: java.sql.Connection): Long = {
import com.daml.platform.store.Conversions.OffsetToStatement
// This query could be: "select max(event_sequential_id) from participant_events where event_offset <= ${range.endInclusive}"
// however tests using PostgreSQL 12 with tens of millions of events have shown that the index
// on `event_offset` is not used unless we _hint_ at it by specifying `order by event_offset`
SQL"select max(event_sequential_id) from participant_events where event_offset <= ${offset} group by event_offset order by event_offset desc limit 1"
.as(get[Long](1).singleOpt)(connection)
.getOrElse(EmptyLedgerEventSeqId)
/** Converts ledger end offset to Event Sequential ID.
*
* @param endInclusive ledger end offset
* @param connection SQL connection
* @return Event Sequential ID range.
*/
def readEventSeqIdRange(endInclusive: Offset)(
connection: java.sql.Connection
): EventsRange[Long] = {
if (endInclusive == Offset.beforeBegin) EmptyEventSeqIdRange
else EmptyEventSeqIdRange.copy(endInclusive = readEventSeqId(endInclusive)(connection))
}
private def readEventSeqId(offset: Offset)(connection: java.sql.Connection): Long =
lookupEventSeqId(offset)(connection)
.getOrElse(EmptyLedgerEventSeqId)
}
private[events] def readPage[A](
read: (EventsRange[Long], String) => SimpleSql[Row], // takes range and limit sub-expression
row: RowParser[A],
read: (
EventsRange[Long],
Option[Int],
Option[Int],
) => Connection => Vector[A], // takes range, limit, fetchSize hint
range: EventsRange[Long],
pageSize: Int,
): SqlSequence[Vector[A]] = {
val minPageSize = 10 min pageSize max (pageSize / 10)
val guessedPageEnd = range.endInclusive min (range.startExclusive + pageSize)
SqlSequence
.vector(
read(range copy (endInclusive = guessedPageEnd), "") withFetchSize Some(pageSize),
row,
)
.plainQuery(read(range copy (endInclusive = guessedPageEnd), None, Some(pageSize)))
.flatMap { arithPage =>
val found = arithPage.size
if (guessedPageEnd == range.endInclusive || found >= minPageSize)
SqlSequence point arithPage
else
SqlSequence
.vector(
.plainQuery(
read(
range copy (startExclusive = guessedPageEnd),
s"limit ${minPageSize - found: Int}",
) withFetchSize Some(minPageSize - found),
row,
Some(minPageSize - found),
Some(minPageSize - found),
)
)
.map(arithPage ++ _)
}

View File

@ -3,12 +3,8 @@
package com.daml.platform.store.appendonlydao.events
import java.io.InputStream
import java.sql.Connection
import java.time.Instant
import anorm.SqlParser.{array, binaryStream, bool, int, long, str}
import anorm.{RowParser, ~}
import com.daml.ledger.participant.state.v1.Offset
import com.daml.ledger.api.v1.active_contracts_service.GetActiveContractsResponse
import com.daml.ledger.api.v1.event.Event
@ -26,64 +22,10 @@ import com.daml.ledger.api.v1.transaction_service.{
import com.daml.platform.ApiOffset
import com.daml.platform.api.v1.event.EventOps.{EventOps, TreeEventOps}
import com.daml.platform.index.TransactionConversion
import com.daml.platform.store.Conversions.{identifier, instant, offset}
import com.google.protobuf.timestamp.Timestamp
private[events] object EventsTable {
private type SharedRow =
Offset ~ String ~ Int ~ Long ~ String ~ String ~ Instant ~ Identifier ~ Option[String] ~
Option[String] ~ Array[String]
private val sharedRow: RowParser[SharedRow] =
offset("event_offset") ~
str("transaction_id") ~
int("node_index") ~
long("event_sequential_id") ~
str("event_id") ~
str("contract_id") ~
instant("ledger_effective_time") ~
identifier("template_id") ~
str("command_id").? ~
str("workflow_id").? ~
array[String]("event_witnesses")
type CreatedEventRow =
SharedRow ~ InputStream ~ Option[Int] ~ Array[String] ~ Array[String] ~ Option[String] ~
Option[InputStream] ~ Option[Int]
val createdEventRow: RowParser[CreatedEventRow] =
sharedRow ~
binaryStream("create_argument") ~
int("create_argument_compression").? ~
array[String]("create_signatories") ~
array[String]("create_observers") ~
str("create_agreement_text").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").?
type ExercisedEventRow =
SharedRow ~ Boolean ~ String ~ InputStream ~ Option[Int] ~ Option[InputStream] ~ Option[Int] ~
Array[String] ~ Array[String]
val exercisedEventRow: RowParser[ExercisedEventRow] =
sharedRow ~
bool("exercise_consuming") ~
str("exercise_choice") ~
binaryStream("exercise_argument") ~
int("exercise_argument_compression").? ~
binaryStream("exercise_result").? ~
int("exercise_result_compression").? ~
array[String]("exercise_actors") ~
array[String]("exercise_child_event_ids")
type ArchiveEventRow = SharedRow
val archivedEventRow: RowParser[ArchiveEventRow] = sharedRow
trait Batches {
def execute()(implicit connection: Connection): Unit
}
// TODO append-only: FIXME: move to the right place
object EventsTable {
final case class Entry[+E](
eventOffset: Offset,

View File

@ -1,48 +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.store.appendonlydao.events
import anorm.{Row, SimpleSql, SqlStringInterpolation}
import com.daml.ledger.participant.state.v1.Offset
import com.daml.platform.store.Conversions.OffsetToStatement
object EventsTableDelete {
/** Delete
* - archive events before or at specified offset and
* - create events that have been archived before or at the specific offset.
*/
def prepareEventsDelete(endInclusive: Offset): SimpleSql[Row] =
SQL"""
-- Divulgence events (only for contracts archived before the specified offset)
delete from participant_events_divulgence as delete_events
where
delete_events.event_offset <= $endInclusive and
exists (
SELECT 1 FROM participant_events_consuming_exercise as archive_events
WHERE
archive_events.event_offset <= $endInclusive AND
archive_events.contract_id = delete_events.contract_id
);
-- Create events (only for contracts archived before the specified offset)
delete from participant_events_create as delete_events
where
delete_events.event_offset <= $endInclusive and
exists (
SELECT 1 FROM participant_events_consuming_exercise as archive_events
WHERE
archive_events.event_offset <= $endInclusive AND
archive_events.contract_id = delete_events.contract_id
);
-- Exercise events (consuming)
delete from participant_events_consuming_exercise as delete_events
where
delete_events.event_offset <= $endInclusive;
-- Exercise events (non-consuming)
delete from participant_events_non_consuming_exercise as delete_events
where
delete_events.event_offset <= $endInclusive;
"""
}

View File

@ -1,188 +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.store.appendonlydao.events
import anorm.{Row, RowParser, SimpleSql, SqlStringInterpolation, ~}
import com.daml.ledger.participant.state.v1.Offset
import com.daml.ledger.TransactionId
import com.daml.platform.store.Conversions._
import scala.collection.compat.immutable.ArraySeq
private[events] object EventsTableFlatEvents {
private val createdFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent.Created]] =
EventsTable.createdEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~
templateId ~ commandId ~ workflowId ~ eventWitnesses ~ createArgument ~ createArgumentCompression ~
createSignatories ~ createObservers ~ createAgreementText ~ createKeyValue ~ createKeyValueCompression =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.FlatEvent.Created(
eventId = eventId,
contractId = contractId,
templateId = templateId,
createArgument = createArgument,
createArgumentCompression = createArgumentCompression,
createSignatories = ArraySeq.unsafeWrapArray(createSignatories),
createObservers = ArraySeq.unsafeWrapArray(createObservers),
createAgreementText = createAgreementText,
createKeyValue = createKeyValue,
createKeyValueCompression = createKeyValueCompression,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val archivedFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent.Archived]] =
EventsTable.archivedEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.FlatEvent.Archived(
eventId = eventId,
contractId = contractId,
templateId = templateId,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
val rawFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent]] =
createdFlatEventParser | archivedFlatEventParser
private val selectColumns =
Seq(
"event_offset",
"transaction_id",
"node_index",
"event_sequential_id",
"ledger_effective_time",
"workflow_id",
"participant_events.event_id",
"contract_id",
"template_id",
"create_argument",
"create_argument_compression",
"create_signatories",
"create_observers",
"create_agreement_text",
"create_key_value",
"create_key_value_compression",
).mkString(", ")
private val selectColumnsForACS =
Seq(
"active_cs.event_offset",
"active_cs.transaction_id",
"active_cs.node_index",
"active_cs.event_sequential_id",
"active_cs.ledger_effective_time",
"active_cs.workflow_id",
"active_cs.event_id",
"active_cs.contract_id",
"active_cs.template_id",
"active_cs.create_argument",
"active_cs.create_argument_compression",
"active_cs.create_signatories",
"active_cs.create_observers",
"active_cs.create_agreement_text",
"active_cs.create_key_value",
"active_cs.create_key_value_compression",
).mkString(", ")
def prepareLookupFlatTransactionById(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParties: Set[Party],
): SimpleSql[Row] =
route(requestingParties)(
single = singlePartyLookup(sqlFunctions)(transactionId, _),
multi = multiPartyLookup(sqlFunctions)(transactionId, _),
)
private def singlePartyLookup(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParty: Party,
): SimpleSql[Row] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", requestingParty)
SQL"""select #$selectColumns, array[$requestingParty] as event_witnesses,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id"""
}
private def multiPartyLookup(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParties: Set[Party],
): SimpleSql[Row] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", requestingParties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", requestingParties)
// TODO check existence of a test which fails when the transaction_id and flat_event_witnesses restriction wouldn't be there, if not add test.
SQL"""select #$selectColumns, flat_event_witnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id"""
}
private def getFlatTransactionsQueries(sqlFunctions: SqlFunctions) =
new EventsTableFlatEventsRangeQueries.GetTransactions(
selectColumns = selectColumns,
sqlFunctions = sqlFunctions,
)
def preparePagedGetFlatTransactions(sqlFunctions: SqlFunctions)(
range: EventsRange[Long],
filter: FilterRelation,
pageSize: Int,
): SqlSequence[Vector[EventsTable.Entry[Raw.FlatEvent]]] =
getFlatTransactionsQueries(sqlFunctions)(
range,
filter,
pageSize,
)
private def getActiveContractsQueries(sqlFunctions: SqlFunctions) =
new EventsTableFlatEventsRangeQueries.GetActiveContracts(
selectColumns = selectColumnsForACS,
sqlFunctions = sqlFunctions,
)
def preparePagedGetActiveContracts(sqlFunctions: SqlFunctions)(
range: EventsRange[(Offset, Long)],
filter: FilterRelation,
pageSize: Int,
): SqlSequence[Vector[EventsTable.Entry[Raw.FlatEvent]]] =
getActiveContractsQueries(sqlFunctions)(
range,
filter,
pageSize,
)
}

View File

@ -3,9 +3,10 @@
package com.daml.platform.store.appendonlydao.events
import anorm.{Row, SimpleSql, SqlStringInterpolation}
import java.sql.Connection
import com.daml.lf.data.Ref.{Identifier => ApiIdentifier}
import com.daml.platform.store.Conversions._
import com.daml.platform.store.backend.EventStorageBackend
private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset] {
@ -14,40 +15,34 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
protected def singleWildcardParty(
offset: Offset,
party: Party,
pageSize: Int,
): QueryParts
protected def singlePartyWithTemplates(
offset: Offset,
party: Party,
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts
protected def onlyWildcardParties(
offset: Offset,
parties: Set[Party],
pageSize: Int,
): QueryParts
protected def sameTemplates(
offset: Offset,
parties: Set[Party],
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts
protected def mixedTemplates(
offset: Offset,
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts
protected def mixedTemplatesWithWildcardParties(
offset: Offset,
wildcardParties: Set[Party],
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts
protected def offsetRange(offset: Offset): EventsRange[Long]
@ -64,10 +59,10 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
val (party, templateIds) = filter.iterator.next()
if (templateIds.isEmpty) {
// Single-party request, no specific template identifier
singleWildcardParty(offset, party, pageSize)
singleWildcardParty(offset, party)
} else {
// Single-party request, restricted to a set of template identifiers
singlePartyWithTemplates(offset, party, templateIds, pageSize)
singlePartyWithTemplates(offset, party, templateIds)
}
} else {
// Multi-party requests
@ -77,7 +72,6 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
onlyWildcardParties(
offset = offset,
parties = parties,
pageSize = pageSize,
)
else {
// If all parties request the same template identifier
@ -87,7 +81,6 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
offset,
parties = parties,
templateIds = templateIds,
pageSize = pageSize,
)
} else {
// If there are different template identifier but there are no wildcard parties
@ -97,7 +90,6 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
mixedTemplates(
offset,
partiesAndTemplateIds = partiesAndTemplateIds,
pageSize = pageSize,
)
} else {
// If there are wildcard parties and different template identifiers
@ -105,7 +97,6 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
offset,
wildcardParties,
partiesAndTemplateIds,
pageSize,
)
}
}
@ -116,15 +107,11 @@ private[events] sealed abstract class EventsTableFlatEventsRangeQueries[Offset]
case QueryParts.ByArith(read) =>
EventsRange.readPage(
read,
EventsTableFlatEvents.rawFlatEventParser,
offsetRange(offset),
pageSize,
)
case QueryParts.ByLimit(sql) =>
SqlSequence.vector(
sql withFetchSize Some(pageSize),
EventsTableFlatEvents.rawFlatEventParser,
)
SqlSequence.plainQuery(sql(pageSize))
}
}
}
@ -137,161 +124,110 @@ private[events] object EventsTableFlatEventsRangeQueries {
extends Product
with Serializable
private[EventsTableFlatEventsRangeQueries] object QueryParts {
final case class ByArith(read: (EventsRange[Long], String) => SimpleSql[Row]) extends QueryParts
final case class ByLimit(saferRead: SimpleSql[Row]) extends QueryParts
import language.implicitConversions
implicit def `go by limit`(saferRead: SimpleSql[Row]): ByLimit = ByLimit(saferRead)
final case class ByArith(
read: (
EventsRange[Long],
Option[Int],
Option[Int],
) => Connection => Vector[EventsTable.Entry[Raw.FlatEvent]]
) extends QueryParts
final case class ByLimit(
saferRead: Int => Connection => Vector[EventsTable.Entry[Raw.FlatEvent]]
) extends QueryParts
}
final class GetTransactions(
selectColumns: String,
sqlFunctions: SqlFunctions,
storageBackend: EventStorageBackend
) extends EventsTableFlatEventsRangeQueries[EventsRange[Long]] {
override protected def singleWildcardParty(
range: EventsRange[Long],
party: Party,
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", party)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, array[$party] as event_witnesses,
case when submitters = array[$party]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
order by event_sequential_id #$limitExpr"""
): QueryParts =
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsSingleWildcardParty(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
party = party,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
)
}
override protected def singlePartyWithTemplates(
range: EventsRange[Long],
party: Party,
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", party)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, array[$party] as event_witnesses,
case when submitters = array[$party]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
and template_id in ($templateIds)
order by event_sequential_id #$limitExpr"""
): QueryParts =
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsSinglePartyWithTemplates(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
party = party,
templateIds = templateIds,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
)
}
override protected def onlyWildcardParties(
range: EventsRange[Long],
parties: Set[Party],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", parties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", parties)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
order by event_sequential_id #$limitExpr"""
): QueryParts =
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsOnlyWildcardParties(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
parties = parties,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
)
}
override protected def sameTemplates(
range: EventsRange[Long],
parties: Set[Party],
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", parties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", parties)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
and template_id in ($templateIds)
order by event_sequential_id #$limitExpr"""
): QueryParts =
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsSameTemplates(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
parties = parties,
templateIds = templateIds,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
)
}
override protected def mixedTemplates(
range: EventsRange[Long],
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts = {
val parties = partiesAndTemplateIds.map(_._1)
val partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause(
sqlFunctions,
"flat_event_witnesses",
partiesAndTemplateIds,
): QueryParts =
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsMixedTemplates(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
partiesAndTemplateIds = partiesAndTemplateIds,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", parties)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$partiesAndTemplatesCondition
order by event_sequential_id #$limitExpr"""
)
}
override protected def mixedTemplatesWithWildcardParties(
range: EventsRange[Long],
wildcardParties: Set[Party],
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts = {
val parties = wildcardParties ++ partiesAndTemplateIds.map(_._1)
val partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause(
sqlFunctions,
"flat_event_witnesses",
partiesAndTemplateIds,
QueryParts.ByArith((range, limit, fetchSizeHint) =>
storageBackend.transactionsEventsMixedTemplatesWithWildcardParties(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
partiesAndTemplateIds = partiesAndTemplateIds,
wildcardParties = wildcardParties,
limit = limit,
fetchSizeHint = fetchSizeHint,
)
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("flat_event_witnesses", wildcardParties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", parties)
QueryParts.ByArith(
read = (range, limitExpr) => SQL"""
select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and (#$witnessesWhereClause or #$partiesAndTemplatesCondition)
order by event_sequential_id #$limitExpr"""
)
}
@ -299,205 +235,105 @@ private[events] object EventsTableFlatEventsRangeQueries {
}
final class GetActiveContracts(
selectColumns: String,
sqlFunctions: SqlFunctions,
storageBackend: EventStorageBackend
) extends EventsTableFlatEventsRangeQueries[EventsRange[(Offset, Long)]] {
override protected def singleWildcardParty(
range: EventsRange[(Offset, Long)],
party: Party,
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause(s"active_cs.flat_event_witnesses", party)
SQL"""select #$selectColumns, array[$party] as event_witnesses,
case when active_cs.submitters = array[$party]::text[] then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and #$witnessesWhereClause
order by active_cs.event_sequential_id limit $pageSize"""
}
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsSingleWildcardParty(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
party = party,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
)
override protected def singlePartyWithTemplates(
range: EventsRange[(Offset, Long)],
party: Party,
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.flat_event_witnesses", party)
SQL"""select #$selectColumns, array[$party] as event_witnesses,
case when active_cs.submitters = array[$party]::text[] then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and #$witnessesWhereClause
and active_cs.template_id in ($templateIds)
order by active_cs.event_sequential_id limit $pageSize"""
}
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsSinglePartyWithTemplates(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
party = party,
templateIds = templateIds,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
)
override def onlyWildcardParties(
range: EventsRange[(Offset, Long)],
parties: Set[Party],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.flat_event_witnesses", parties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("active_cs.flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.submitters", parties)
SQL"""select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and #$witnessesWhereClause
order by active_cs.event_sequential_id limit $pageSize"""
}
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsOnlyWildcardParties(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
parties = parties,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
)
override def sameTemplates(
range: EventsRange[(Offset, Long)],
parties: Set[Party],
templateIds: Set[ApiIdentifier],
pageSize: Int,
): QueryParts = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.flat_event_witnesses", parties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("active_cs.flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.submitters", parties)
SQL"""select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and #$witnessesWhereClause
and active_cs.template_id in ($templateIds)
order by active_cs.event_sequential_id limit $pageSize"""
}
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsSameTemplates(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
parties = parties,
templateIds = templateIds,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
)
override def mixedTemplates(
range: EventsRange[(Offset, Long)],
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts = {
val parties = partiesAndTemplateIds.map(_._1)
val partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause(
sqlFunctions,
"active_cs.flat_event_witnesses",
partiesAndTemplateIds,
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsMixedTemplates(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
partiesAndTemplateIds = partiesAndTemplateIds,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("active_cs.flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.submitters", parties)
// TODO these repetitions are painful to maintain. Ideas: Adding SQL views (do reuse in SQL schema), pursue anorm to enable reuse, using something else then anorm here.
SQL"""select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and #$partiesAndTemplatesCondition
order by active_cs.event_sequential_id limit $pageSize"""
}
)
override def mixedTemplatesWithWildcardParties(
range: EventsRange[(Offset, Long)],
wildcardParties: Set[Party],
partiesAndTemplateIds: Set[(Party, ApiIdentifier)],
pageSize: Int,
): QueryParts = {
val parties = wildcardParties ++ partiesAndTemplateIds.map(_._1)
val partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause(
sqlFunctions,
"flat_event_witnesses",
partiesAndTemplateIds,
): QueryParts =
QueryParts.ByLimit(limit =>
storageBackend.activeContractsEventsMixedTemplatesWithWildcardParties(
startExclusive = range.startExclusive._2,
endInclusiveSeq = range.endInclusive._2,
endInclusiveOffset = range.endInclusive._1,
partiesAndTemplateIds = partiesAndTemplateIds,
wildcardParties = wildcardParties,
limit = Some(limit),
fetchSizeHint = Some(limit),
)
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.flat_event_witnesses", wildcardParties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("active_cs.flat_event_witnesses", parties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("active_cs.submitters", parties)
SQL"""select #$selectColumns, #$filteredWitnesses as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > ${range.startExclusive._2: Long}
and active_cs.event_sequential_id <= ${range.endInclusive._2: Long}
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= ${range.endInclusive._1: Offset}
)
and (#$witnessesWhereClause or #$partiesAndTemplatesCondition)
order by active_cs.event_sequential_id limit $pageSize"""
}
)
override protected def offsetRange(offset: EventsRange[(Offset, Long)]) = offset map (_._2)
}
private def formatPartiesAndTemplatesWhereClause(
sqlFunctions: SqlFunctions,
witnessesAggregationColumn: String,
partiesAndTemplateIds: Set[(Party, Identifier)],
): String =
partiesAndTemplateIds.view
.map { case (p, i) =>
s"(${sqlFunctions.arrayIntersectionWhereClause(witnessesAggregationColumn, p)} and template_id = '$i')"
}
.mkString("(", " or ", ")")
}

View File

@ -1,211 +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.store.appendonlydao.events
import anorm.{Row, RowParser, SimpleSql, SqlStringInterpolation, ~}
import com.daml.ledger.TransactionId
import com.daml.platform.store.Conversions._
import scala.collection.compat.immutable.ArraySeq
private[events] object EventsTableTreeEvents {
private val createdTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent.Created]] =
EventsTable.createdEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses ~ createArgument ~ createArgumentCompression ~ createSignatories ~ createObservers ~ createAgreementText ~ createKeyValue ~ createKeyValueCompression =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.TreeEvent.Created(
eventId = eventId,
contractId = contractId,
templateId = templateId,
createArgument = createArgument,
createArgumentCompression = createArgumentCompression,
createSignatories = ArraySeq.unsafeWrapArray(createSignatories),
createObservers = ArraySeq.unsafeWrapArray(createObservers),
createAgreementText = createAgreementText,
createKeyValue = createKeyValue,
createKeyValueCompression = createKeyValueCompression,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val exercisedTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent.Exercised]] =
EventsTable.exercisedEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses ~ exerciseConsuming ~ exerciseChoice ~ exerciseArgument ~ exerciseArgumentCompression ~ exerciseResult ~ exerciseResultCompression ~ exerciseActors ~ exerciseChildEventIds =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.TreeEvent.Exercised(
eventId = eventId,
contractId = contractId,
templateId = templateId,
exerciseConsuming = exerciseConsuming,
exerciseChoice = exerciseChoice,
exerciseArgument = exerciseArgument,
exerciseArgumentCompression = exerciseArgumentCompression,
exerciseResult = exerciseResult,
exerciseResultCompression = exerciseResultCompression,
exerciseActors = ArraySeq.unsafeWrapArray(exerciseActors),
exerciseChildEventIds = ArraySeq.unsafeWrapArray(exerciseChildEventIds),
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
val rawTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent]] =
createdTreeEventParser | exercisedTreeEventParser
private val selectColumns = Seq(
"event_offset",
"transaction_id",
"node_index",
"event_sequential_id",
"participant_events.event_id",
"contract_id",
"ledger_effective_time",
"template_id",
"workflow_id",
"create_argument",
"create_argument_compression",
"create_signatories",
"create_observers",
"create_agreement_text",
"create_key_value",
"create_key_value_compression",
"exercise_choice",
"exercise_argument",
"exercise_argument_compression",
"exercise_result",
"exercise_result_compression",
"exercise_actors",
"exercise_child_event_ids",
).mkString(", ")
def prepareLookupTransactionTreeById(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParties: Set[Party],
): SimpleSql[Row] =
route(requestingParties)(
single = singlePartyLookup(sqlFunctions)(transactionId, _),
multi = multiPartyLookup(sqlFunctions)(transactionId, _),
)
private def singlePartyLookup(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParty: Party,
): SimpleSql[Row] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", requestingParty)
SQL"""select #$selectColumns, array[$requestingParty] as event_witnesses,
event_kind = 20 as exercise_consuming,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by node_index asc"""
}
private def multiPartyLookup(sqlFunctions: SqlFunctions)(
transactionId: TransactionId,
requestingParties: Set[Party],
): SimpleSql[Row] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", requestingParties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("tree_event_witnesses", requestingParties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", requestingParties)
SQL"""select #$selectColumns, #$filteredWitnesses as event_witnesses,
event_kind = 20 as exercise_consuming,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by node_index asc"""
}
def preparePagedGetTransactionTrees(sqlFunctions: SqlFunctions)(
eventsRange: EventsRange[Long],
requestingParties: Set[Party],
pageSize: Int,
): SqlSequence[Vector[EventsTable.Entry[Raw.TreeEvent]]] =
route(requestingParties)(
single = singlePartyTrees(sqlFunctions)(eventsRange, _, pageSize),
multi = multiPartyTrees(sqlFunctions)(eventsRange, _, pageSize),
)
private def singlePartyTrees(sqlFunctions: SqlFunctions)(
range: EventsRange[Long],
requestingParty: Party,
pageSize: Int,
): SqlSequence[Vector[EventsTable.Entry[Raw.TreeEvent]]] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", requestingParty)
EventsRange.readPage(
read = (range, limitExpr) => SQL"""
select #$selectColumns, array[$requestingParty] as event_witnesses,
event_kind = 20 as exercise_consuming,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id #$limitExpr""",
rawTreeEventParser,
range,
pageSize,
)
}
private def multiPartyTrees(sqlFunctions: SqlFunctions)(
range: EventsRange[Long],
requestingParties: Set[Party],
pageSize: Int,
): SqlSequence[Vector[EventsTable.Entry[Raw.TreeEvent]]] = {
val witnessesWhereClause =
sqlFunctions.arrayIntersectionWhereClause("tree_event_witnesses", requestingParties)
val filteredWitnesses =
sqlFunctions.arrayIntersectionValues("tree_event_witnesses", requestingParties)
val submittersInPartiesClause =
sqlFunctions.arrayIntersectionWhereClause("submitters", requestingParties)
EventsRange.readPage(
read = (range, limitExpr) => SQL"""
select #$selectColumns, #$filteredWitnesses as event_witnesses,
event_kind = 20 as exercise_consuming,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > ${range.startExclusive}
and event_sequential_id <= ${range.endInclusive}
and #$witnessesWhereClause
and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id #$limitExpr""",
rawTreeEventParser,
range,
pageSize,
)
}
}

View File

@ -7,6 +7,7 @@ import java.sql.Connection
import java.time.Instant
import com.daml.ledger.participant.state.v1.{CommittedTransaction, RejectionReasonV0}
import com.daml.platform.store.backend.{ContractStorageBackend, PartyStorageBackend}
/** Performs post-commit validation on transactions for Sandbox Classic.
* This is intended exclusively as a temporary replacement for
@ -49,8 +50,10 @@ private[appendonlydao] object PostCommitValidation {
None
}
final class BackedBy(data: PostCommitValidationData, validatePartyAllocation: Boolean)
extends PostCommitValidation {
final class BackedBy(
dao: PartyStorageBackend with ContractStorageBackend,
validatePartyAllocation: Boolean,
) extends PostCommitValidation {
def validate(
transaction: CommittedTransaction,
@ -85,8 +88,8 @@ private[appendonlydao] object PostCommitValidation {
if (referredContracts.isEmpty) {
None
} else {
data
.lookupMaximumLedgerTime(referredContracts)
dao
.maximumLedgerTime(referredContracts)(connection)
.map(validateCausalMonotonicity(_, transactionLedgerEffectiveTime))
.getOrElse(Some(UnknownContract))
}
@ -111,7 +114,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] = {
val informees = transaction.informees
val allocatedInformees = data.lookupParties(informees.toSeq).map(_.party)
val allocatedInformees = dao.parties(informees.toSeq)(connection).map(_.party)
if (allocatedInformees.toSet == informees)
None
else
@ -129,7 +132,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction
)(implicit connection: Connection): Option[RejectionReasonV0] =
transaction
.foldInExecutionOrder[Result](Right(State.empty(data)))(
.foldInExecutionOrder[Result](Right(State.empty(dao)))(
exerciseBegin = (acc, _, exe) => {
val newAcc = acc.flatMap(validateKeyUsages(exe, _))
(newAcc, true)
@ -201,13 +204,13 @@ private[appendonlydao] object PostCommitValidation {
* @param rollbackStack Stack of states at the beginning of rollback nodes so we can
* restore the state at the end of the rollback. The most recent rollback
* comes first.
* @param data Data about committed contracts for post-commit validation purposes.
* @param dao Dao about committed contracts for post-commit validation purposes.
* This is never changed during the traversal of the transaction.
*/
private final case class State(
private val currentState: ActiveState,
private val rollbackStack: List[ActiveState],
private val data: PostCommitValidationData,
private val dao: PartyStorageBackend with ContractStorageBackend,
) {
def validateCreate(maybeKey: Option[Key], id: ContractId)(implicit
@ -257,14 +260,14 @@ private[appendonlydao] object PostCommitValidation {
private def lookup(key: Key)(implicit connection: Connection): Option[ContractId] =
currentState.contracts.get(key.hash).orElse {
if (currentState.removed(key.hash)) None
else data.lookupContractKeyGlobally(key)
else dao.contractKeyGlobally(key)(connection)
}
}
private object State {
def empty(data: PostCommitValidationData): State =
State(ActiveState(Map.empty, Set.empty), Nil, data)
def empty(dao: PartyStorageBackend with ContractStorageBackend): State =
State(ActiveState(Map.empty, Set.empty), Nil, dao)
}
private[events] val DuplicateKey: RejectionReasonV0 =

View File

@ -1,22 +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.store.appendonlydao.events
import java.sql.Connection
import java.time.Instant
import com.daml.ledger.api.domain.PartyDetails
import scala.util.Try
private[events] trait PostCommitValidationData {
def lookupContractKeyGlobally(key: Key)(implicit connection: Connection): Option[ContractId]
def lookupMaximumLedgerTime(ids: Set[ContractId])(implicit
connection: Connection
): Try[Option[Instant]]
def lookupParties(parties: Seq[Party])(implicit connection: Connection): List[PartyDetails]
}

View File

@ -5,12 +5,22 @@ package com.daml.platform.store.appendonlydao.events
import java.sql.Connection
import anorm.SQL
import com.daml.ledger.participant.state.v1.Offset
import com.daml.platform.server.api.validation.ErrorFactories
import com.daml.platform.store.Conversions.offset
import com.daml.platform.store.backend.ParameterStorageBackend
object QueryNonPruned {
trait QueryNonPruned {
// TODO append-only: consider keeping the Throwing-one only for simplicity
def executeSql[T](query: => T, minOffsetExclusive: Offset, error: Offset => String)(implicit
conn: Connection
): Either[Throwable, T]
def executeSqlOrThrow[T](query: => T, minOffsetExclusive: Offset, error: Offset => String)(
implicit conn: Connection
): T
}
case class QueryNonPrunedImpl(storageBackend: ParameterStorageBackend) extends QueryNonPruned {
/** Runs a query and throws an error if the query accesses pruned offsets.
*
@ -32,8 +42,8 @@ object QueryNonPruned {
): Either[Throwable, T] = {
val result = query
SQL_SELECT_MOST_RECENT_PRUNING
.as(offset("participant_pruned_up_to_inclusive").?.single)
storageBackend
.prunedUptoInclusive(conn)
.fold(Right(result): Either[Throwable, T])(pruningOffsetUpToInclusive =>
Either.cond(
minOffsetExclusive >= pruningOffsetUpToInclusive,
@ -48,8 +58,4 @@ object QueryNonPruned {
): T =
executeSql(query, minOffsetExclusive, error).fold(throw _, identity)
private val SQL_SELECT_MOST_RECENT_PRUNING = SQL(
"select participant_pruned_up_to_inclusive from parameters"
)
}

View File

@ -24,7 +24,7 @@ import scala.concurrent.{ExecutionContext, Future}
* wrap events from the database while delaying deserialization
* so that it doesn't happen on the database thread pool.
*/
private[events] sealed trait Raw[+E] {
sealed trait Raw[+E] {
/** Fill the blanks left in the raw event by running
* the deserialization on contained values.
@ -42,13 +42,14 @@ private[events] sealed trait Raw[+E] {
}
private[events] object Raw {
// TODO append-only: FIXME move
object Raw {
/** Since created events can be both a flat event or a tree event
* we share common code between the two variants here. What's left
* out is wrapping the result in the proper envelope.
*/
private[events] sealed abstract class Created[E](
sealed abstract class Created[E](
val partial: PbCreatedEvent,
val createArgument: InputStream,
val createArgumentCompression: Compression.Algorithm,
@ -67,7 +68,7 @@ private[events] object Raw {
lfValueTranslation.deserialize(this, verbose).map(wrapInEvent)
}
private object Created {
object Created {
def apply(
eventId: String,
contractId: String,

View File

@ -1,46 +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.store.appendonlydao.events
import com.daml.platform.store.DbType
import com.daml.platform.store.appendonlydao.events.EventsTableQueries.format
private[appendonlydao] trait SqlFunctions {
def arrayIntersectionWhereClause(arrayColumn: String, party: Party): String =
arrayIntersectionWhereClause(arrayColumn: String, Set(party))
def arrayIntersectionWhereClause(arrayColumn: String, parties: Set[Party]): String
def arrayIntersectionValues(arrayColumn: String, parties: Set[Party]): String
}
private[appendonlydao] object SqlFunctions {
def arrayIntersection(a: Array[String], b: Array[String]): Array[String] =
a.toSet.intersect(b.toSet).toArray
def apply(dbType: DbType): SqlFunctions = dbType match {
case DbType.Postgres => PostgresSqlFunctions
case DbType.H2Database => H2SqlFunctions
case DbType.Oracle => throw new NotImplementedError("not yet supported")
}
object PostgresSqlFunctions extends SqlFunctions {
override def arrayIntersectionWhereClause(arrayColumn: String, parties: Set[Party]): String =
s"$arrayColumn::text[] && array[${format(parties)}]::text[]"
def arrayIntersectionValues(arrayColumn: String, parties: Set[Party]): String =
s"array(select unnest($arrayColumn) intersect select unnest(array[${format(parties)}]))"
}
object H2SqlFunctions extends SqlFunctions {
override def arrayIntersectionWhereClause(arrayColumn: String, parties: Set[Party]): String =
if (parties.isEmpty)
"false"
else
parties.view.map(p => s"array_contains($arrayColumn, '$p')").mkString("(", " or ", ")")
def arrayIntersectionValues(arrayColumn: String, parties: Set[Party]): String =
s"array_intersection($arrayColumn, array[${format(parties)}])"
}
}

View File

@ -3,8 +3,6 @@
package com.daml.platform.store.appendonlydao.events
import com.daml.platform.store.SimpleSqlAsVectorOf._
import anorm.{ResultSetParser, Row, RowParser, SimpleSql}
import scalaz.{-\/, Free, Functor, \/-}
import java.sql.Connection
@ -38,11 +36,9 @@ object SqlSequence {
}
}
def apply[A](s: SimpleSql[_], p: ResultSetParser[A]): T[A] =
Free liftF new Element(implicit conn => s as p)
def vector[A](s: SimpleSql[Row], p: RowParser[A]): T[Vector[A]] =
Free liftF new Element(implicit conn => s asVectorOf p)
// TODO append-only: This probably defeats the purpose, only needed to limit the impact of the SQL query abstraction change. Consider to remove SqlSequence.
def plainQuery[A](query: Connection => A): T[A] =
Free liftF new Element(query)
def point[A](a: A): T[A] = Free point a
}

View File

@ -4,132 +4,13 @@
package com.daml.platform.store.appendonlydao.events
import java.io.InputStream
import java.sql.Connection
import java.time.Instant
import anorm.SqlParser._
import anorm._
import com.daml.ledger.participant.state.v1.Offset
import com.daml.lf.data.Ref
import com.daml.lf.ledger.EventId
import com.daml.platform
import com.daml.platform.store.Conversions.{contractId, eventId, offset, _}
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import com.daml.platform.store.backend.StorageBackend.RawTransactionEvent
import com.daml.platform.store.interfaces.TransactionLogUpdate
import com.daml.platform.store.serialization.{Compression, ValueSerializer}
object TransactionLogUpdatesReader {
private val createsAndArchives: EventsRange[Long] => SimpleSql[Row] =
(range: EventsRange[Long]) => SQL"""
SELECT
event_kind,
transaction_id,
node_index,
command_id,
workflow_id,
event_id,
contract_id,
template_id,
ledger_effective_time,
create_signatories,
create_observers,
create_agreement_text,
create_key_value,
create_key_value_compression,
create_argument,
create_argument_compression,
tree_event_witnesses,
flat_event_witnesses,
exercise_choice,
exercise_argument,
exercise_argument_compression,
exercise_result,
exercise_result_compression,
exercise_actors,
exercise_child_event_ids,
event_sequential_id,
event_offset
FROM
participant_events
WHERE
event_sequential_id > #${range.startExclusive}
and event_sequential_id <= #${range.endInclusive}
and event_kind != 0
ORDER BY event_sequential_id ASC
"""
private val rawTransactionEventParser: RowParser[RawTransactionEvent] =
(int("event_kind") ~
str("transaction_id") ~
int("node_index") ~
str("command_id").? ~
str("workflow_id").? ~
eventId("event_id") ~
contractId("contract_id") ~
identifier("template_id").? ~
instant("ledger_effective_time").? ~
array[String]("create_signatories").? ~
array[String]("create_observers").? ~
str("create_agreement_text").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").? ~
binaryStream("create_argument").? ~
int("create_argument_compression").? ~
array[String]("tree_event_witnesses") ~
array[String]("flat_event_witnesses") ~
str("exercise_choice").? ~
binaryStream("exercise_argument").? ~
int("exercise_argument_compression").? ~
binaryStream("exercise_result").? ~
int("exercise_result_compression").? ~
array[String]("exercise_actors").? ~
array[String]("exercise_child_event_ids").? ~
long("event_sequential_id") ~
offset("event_offset")).map {
case eventKind ~ transactionId ~ nodeIndex ~ commandId ~ workflowId ~ eventId ~ contractId ~ templateId ~ ledgerEffectiveTime ~ createSignatories ~
createObservers ~ createAgreementText ~ createKeyValue ~ createKeyCompression ~
createArgument ~ createArgumentCompression ~ treeEventWitnesses ~ flatEventWitnesses ~ exerciseChoice ~
exerciseArgument ~ exerciseArgumentCompression ~ exerciseResult ~ exerciseResultCompression ~ exerciseActors ~
exerciseChildEventIds ~ eventSequentialId ~ offset =>
new RawTransactionEvent(
eventKind,
transactionId,
nodeIndex,
commandId,
workflowId,
eventId,
contractId,
templateId,
ledgerEffectiveTime,
createSignatories,
createObservers,
createAgreementText,
createKeyValue,
createKeyCompression,
createArgument,
createArgumentCompression,
treeEventWitnesses.toSet,
flatEventWitnesses.toSet,
exerciseChoice,
exerciseArgument,
exerciseArgumentCompression,
exerciseResult,
exerciseResultCompression,
exerciseActors,
exerciseChildEventIds,
eventSequentialId,
offset,
)
}
/** This method intentionally produces a generic DTO to perform as much work as possible outside of the db thread pool
* (specifically the translation to the `TransactionLogUpdate.Event`)
*/
def readRawEvents(range: EventsRange[Long])(implicit
conn: Connection
): Vector[RawTransactionEvent] =
createsAndArchives(EventsRange(range.startExclusive, range.endInclusive))
.asVectorOf(rawTransactionEventParser)
def toTransactionEvent(
raw: RawTransactionEvent
@ -238,36 +119,6 @@ object TransactionLogUpdatesReader {
val NonConsumingExercise = 25
}
class RawTransactionEvent(
val eventKind: Int,
val transactionId: String,
val nodeIndex: Int,
val commandId: Option[String],
val workflowId: Option[String],
val eventId: EventId,
val contractId: platform.store.appendonlydao.events.ContractId,
val templateId: Option[platform.store.appendonlydao.events.Identifier],
val ledgerEffectiveTime: Option[Instant],
val createSignatories: Option[Array[String]],
val createObservers: Option[Array[String]],
val createAgreementText: Option[String],
val createKeyValue: Option[InputStream],
val createKeyCompression: Option[Int],
val createArgument: Option[InputStream],
val createArgumentCompression: Option[Int],
val treeEventWitnesses: Set[String],
val flatEventWitnesses: Set[String],
val exerciseChoice: Option[String],
val exerciseArgument: Option[InputStream],
val exerciseArgumentCompression: Option[Int],
val exerciseResult: Option[InputStream],
val exerciseResultCompression: Option[Int],
val exerciseActors: Option[Array[String]],
val exerciseChildEventIds: Option[Array[String]],
val eventSequentialId: Long,
val offset: Offset,
)
private implicit class MandatoryField[T](val opt: Option[T]) extends AnyVal {
def mandatory(fieldName: String): T = opt.getOrElse(throw FieldMissingError(fieldName))
}

View File

@ -23,9 +23,8 @@ import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.metrics._
import com.daml.nameof.NameOf.qualifiedNameOfCurrentFunc
import com.daml.platform.ApiOffset
import com.daml.platform.store.DbType
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import com.daml.platform.store.appendonlydao.{DbDispatcher, PaginatingAsyncStream}
import com.daml.platform.store.backend.StorageBackend
import com.daml.platform.store.dao.LedgerDaoTransactionsReader
import com.daml.platform.store.dao.events.ContractStateEvent
import com.daml.platform.store.interfaces.TransactionLogUpdate
@ -46,7 +45,8 @@ import scala.util.{Failure, Success}
*/
private[appendonlydao] final class TransactionsReader(
dispatcher: DbDispatcher,
dbType: DbType,
queryNonPruned: QueryNonPruned,
storageBackend: StorageBackend[_],
pageSize: Int,
eventProcessingParallelism: Int,
metrics: Metrics,
@ -55,8 +55,12 @@ private[appendonlydao] final class TransactionsReader(
extends LedgerDaoTransactionsReader {
private val dbMetrics = metrics.daml.index.db
private val sqlFunctions = SqlFunctions(dbType)
private val eventSeqIdReader =
new EventsRange.EventSeqIdReader(storageBackend.maxEventSeqIdForOffset)
private val getTransactions =
new EventsTableFlatEventsRangeQueries.GetTransactions(storageBackend)
private val getActiveContracts =
new EventsTableFlatEventsRangeQueries.GetActiveContracts(storageBackend)
private val logger = ContextualizedLogger.get(this.getClass)
@ -82,6 +86,7 @@ private[appendonlydao] final class TransactionsReader(
)(implicit loggingContext: LoggingContext): Future[EventsTable.Entry[E]] =
deserializeEvent(verbose)(entry).map(event => entry.copy(event = event))
// TODO append-only: as consolidating event queries, consider consolidating functionality serving the flatTransactions and transactionTrees
override def getFlatTransactions(
startExclusive: Offset,
endInclusive: Offset,
@ -97,14 +102,12 @@ private[appendonlydao] final class TransactionsReader(
val query = (range: EventsRange[(Offset, Long)]) => {
implicit connection: Connection =>
logger.debug(s"getFlatTransactions query($range)")
QueryNonPruned.executeSqlOrThrow(
EventsTableFlatEvents
.preparePagedGetFlatTransactions(sqlFunctions)(
range = EventsRange(range.startExclusive._2, range.endInclusive._2),
filter = filter,
pageSize = pageSize,
)
.executeSql,
queryNonPruned.executeSqlOrThrow(
getTransactions(
EventsRange(range.startExclusive._2, range.endInclusive._2),
filter,
pageSize,
).executeSql,
range.startExclusive._1,
pruned =>
s"Transactions request from ${range.startExclusive._1.toHexString} to ${range.endInclusive._1.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -144,16 +147,14 @@ private[appendonlydao] final class TransactionsReader(
override def lookupFlatTransactionById(
transactionId: TransactionId,
requestingParties: Set[Party],
)(implicit loggingContext: LoggingContext): Future[Option[GetFlatTransactionResponse]] = {
val query =
EventsTableFlatEvents.prepareLookupFlatTransactionById(sqlFunctions)(
transactionId,
requestingParties,
)
)(implicit loggingContext: LoggingContext): Future[Option[GetFlatTransactionResponse]] =
dispatcher
.executeSql(dbMetrics.lookupFlatTransactionById) { implicit connection =>
query.asVectorOf(EventsTableFlatEvents.rawFlatEventParser)
}
.executeSql(dbMetrics.lookupFlatTransactionById)(
route(requestingParties)(
single = storageBackend.flatTransactionSingleParty(transactionId, _) _,
multi = storageBackend.flatTransactionMultiParty(transactionId, _) _,
)
)
.flatMap(rawEvents =>
Timed.value(
timer = dbMetrics.lookupFlatTransactionById.translationTimer,
@ -161,7 +162,6 @@ private[appendonlydao] final class TransactionsReader(
)
)
.map(EventsTable.Entry.toGetFlatTransactionResponse)
}
override def getTransactionTrees(
startExclusive: Offset,
@ -182,14 +182,35 @@ private[appendonlydao] final class TransactionsReader(
val query = (range: EventsRange[(Offset, Long)]) => {
implicit connection: Connection =>
logger.debug(s"getTransactionTrees query($range)")
QueryNonPruned.executeSqlOrThrow(
EventsTableTreeEvents
.preparePagedGetTransactionTrees(sqlFunctions)(
eventsRange = EventsRange(range.startExclusive._2, range.endInclusive._2),
requestingParties = requestingParties,
pageSize = pageSize,
)
.executeSql,
queryNonPruned.executeSqlOrThrow(
route(requestingParties)(
single = party =>
EventsRange.readPage(
read = (range, limit, fetchSizeHint) =>
storageBackend.transactionTreeEventsSingleParty(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
requestingParty = party,
limit = limit,
fetchSizeHint = fetchSizeHint,
),
range = EventsRange(range.startExclusive._2, range.endInclusive._2),
pageSize = pageSize,
),
multi = parties =>
EventsRange.readPage(
read = (range, limit, fetchSizeHint) =>
storageBackend.transactionTreeEventsMultiParty(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
requestingParties = parties,
limit = limit,
fetchSizeHint = fetchSizeHint,
),
range = EventsRange(range.startExclusive._2, range.endInclusive._2),
pageSize = pageSize,
),
).executeSql,
range.startExclusive._1,
pruned =>
s"Transactions request from ${range.startExclusive._1.toHexString} to ${range.endInclusive._1.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -229,16 +250,14 @@ private[appendonlydao] final class TransactionsReader(
override def lookupTransactionTreeById(
transactionId: TransactionId,
requestingParties: Set[Party],
)(implicit loggingContext: LoggingContext): Future[Option[GetTransactionResponse]] = {
val query =
EventsTableTreeEvents.prepareLookupTransactionTreeById(sqlFunctions)(
transactionId,
requestingParties,
)
)(implicit loggingContext: LoggingContext): Future[Option[GetTransactionResponse]] =
dispatcher
.executeSql(dbMetrics.lookupTransactionTreeById) { implicit connection =>
query.asVectorOf(EventsTableTreeEvents.rawTreeEventParser)
}
.executeSql(dbMetrics.lookupTransactionTreeById)(
route(requestingParties)(
single = storageBackend.transactionTreeSingleParty(transactionId, _) _,
multi = storageBackend.transactionTreeMultiParty(transactionId, _) _,
)
)
.flatMap(rawEvents =>
Timed.value(
timer = dbMetrics.lookupTransactionTreeById.translationTimer,
@ -246,7 +265,6 @@ private[appendonlydao] final class TransactionsReader(
)
)
.map(EventsTable.Entry.toGetTransactionResponse)
}
override def getTransactionLogUpdates(
startExclusive: (Offset, Long),
@ -275,8 +293,11 @@ private[appendonlydao] final class TransactionsReader(
// Dispatch database fetches in parallel
.mapAsync(eventProcessingParallelism) { range =>
dispatcher.executeSql(dbMetrics.getTransactionLogUpdates) { implicit conn =>
QueryNonPruned.executeSqlOrThrow(
query = TransactionLogUpdatesReader.readRawEvents(range),
queryNonPruned.executeSqlOrThrow(
query = storageBackend.rawEvents(
startExclusive = range.startExclusive,
endInclusive = range.endInclusive,
)(conn),
minOffsetExclusive = startExclusive._1,
error = pruned =>
s"Active contracts request after ${startExclusive._1.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -334,14 +355,12 @@ private[appendonlydao] final class TransactionsReader(
val query = (range: EventsRange[(Offset, Long)]) => {
implicit connection: Connection =>
logger.debug(s"getActiveContracts query($range)")
QueryNonPruned.executeSqlOrThrow(
EventsTableFlatEvents
.preparePagedGetActiveContracts(sqlFunctions)(
range = range,
filter = filter,
pageSize = pageSize,
)
.executeSql,
queryNonPruned.executeSqlOrThrow(
getActiveContracts(
range,
filter,
pageSize,
).executeSql,
activeAt,
pruned =>
s"Active contracts request after ${activeAt.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -378,8 +397,10 @@ private[appendonlydao] final class TransactionsReader(
val query = (range: EventsRange[(Offset, Long)]) => {
implicit connection: Connection =>
QueryNonPruned.executeSqlOrThrow(
ContractStateEventsReader.readRawEvents(range),
queryNonPruned.executeSqlOrThrow(
storageBackend.contractStateEvents(range.startExclusive._2, range.endInclusive._2)(
connection
),
range.startExclusive._1,
pruned =>
s"Transactions request from ${range.startExclusive._1.toHexString} to ${range.endInclusive._1.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -416,7 +437,7 @@ private[appendonlydao] final class TransactionsReader(
EventsRange(startExclusive = (a.eventOffset, a.eventSequentialId), endInclusive = endEventSeqId)
private def nextPageRangeContracts(endEventSeqId: (Offset, Long))(
raw: ContractStateEventsReader.RawContractStateEvent
raw: StorageBackend.RawContractStateEvent
): EventsRange[(Offset, Long)] =
EventsRange(
startExclusive = (raw.offset, raw.eventSequentialId),
@ -428,8 +449,8 @@ private[appendonlydao] final class TransactionsReader(
): Future[EventsRange[(Offset, Long)]] =
dispatcher
.executeSql(dbMetrics.getAcsEventSeqIdRange)(implicit connection =>
QueryNonPruned.executeSql(
EventsRange.readEventSeqIdRange(activeAt)(connection),
queryNonPruned.executeSql(
eventSeqIdReader.readEventSeqIdRange(activeAt)(connection),
activeAt,
pruned =>
s"Active contracts request after ${activeAt.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -449,8 +470,10 @@ private[appendonlydao] final class TransactionsReader(
)(implicit loggingContext: LoggingContext): Future[EventsRange[(Offset, Long)]] =
dispatcher
.executeSql(dbMetrics.getEventSeqIdRange)(implicit connection =>
QueryNonPruned.executeSql(
EventsRange.readEventSeqIdRange(EventsRange(startExclusive, endInclusive))(connection),
queryNonPruned.executeSql(
eventSeqIdReader.readEventSeqIdRange(EventsRange(startExclusive, endInclusive))(
connection
),
startExclusive,
pruned =>
s"Transactions request from ${startExclusive.toHexString} to ${endInclusive.toHexString} precedes pruned offset ${pruned.toHexString}",
@ -495,14 +518,14 @@ private[appendonlydao] final class TransactionsReader(
private def streamContractStateEvents(
queryMetric: DatabaseMetrics,
query: EventsRange[(Offset, Long)] => Connection => Vector[
ContractStateEventsReader.RawContractStateEvent
StorageBackend.RawContractStateEvent
],
getNextPageRange: ContractStateEventsReader.RawContractStateEvent => EventsRange[
getNextPageRange: StorageBackend.RawContractStateEvent => EventsRange[
(Offset, Long)
],
)(range: EventsRange[(Offset, Long)])(implicit
loggingContext: LoggingContext
): Source[ContractStateEventsReader.RawContractStateEvent, NotUsed] =
): Source[StorageBackend.RawContractStateEvent, NotUsed] =
PaginatingAsyncStream.streamFrom(range, getNextPageRange) { range1 =>
if (EventsRange.isEmpty(range1))
Future.successful(Vector.empty)

View File

@ -4,7 +4,6 @@
package com.daml.platform.store.appendonlydao
import akka.stream.scaladsl.Source
import anorm.{BatchSql, NamedParameter}
import com.daml.lf.transaction.Node.KeyWithMaintainers
// TODO append-only: revisit visibility, and necessity during cleanup
@ -110,7 +109,4 @@ package object events {
) =
Key.assertBuild(template, key.key)
def batch(query: String, parameters: Seq[Seq[NamedParameter]]): Option[BatchSql] =
if (parameters.isEmpty) None else Some(BatchSql(query, parameters.head, parameters.tail: _*))
}

View File

@ -3,17 +3,52 @@
package com.daml.platform.store.backend
import java.io.InputStream
import java.sql.Connection
import java.time.Instant
import com.daml.ledger.participant.state.v1.Offset
import com.daml.ledger.{ApplicationId, TransactionId}
import com.daml.ledger.api.domain.{LedgerId, ParticipantId, PartyDetails}
import com.daml.ledger.api.v1.command_completion_service.CompletionStreamResponse
import com.daml.ledger.participant.state.index.v2.PackageDetails
import com.daml.ledger.participant.state.v1.{Configuration, Offset}
import com.daml.lf.data.Ref
import com.daml.lf.data.Ref.PackageId
import com.daml.lf.ledger.EventId
import com.daml.platform
import com.daml.platform.store.DbType
import com.daml.platform.store.appendonlydao.events.{ContractId, EventsTable, Key, Raw}
import com.daml.platform.store.backend.StorageBackend.RawTransactionEvent
import com.daml.platform.store.backend.postgresql.PostgresStorageBackend
import com.daml.platform.store.entries.{ConfigurationEntry, PackageLedgerEntry, PartyLedgerEntry}
import com.daml.platform.store.interfaces.LedgerDaoContractsReader.KeyState
import com.daml.scalautil.NeverEqualsOverride
/** Encapsulates the interface which hides database technology specific implementations for parallel ingestion.
import scala.util.Try
/** Encapsulates the interface which hides database technology specific implementations.
* Naming convention for the interface methods, which requiring Connection:
* - read operations are represented as nouns (plural, singular form indicates cardinality)
* - write operations are represented as verbs
*
* @tparam DB_BATCH Since parallel ingestion comes also with batching, this implementation specific type allows separation of the CPU intensive batching operation from the pure IO intensive insertBatch operation.
*/
trait StorageBackend[DB_BATCH] {
trait StorageBackend[DB_BATCH]
extends IngestionStorageBackend[DB_BATCH]
with ParameterStorageBackend
with ConfigurationStorageBackend
with PartyStorageBackend
with PackageStorageBackend
with DeduplicationStorageBackend
with CompletionStorageBackend
with ContractStorageBackend
with EventStorageBackend {
def reset(connection: Connection): Unit
def enforceSynchronousCommit(connection: Connection): Unit
def duplicateKeyError: String // TODO: Avoid brittleness of error message checks
}
trait IngestionStorageBackend[DB_BATCH] {
/** The CPU intensive batching operation hides the batching logic, and the mapping to the database specific representation of the inserted data.
* This should be pure CPU logic without IO.
@ -27,18 +62,10 @@ trait StorageBackend[DB_BATCH] {
* No significant CPU load, mostly blocking JDBC communication with the database backend.
*
* @param connection to be used when inserting the batch
* @param batch to be inserted
* @param batch to be inserted
*/
def insertBatch(connection: Connection, batch: DB_BATCH): Unit
/** This method is used to update the parameters table: setting the new observable ledger-end, and other parameters.
* No significant CPU load, mostly blocking JDBC communication with the database backend.
*
* @param connection to be used when updating the parameters table
* @param params the parameters
*/
def updateParams(connection: Connection, params: StorageBackend.Params): Unit
/** Custom initialization code before the start of an ingestion.
* This method is responsible for the recovery after a possibly non-graceful stop of previous indexing.
* No significant CPU load, mostly blocking JDBC communication with the database backend.
@ -47,6 +74,18 @@ trait StorageBackend[DB_BATCH] {
* @return the LedgerEnd, which should be the basis for further indexing.
*/
def initialize(connection: Connection): StorageBackend.LedgerEnd
}
// TODO append-only: consolidate parameters table facing functions
trait ParameterStorageBackend {
/** This method is used to update the parameters table: setting the new observable ledger-end, and other parameters.
* No significant CPU load, mostly blocking JDBC communication with the database backend.
*
* @param connection to be used when updating the parameters table
* @param params the parameters
*/
def updateParams(params: StorageBackend.Params)(connection: Connection): Unit
/** Query the ledgerEnd, read from the parameters table.
* No significant CPU load, mostly blocking JDBC communication with the database backend.
@ -55,6 +94,233 @@ trait StorageBackend[DB_BATCH] {
* @return the LedgerEnd, which should be the basis for further indexing
*/
def ledgerEnd(connection: Connection): StorageBackend.LedgerEnd
def ledgerId(connection: Connection): Option[LedgerId]
def updateLedgerId(ledgerId: String)(connection: Connection): Unit
def participantId(connection: Connection): Option[ParticipantId]
def updateParticipantId(participantId: String)(connection: Connection): Unit
def ledgerEndOffset(connection: Connection): Offset
def ledgerEndOffsetAndSequentialId(connection: Connection): (Offset, Long)
def initialLedgerEnd(connection: Connection): Option[Offset]
/** Part of pruning process, this needs to be in the same transaction as the other pruning related database operations
*/
def updatePrunedUptoInclusive(prunedUpToInclusive: Offset)(connection: Connection): Unit
def prunedUptoInclusive(connection: Connection): Option[Offset]
}
trait ConfigurationStorageBackend {
def ledgerConfiguration(connection: Connection): Option[(Offset, Configuration)]
def configurationEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, ConfigurationEntry)]
}
trait PartyStorageBackend {
def partyEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, PartyLedgerEntry)]
def parties(parties: Seq[Ref.Party])(connection: Connection): List[PartyDetails]
def knownParties(connection: Connection): List[PartyDetails]
}
trait PackageStorageBackend {
def lfPackages(connection: Connection): Map[PackageId, PackageDetails]
def lfArchive(packageId: PackageId)(connection: Connection): Option[Array[Byte]]
def packageEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, PackageLedgerEntry)]
}
trait DeduplicationStorageBackend {
def deduplicatedUntil(deduplicationKey: String)(connection: Connection): Instant
def upsertDeduplicationEntry(
key: String,
submittedAt: Instant,
deduplicateUntil: Instant,
)(connection: Connection): Int
def removeExpiredDeduplicationData(currentTime: Instant)(connection: Connection): Unit
def stopDeduplicatingCommand(deduplicationKey: String)(connection: Connection): Unit
}
trait CompletionStorageBackend {
def commandCompletions(
startExclusive: Offset,
endInclusive: Offset,
applicationId: ApplicationId,
parties: Set[Ref.Party],
)(connection: Connection): List[CompletionStreamResponse]
/** Part of pruning process, this needs to be in the same transaction as the other pruning related database operations
*/
def pruneCompletions(pruneUpToInclusive: Offset)(connection: Connection): Unit
}
trait ContractStorageBackend {
def contractKeyGlobally(key: Key)(connection: Connection): Option[ContractId]
def maximumLedgerTime(ids: Set[ContractId])(connection: Connection): Try[Option[Instant]]
def keyState(key: Key, validAt: Long)(connection: Connection): KeyState
def contractState(contractId: ContractId, before: Long)(
connection: Connection
): Option[StorageBackend.RawContractState]
def activeContractWithArgument(readers: Set[Ref.Party], contractId: ContractId)(
connection: Connection
): Option[StorageBackend.RawContract]
def activeContractWithoutArgument(readers: Set[Ref.Party], contractId: ContractId)(
connection: Connection
): Option[String]
def contractKey(readers: Set[Ref.Party], key: Key)(
connection: Connection
): Option[ContractId]
def contractStateEvents(startExclusive: Long, endInclusive: Long)(
connection: Connection
): Vector[StorageBackend.RawContractStateEvent]
}
// TODO append-only: Event related query consolidation
trait EventStorageBackend {
/** Part of pruning process, this needs to be in the same transaction as the other pruning related database operations
*/
def pruneEvents(pruneUpToInclusive: Offset)(connection: Connection): Unit
def transactionsEventsSingleWildcardParty(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusive: Long,
parties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionsEventsSameTemplates(
startExclusive: Long,
endInclusive: Long,
parties: Set[Ref.Party],
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionsEventsMixedTemplates(
startExclusive: Long,
endInclusive: Long,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusive: Long,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
wildcardParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsSingleWildcardParty(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
parties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsSameTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
parties: Set[Ref.Party],
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsMixedTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def activeContractsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
wildcardParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def flatTransactionSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def flatTransactionMultiParty(
transactionId: TransactionId,
requestingParties: Set[Ref.Party],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]]
def transactionTreeSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]]
def transactionTreeMultiParty(
transactionId: TransactionId,
requestingParties: Set[Ref.Party],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]]
def transactionTreeEventsSingleParty(
startExclusive: Long,
endInclusive: Long,
requestingParty: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]]
def transactionTreeEventsMultiParty(
startExclusive: Long,
endInclusive: Long,
requestingParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]]
def maxEventSeqIdForOffset(offset: Offset)(connection: Connection): Option[Long]
def rawEvents(startExclusive: Long, endInclusive: Long)(
connection: Connection
): Vector[RawTransactionEvent]
}
object StorageBackend {
@ -62,6 +328,65 @@ object StorageBackend {
case class LedgerEnd(lastOffset: Option[Offset], lastEventSeqId: Option[Long])
case class RawContractState(
templateId: Option[String],
flatEventWitnesses: Set[Ref.Party],
createArgument: Option[InputStream],
createArgumentCompression: Option[Int],
eventKind: Int,
ledgerEffectiveTime: Option[Instant],
)
case class RawContract(
templateId: String,
createArgument: InputStream,
createArgumentCompression: Option[Int],
)
case class RawContractStateEvent(
eventKind: Int,
contractId: ContractId,
templateId: Option[Ref.Identifier],
ledgerEffectiveTime: Option[Instant],
createKeyValue: Option[InputStream],
createKeyCompression: Option[Int],
createArgument: Option[InputStream],
createArgumentCompression: Option[Int],
flatEventWitnesses: Set[Ref.Party],
eventSequentialId: Long,
offset: Offset,
)
case class RawTransactionEvent(
eventKind: Int,
transactionId: String,
nodeIndex: Int,
commandId: Option[String],
workflowId: Option[String],
eventId: EventId,
contractId: platform.store.appendonlydao.events.ContractId,
templateId: Option[platform.store.appendonlydao.events.Identifier],
ledgerEffectiveTime: Option[Instant],
createSignatories: Option[Array[String]],
createObservers: Option[Array[String]],
createAgreementText: Option[String],
createKeyValue: Option[InputStream],
createKeyCompression: Option[Int],
createArgument: Option[InputStream],
createArgumentCompression: Option[Int],
treeEventWitnesses: Set[String],
flatEventWitnesses: Set[String],
exerciseChoice: Option[String],
exerciseArgument: Option[InputStream],
exerciseArgumentCompression: Option[Int],
exerciseResult: Option[InputStream],
exerciseResultCompression: Option[Int],
exerciseActors: Option[Array[String]],
exerciseChildEventIds: Option[Array[String]],
eventSequentialId: Long,
offset: Offset,
) extends NeverEqualsOverride
def of(dbType: DbType): StorageBackend[_] =
dbType match {
case DbType.H2Database => throw new UnsupportedOperationException("H2 not supported yet")

View File

@ -0,0 +1,907 @@
// 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.backend.common
import java.sql.{Connection, PreparedStatement, ResultSet}
import java.time.Instant
import java.util.Date
import anorm.SqlParser.{array, binaryStream, bool, byteArray, date, flatten, get, int, long, str}
import anorm.{Macro, Row, RowParser, SQL, SimpleSql, SqlParser, SqlStringInterpolation, ~}
import com.daml.ledger.api.domain.{LedgerId, ParticipantId, PartyDetails}
import com.daml.ledger.participant.state.index.v2.PackageDetails
import com.daml.platform.store.Conversions.{
contractId,
eventId,
flatEventWitnessesColumn,
identifier,
instant,
ledgerString,
offset,
party,
}
import com.daml.ledger.participant.state.v1.{Configuration, Offset}
import com.daml.lf.data.Ref
import com.daml.lf.data.Ref.PackageId
import com.daml.platform.store.Conversions
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import com.daml.platform.store.appendonlydao.JdbcLedgerDao.{acceptType, rejectType}
import com.daml.platform.store.appendonlydao.events.{ContractId, Key}
import com.daml.platform.store.backend.StorageBackend
import com.daml.platform.store.backend.StorageBackend.RawTransactionEvent
import com.daml.platform.store.entries.{ConfigurationEntry, PackageLedgerEntry, PartyLedgerEntry}
import com.daml.platform.store.interfaces.LedgerDaoContractsReader.{
KeyAssigned,
KeyState,
KeyUnassigned,
}
import com.daml.scalautil.Statement.discard
import scala.collection.mutable
import scala.util.{Failure, Success, Try}
trait CommonStorageBackend[DB_BATCH] extends StorageBackend[DB_BATCH] {
// Parameters
private val preparedUpdateLedgerEnd: Connection => PreparedStatement = _.prepareStatement(
"""
|UPDATE
| parameters
|SET
| ledger_end = ?,
| ledger_end_sequential_id = ?
|
|""".stripMargin
)
def updateParams(params: StorageBackend.Params)(connection: Connection): Unit = {
val preparedStatement = preparedUpdateLedgerEnd(connection)
preparedStatement.setString(1, params.ledgerEnd.toHexString)
preparedStatement.setLong(2, params.eventSeqId)
preparedStatement.execute()
preparedStatement.close()
()
}
def ledgerEnd(connection: Connection): StorageBackend.LedgerEnd = {
val queryStatement = connection.createStatement()
val params = fetch(
queryStatement.executeQuery(
"""
|SELECT
| ledger_end,
| ledger_end_sequential_id
|FROM
| parameters
|
|""".stripMargin
)
)(rs =>
StorageBackend.LedgerEnd(
lastOffset =
if (rs.getString(1) == null) None
else Some(Offset.fromHexString(Ref.HexString.assertFromString(rs.getString(1)))),
lastEventSeqId = Option(rs.getLong(2)),
)
)
queryStatement.close()
assert(params.size == 1)
params.head
}
private def fetch[T](resultSet: ResultSet)(parse: ResultSet => T): Vector[T] = {
val buffer = mutable.ArrayBuffer.empty[T]
while (resultSet.next()) {
buffer += parse(resultSet)
}
resultSet.close()
buffer.toVector
}
private val EventSequentialIdBeforeBegin = 0L
private val TableName: String = "parameters"
private val LedgerIdColumnName: String = "ledger_id"
private val ParticipantIdColumnName: String = "participant_id"
private val LedgerEndColumnName: String = "ledger_end"
private val LedgerEndSequentialIdColumnName: String = "ledger_end_sequential_id"
private val LedgerIdParser: RowParser[LedgerId] =
ledgerString(LedgerIdColumnName).map(LedgerId(_))
private val ParticipantIdParser: RowParser[Option[ParticipantId]] =
Conversions.participantId(ParticipantIdColumnName).map(ParticipantId(_)).?
private val LedgerEndParser: RowParser[Option[Offset]] =
offset(LedgerEndColumnName).?
private val LedgerEndOrBeforeBeginParser: RowParser[Offset] =
LedgerEndParser.map(_.getOrElse(Offset.beforeBegin))
private val LedgerEndOffsetAndSequentialIdParser =
(offset(LedgerEndColumnName).? ~ long(LedgerEndSequentialIdColumnName).?)
.map(SqlParser.flatten)
.map {
case (Some(offset), Some(seqId)) => (offset, seqId)
case (Some(offset), None) => (offset, EventSequentialIdBeforeBegin)
case (None, None) => (Offset.beforeBegin, EventSequentialIdBeforeBegin)
case (None, Some(_)) =>
throw InvalidLedgerEnd("Parameters table in invalid state: ledger_end is not set")
}
private val SelectLedgerEnd: SimpleSql[Row] = SQL"select #$LedgerEndColumnName from #$TableName"
private val SelectLedgerEndOffsetAndSequentialId =
SQL"select #$LedgerEndColumnName, #$LedgerEndSequentialIdColumnName from #$TableName"
def ledgerId(connection: Connection): Option[LedgerId] =
SQL"select #$LedgerIdColumnName from #$TableName".as(LedgerIdParser.singleOpt)(connection)
def updateLedgerId(ledgerId: String)(connection: Connection): Unit =
discard(
SQL"insert into #$TableName(#$LedgerIdColumnName) values($ledgerId)".execute()(connection)
)
def participantId(connection: Connection): Option[ParticipantId] =
SQL"select #$ParticipantIdColumnName from #$TableName".as(ParticipantIdParser.single)(
connection
)
def updateParticipantId(participantId: String)(connection: Connection): Unit =
discard(
SQL"update #$TableName set #$ParticipantIdColumnName = $participantId".execute()(
connection
)
)
def ledgerEndOffset(connection: Connection): Offset =
SelectLedgerEnd.as(LedgerEndOrBeforeBeginParser.single)(connection)
// TODO mutable contract state cache - use only one getLedgerEnd method
def ledgerEndOffsetAndSequentialId(connection: Connection): (Offset, Long) =
SelectLedgerEndOffsetAndSequentialId.as(LedgerEndOffsetAndSequentialIdParser.single)(connection)
def initialLedgerEnd(connection: Connection): Option[Offset] =
SelectLedgerEnd.as(LedgerEndParser.single)(connection)
case class InvalidLedgerEnd(msg: String) extends RuntimeException(msg)
private val SQL_UPDATE_MOST_RECENT_PRUNING = SQL("""
|update parameters set participant_pruned_up_to_inclusive={pruned_up_to_inclusive}
|where participant_pruned_up_to_inclusive < {pruned_up_to_inclusive} or participant_pruned_up_to_inclusive is null
|""".stripMargin)
def updatePrunedUptoInclusive(prunedUpToInclusive: Offset)(connection: Connection): Unit = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL_UPDATE_MOST_RECENT_PRUNING
.on("pruned_up_to_inclusive" -> prunedUpToInclusive)
.execute()(connection)
()
}
private val SQL_SELECT_MOST_RECENT_PRUNING = SQL(
"select participant_pruned_up_to_inclusive from parameters"
)
def prunedUptoInclusive(connection: Connection): Option[Offset] =
SQL_SELECT_MOST_RECENT_PRUNING
.as(offset("participant_pruned_up_to_inclusive").?.single)(connection)
// Configurations
private val SQL_GET_CONFIGURATION_ENTRIES = SQL(
"""select
| configuration_entries.ledger_offset,
| configuration_entries.recorded_at,
| configuration_entries.submission_id,
| configuration_entries.typ,
| configuration_entries.configuration,
| configuration_entries.rejection_reason
| from
| configuration_entries,
| parameters
| where
| ledger_offset > {startExclusive} and
| ledger_offset <= {endInclusive} and
| parameters.ledger_end >= ledger_offset
| order by ledger_offset asc
| limit {pageSize}
| offset {queryOffset}""".stripMargin
)
private val SQL_GET_LATEST_CONFIGURATION_ENTRY = SQL(
s"""select
| configuration_entries.ledger_offset,
| configuration_entries.recorded_at,
| configuration_entries.submission_id,
| configuration_entries.typ,
| configuration_entries.configuration,
| configuration_entries.rejection_reason
| from
| configuration_entries,
| parameters
| where
| configuration_entries.typ = '$acceptType' and
| parameters.ledger_end >= ledger_offset
| order by ledger_offset desc
| limit 1""".stripMargin
)
private val configurationEntryParser: RowParser[(Offset, ConfigurationEntry)] =
(offset("ledger_offset") ~
str("typ") ~
str("submission_id") ~
str("rejection_reason").map(s => if (s.isEmpty) null else s).? ~
byteArray("configuration"))
.map(flatten)
.map { case (offset, typ, submissionId, rejectionReason, configBytes) =>
val config = Configuration
.decode(configBytes)
.fold(err => sys.error(s"Failed to decode configuration: $err"), identity)
offset ->
(typ match {
case `acceptType` =>
ConfigurationEntry.Accepted(
submissionId = submissionId,
configuration = config,
)
case `rejectType` =>
ConfigurationEntry.Rejected(
submissionId = submissionId,
rejectionReason = rejectionReason.getOrElse("<missing reason>"),
proposedConfiguration = config,
)
case _ =>
sys.error(s"getConfigurationEntries: Unknown configuration entry type: $typ")
})
}
def ledgerConfiguration(connection: Connection): Option[(Offset, Configuration)] =
SQL_GET_LATEST_CONFIGURATION_ENTRY
.on()
.asVectorOf(configurationEntryParser)(connection)
.collectFirst { case (offset, ConfigurationEntry.Accepted(_, configuration)) =>
offset -> configuration
}
def configurationEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, ConfigurationEntry)] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL_GET_CONFIGURATION_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> pageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(configurationEntryParser)(connection)
}
// Parties
private val SQL_GET_PARTY_ENTRIES = SQL(
"select * from party_entries where ledger_offset>{startExclusive} and ledger_offset<={endInclusive} order by ledger_offset asc limit {pageSize} offset {queryOffset}"
)
private val partyEntryParser: RowParser[(Offset, PartyLedgerEntry)] =
(offset("ledger_offset") ~
date("recorded_at") ~
ledgerString("submission_id").? ~
party("party").? ~
str("display_name").? ~
str("typ") ~
str("rejection_reason").? ~
bool("is_local").?)
.map(flatten)
.map {
case (
offset,
recordTime,
submissionIdOpt,
Some(party),
displayNameOpt,
`acceptType`,
None,
Some(isLocal),
) =>
offset ->
PartyLedgerEntry.AllocationAccepted(
submissionIdOpt,
recordTime.toInstant,
PartyDetails(party, displayNameOpt, isLocal),
)
case (
offset,
recordTime,
Some(submissionId),
None,
None,
`rejectType`,
Some(reason),
None,
) =>
offset -> PartyLedgerEntry.AllocationRejected(
submissionId,
recordTime.toInstant,
reason,
)
case invalidRow =>
sys.error(s"getPartyEntries: invalid party entry row: $invalidRow")
}
def partyEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, PartyLedgerEntry)] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL_GET_PARTY_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> pageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(partyEntryParser)(connection)
}
private val SQL_SELECT_MULTIPLE_PARTIES =
SQL(
"select parties.party, parties.display_name, parties.ledger_offset, parties.explicit, parties.is_local from parties, parameters where party in ({parties}) and parties.ledger_offset <= parameters.ledger_end"
)
private case class ParsedPartyData(
party: String,
displayName: Option[String],
ledgerOffset: Offset,
explicit: Boolean,
isLocal: Boolean,
)
private val PartyDataParser: RowParser[ParsedPartyData] = {
import com.daml.platform.store.Conversions.columnToOffset
Macro.parser[ParsedPartyData](
"party",
"display_name",
"ledger_offset",
"explicit",
"is_local",
)
}
private def constructPartyDetails(data: ParsedPartyData): PartyDetails =
PartyDetails(Ref.Party.assertFromString(data.party), data.displayName, data.isLocal)
def parties(parties: Seq[Ref.Party])(connection: Connection): List[PartyDetails] = {
import com.daml.platform.store.Conversions.partyToStatement
SQL_SELECT_MULTIPLE_PARTIES
.on("parties" -> parties)
.as(PartyDataParser.*)(connection)
.map(constructPartyDetails)
}
private val SQL_SELECT_ALL_PARTIES =
SQL(
"select parties.party, parties.display_name, parties.ledger_offset, parties.explicit, parties.is_local from parties, parameters where parameters.ledger_end >= parties.ledger_offset"
)
def knownParties(connection: Connection): List[PartyDetails] =
SQL_SELECT_ALL_PARTIES
.as(PartyDataParser.*)(connection)
.map(constructPartyDetails)
// Packages
private val SQL_SELECT_PACKAGES =
SQL(
"""select packages.package_id, packages.source_description, packages.known_since, packages.size
|from packages, parameters
|where packages.ledger_offset <= parameters.ledger_end
|""".stripMargin
)
private case class ParsedPackageData(
packageId: String,
sourceDescription: Option[String],
size: Long,
knownSince: Date,
)
private val PackageDataParser: RowParser[ParsedPackageData] =
Macro.parser[ParsedPackageData](
"package_id",
"source_description",
"size",
"known_since",
)
def lfPackages(connection: Connection): Map[PackageId, PackageDetails] =
SQL_SELECT_PACKAGES
.as(PackageDataParser.*)(connection)
.map(d =>
PackageId.assertFromString(d.packageId) -> PackageDetails(
d.size,
d.knownSince.toInstant,
d.sourceDescription,
)
)
.toMap
private val SQL_SELECT_PACKAGE =
SQL("""select packages.package
|from packages, parameters
|where package_id = {package_id}
|and packages.ledger_offset <= parameters.ledger_end
|""".stripMargin)
def lfArchive(packageId: PackageId)(connection: Connection): Option[Array[Byte]] = {
import com.daml.platform.store.Conversions.packageIdToStatement
SQL_SELECT_PACKAGE
.on(
"package_id" -> packageId
)
.as[Option[Array[Byte]]](SqlParser.byteArray("package").singleOpt)(connection)
}
private val SQL_GET_PACKAGE_ENTRIES = SQL(
"select * from package_entries where ledger_offset>{startExclusive} and ledger_offset<={endInclusive} order by ledger_offset asc limit {pageSize} offset {queryOffset}"
)
private val packageEntryParser: RowParser[(Offset, PackageLedgerEntry)] =
(offset("ledger_offset") ~
date("recorded_at") ~
ledgerString("submission_id").? ~
str("typ") ~
str("rejection_reason").?)
.map(flatten)
.map {
case (offset, recordTime, Some(submissionId), `acceptType`, None) =>
offset ->
PackageLedgerEntry.PackageUploadAccepted(submissionId, recordTime.toInstant)
case (offset, recordTime, Some(submissionId), `rejectType`, Some(reason)) =>
offset ->
PackageLedgerEntry.PackageUploadRejected(submissionId, recordTime.toInstant, reason)
case invalidRow =>
sys.error(s"packageEntryParser: invalid party entry row: $invalidRow")
}
def packageEntries(
startExclusive: Offset,
endInclusive: Offset,
pageSize: Int,
queryOffset: Long,
)(connection: Connection): Vector[(Offset, PackageLedgerEntry)] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL_GET_PACKAGE_ENTRIES
.on(
"startExclusive" -> startExclusive,
"endInclusive" -> endInclusive,
"pageSize" -> pageSize,
"queryOffset" -> queryOffset,
)
.asVectorOf(packageEntryParser)(connection)
}
// Deduplication
private val SQL_SELECT_COMMAND = SQL("""
|select deduplicate_until
|from participant_command_submissions
|where deduplication_key = {deduplicationKey}
""".stripMargin)
private case class ParsedCommandData(deduplicateUntil: Instant)
private val CommandDataParser: RowParser[ParsedCommandData] =
Macro.parser[ParsedCommandData](
"deduplicate_until"
)
def deduplicatedUntil(deduplicationKey: String)(connection: Connection): Instant =
SQL_SELECT_COMMAND
.on("deduplicationKey" -> deduplicationKey)
.as(CommandDataParser.single)(connection)
.deduplicateUntil
private val SQL_DELETE_EXPIRED_COMMANDS = SQL("""
|delete from participant_command_submissions
|where deduplicate_until < {currentTime}
""".stripMargin)
def removeExpiredDeduplicationData(currentTime: Instant)(connection: Connection): Unit = {
SQL_DELETE_EXPIRED_COMMANDS
.on("currentTime" -> currentTime)
.execute()(connection)
()
}
private val SQL_DELETE_COMMAND = SQL("""
|delete from participant_command_submissions
|where deduplication_key = {deduplicationKey}
""".stripMargin)
def stopDeduplicatingCommand(deduplicationKey: String)(connection: Connection): Unit = {
SQL_DELETE_COMMAND
.on("deduplicationKey" -> deduplicationKey)
.execute()(connection)
()
}
// Completions
def pruneCompletions(pruneUpToInclusive: Offset)(connection: Connection): Unit = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"delete from participant_command_completions where completion_offset <= $pruneUpToInclusive"
.execute()(connection)
()
}
// Contracts
def contractKeyGlobally(key: Key)(connection: Connection): Option[ContractId] = {
import com.daml.platform.store.Conversions.HashToStatement
SQL"""
WITH last_contract_key_create AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= parameters.ledger_end_sequential_id
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE NOT EXISTS
(SELECT 1
FROM participant_events, parameters
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND contract_id = last_contract_key_create.contract_id
);
"""
.as(contractId("contract_id").singleOpt)(connection)
}
private def emptyContractIds: Throwable =
new IllegalArgumentException(
"Cannot lookup the maximum ledger time for an empty set of contract identifiers"
)
private def notFound(missingContractIds: Set[ContractId]): Throwable =
new IllegalArgumentException(
s"The following contracts have not been found: ${missingContractIds.map(_.coid).mkString(", ")}"
)
// TODO append-only: revisit this approach when doing cleanup, so we can decide if it is enough or not.
// TODO append-only: consider pulling up traversal logic to upper layer
def maximumLedgerTime(
ids: Set[ContractId]
)(connection: Connection): Try[Option[Instant]] = {
if (ids.isEmpty) {
Failure(emptyContractIds)
} else {
def lookup(id: ContractId): Option[Option[Instant]] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1
),
create_event AS (
SELECT ledger_effective_time
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT ledger_effective_time
FROM participant_events, parameters
WHERE contract_id = $id
AND event_kind = 0 -- divulgence
AND event_sequential_id <= parameters.ledger_end_sequential_id
ORDER BY event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgance events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT ledger_effective_time
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;
""".as(instant("ledger_effective_time").?.singleOpt)(connection)
}
val queriedIds: List[(ContractId, Option[Option[Instant]])] = ids.toList
.map(id => id -> lookup(id))
val foundLedgerEffectiveTimes: List[Option[Instant]] = queriedIds
.collect { case (_, Some(found)) =>
found
}
if (foundLedgerEffectiveTimes.size != ids.size) {
val missingIds = queriedIds.collect { case (missingId, None) =>
missingId
}
Failure(notFound(missingIds.toSet))
} else Success(foundLedgerEffectiveTimes.max)
}
}
def keyState(key: Key, validAt: Long)(connection: Connection): KeyState = {
import com.daml.platform.store.Conversions.HashToStatement
SQL"""
WITH last_contract_key_create AS (
SELECT contract_id, flat_event_witnesses
FROM participant_events
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= $validAt
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id, flat_event_witnesses
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE NOT EXISTS -- check no archival visible
(SELECT 1
FROM participant_events
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= $validAt
AND contract_id = last_contract_key_create.contract_id
);
"""
.as(
(contractId("contract_id") ~ flatEventWitnessesColumn("flat_event_witnesses")).map {
case cId ~ stakeholders => KeyAssigned(cId, stakeholders)
}.singleOpt
)(connection)
.getOrElse(KeyUnassigned)
}
private val fullDetailsContractRowParser: RowParser[StorageBackend.RawContractState] =
(str("template_id").?
~ flatEventWitnessesColumn("flat_event_witnesses")
~ binaryStream("create_argument").?
~ int("create_argument_compression").?
~ int("event_kind") ~ get[Instant]("ledger_effective_time")(anorm.Column.columnToInstant).?)
.map(SqlParser.flatten)
.map(StorageBackend.RawContractState.tupled)
def contractState(contractId: ContractId, before: Long)(
connection: Connection
): Option[StorageBackend.RawContractState] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
SELECT
template_id,
flat_event_witnesses,
create_argument,
create_argument_compression,
event_kind,
ledger_effective_time
FROM participant_events
WHERE
contract_id = $contractId
AND event_sequential_id <= $before
AND (event_kind = 10 OR event_kind = 20)
ORDER BY event_sequential_id DESC
LIMIT 1;
"""
.as(fullDetailsContractRowParser.singleOpt)(connection)
}
private val contractStateRowParser: RowParser[StorageBackend.RawContractStateEvent] =
(int("event_kind") ~
contractId("contract_id") ~
identifier("template_id").? ~
instant("ledger_effective_time").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").? ~
binaryStream("create_argument").? ~
int("create_argument_compression").? ~
long("event_sequential_id") ~
flatEventWitnessesColumn("flat_event_witnesses") ~
offset("event_offset")).map {
case eventKind ~ contractId ~ templateId ~ ledgerEffectiveTime ~ createKeyValue ~ createKeyCompression ~ createArgument ~ createArgumentCompression ~ eventSequentialId ~ flatEventWitnesses ~ offset =>
StorageBackend.RawContractStateEvent(
eventKind,
contractId,
templateId,
ledgerEffectiveTime,
createKeyValue,
createKeyCompression,
createArgument,
createArgumentCompression,
flatEventWitnesses,
eventSequentialId,
offset,
)
}
def contractStateEvents(startExclusive: Long, endInclusive: Long)(
connection: Connection
): Vector[StorageBackend.RawContractStateEvent] =
SQL"""
SELECT
event_kind,
contract_id,
template_id,
create_key_value,
create_key_value_compression,
create_argument,
create_argument_compression,
flat_event_witnesses,
ledger_effective_time,
event_sequential_id,
event_offset
FROM
participant_events
WHERE
event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and (event_kind = 10 or event_kind = 20)
ORDER BY event_sequential_id ASC
"""
.asVectorOf(contractStateRowParser)(connection)
// Events
def pruneEvents(pruneUpToInclusive: Offset)(connection: Connection): Unit = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""
-- Divulgence events (only for contracts archived before the specified offset)
delete from participant_events_divulgence as delete_events
where
delete_events.event_offset <= $pruneUpToInclusive and
exists (
SELECT 1 FROM participant_events_consuming_exercise as archive_events
WHERE
archive_events.event_offset <= $pruneUpToInclusive AND
archive_events.contract_id = delete_events.contract_id
);
-- Create events (only for contracts archived before the specified offset)
delete from participant_events_create as delete_events
where
delete_events.event_offset <= $pruneUpToInclusive and
exists (
SELECT 1 FROM participant_events_consuming_exercise as archive_events
WHERE
archive_events.event_offset <= $pruneUpToInclusive AND
archive_events.contract_id = delete_events.contract_id
);
-- Exercise events (consuming)
delete from participant_events_consuming_exercise as delete_events
where
delete_events.event_offset <= $pruneUpToInclusive;
-- Exercise events (non-consuming)
delete from participant_events_non_consuming_exercise as delete_events
where
delete_events.event_offset <= $pruneUpToInclusive;
"""
.execute()(connection)
()
}
private val rawTransactionEventParser: RowParser[RawTransactionEvent] =
(int("event_kind") ~
str("transaction_id") ~
int("node_index") ~
str("command_id").? ~
str("workflow_id").? ~
eventId("event_id") ~
contractId("contract_id") ~
identifier("template_id").? ~
instant("ledger_effective_time").? ~
array[String]("create_signatories").? ~
array[String]("create_observers").? ~
str("create_agreement_text").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").? ~
binaryStream("create_argument").? ~
int("create_argument_compression").? ~
array[String]("tree_event_witnesses") ~
array[String]("flat_event_witnesses") ~
str("exercise_choice").? ~
binaryStream("exercise_argument").? ~
int("exercise_argument_compression").? ~
binaryStream("exercise_result").? ~
int("exercise_result_compression").? ~
array[String]("exercise_actors").? ~
array[String]("exercise_child_event_ids").? ~
long("event_sequential_id") ~
offset("event_offset")).map {
case eventKind ~ transactionId ~ nodeIndex ~ commandId ~ workflowId ~ eventId ~ contractId ~ templateId ~ ledgerEffectiveTime ~ createSignatories ~
createObservers ~ createAgreementText ~ createKeyValue ~ createKeyCompression ~
createArgument ~ createArgumentCompression ~ treeEventWitnesses ~ flatEventWitnesses ~ exerciseChoice ~
exerciseArgument ~ exerciseArgumentCompression ~ exerciseResult ~ exerciseResultCompression ~ exerciseActors ~
exerciseChildEventIds ~ eventSequentialId ~ offset =>
RawTransactionEvent(
eventKind,
transactionId,
nodeIndex,
commandId,
workflowId,
eventId,
contractId,
templateId,
ledgerEffectiveTime,
createSignatories,
createObservers,
createAgreementText,
createKeyValue,
createKeyCompression,
createArgument,
createArgumentCompression,
treeEventWitnesses.toSet,
flatEventWitnesses.toSet,
exerciseChoice,
exerciseArgument,
exerciseArgumentCompression,
exerciseResult,
exerciseResultCompression,
exerciseActors,
exerciseChildEventIds,
eventSequentialId,
offset,
)
}
def rawEvents(startExclusive: Long, endInclusive: Long)(
connection: Connection
): Vector[RawTransactionEvent] =
SQL"""
SELECT
event_kind,
transaction_id,
node_index,
command_id,
workflow_id,
event_id,
contract_id,
template_id,
ledger_effective_time,
create_signatories,
create_observers,
create_agreement_text,
create_key_value,
create_key_value_compression,
create_argument,
create_argument_compression,
tree_event_witnesses,
flat_event_witnesses,
exercise_choice,
exercise_argument,
exercise_argument_compression,
exercise_result,
exercise_result_compression,
exercise_actors,
exercise_child_event_ids,
event_sequential_id,
event_offset
FROM
participant_events
WHERE
event_sequential_id > #$startExclusive
and event_sequential_id <= #$endInclusive
and event_kind != 0
ORDER BY event_sequential_id ASC"""
.asVectorOf(rawTransactionEventParser)(connection)
}

View File

@ -0,0 +1,927 @@
// 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.backend.common
import java.io.InputStream
import java.sql.Connection
import java.time.Instant
import anorm.SqlParser.{array, binaryStream, bool, int, long, str}
import anorm.{RowParser, SqlParser, SqlStringInterpolation, ~}
import com.daml.ledger.{ApplicationId, TransactionId}
import com.daml.ledger.api.v1.command_completion_service.CompletionStreamResponse
import com.daml.ledger.api.v1.completion.Completion
import com.daml.ledger.participant.state.v1.Offset
import com.daml.lf.data.Ref
import com.daml.platform.store.CompletionFromTransaction.toApiCheckpoint
import com.daml.platform.store.Conversions.{contractId, identifier, instant, offset}
import com.daml.platform.store.appendonlydao.events.{ContractId, EventsTable, Identifier, Key, Raw}
import com.daml.platform.store.backend.StorageBackend
import com.google.rpc.status.Status
import com.daml.platform.store.SimpleSqlAsVectorOf.SimpleSqlAsVectorOf
import scala.collection.compat.immutable.ArraySeq
object TemplatedStorageBackend {
private val sharedCompletionColumns: RowParser[Offset ~ Instant ~ String] =
offset("completion_offset") ~ instant("record_time") ~ str("command_id")
private val acceptedCommandParser: RowParser[CompletionStreamResponse] =
sharedCompletionColumns ~ str("transaction_id") map {
case offset ~ recordTime ~ commandId ~ transactionId =>
CompletionStreamResponse(
checkpoint = toApiCheckpoint(recordTime, offset),
completions = Seq(Completion(commandId, Some(Status()), transactionId)),
)
}
private val rejectedCommandParser: RowParser[CompletionStreamResponse] =
sharedCompletionColumns ~ int("status_code") ~ str("status_message") map {
case offset ~ recordTime ~ commandId ~ statusCode ~ statusMessage =>
CompletionStreamResponse(
checkpoint = toApiCheckpoint(recordTime, offset),
completions = Seq(Completion(commandId, Some(Status(statusCode, statusMessage)))),
)
}
private val completionParser: RowParser[CompletionStreamResponse] =
acceptedCommandParser | rejectedCommandParser
def commandCompletions(
startExclusive: Offset,
endInclusive: Offset,
applicationId: ApplicationId,
submittersInPartiesClause: String,
)(connection: Connection): List[CompletionStreamResponse] = {
import com.daml.platform.store.Conversions.OffsetToStatement
import com.daml.platform.store.Conversions.ledgerStringToStatement
SQL"""
select
completion_offset,
record_time,
command_id,
transaction_id,
status_code,
status_message
from
participant_command_completions
where
completion_offset > $startExclusive and
completion_offset <= $endInclusive and
application_id = $applicationId and
#$submittersInPartiesClause
order by completion_offset asc"""
.as(completionParser.*)(connection)
}
private val contractRowParser: RowParser[StorageBackend.RawContract] =
(str("template_id")
~ binaryStream("create_argument")
~ int("create_argument_compression").?)
.map(SqlParser.flatten)
.map(StorageBackend.RawContract.tupled)
def activeContractWithArgument(
treeEventWitnessesWhereClause: String,
contractId: ContractId,
)(connection: Connection): Option[StorageBackend.RawContract] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause -- only use visible archivals
LIMIT 1
),
create_event AS (
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
-- no visibility check, as it is used to backfill missing template_id and create_arguments for divulged contracts
create_event_unrestricted AS (
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgance_event.template_id and .create_argument can be NULL
-- for certain integrations. For example, the KV integration exploits that
-- every participant node knows about all create events. The integration
-- therefore only communicates the change in visibility to the IndexDB, but
-- does not include a full divulgence event.
COALESCE(divulgence_events.template_id, create_event_unrestricted.template_id),
COALESCE(divulgence_events.create_argument, create_event_unrestricted.create_argument),
COALESCE(divulgence_events.create_argument_compression, create_event_unrestricted.create_argument_compression)
FROM participant_events AS divulgence_events LEFT OUTER JOIN create_event_unrestricted USING (contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_kind = 0 -- divulgence
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause
ORDER BY divulgence_events.event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgance events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, template_id, create_argument, create_argument_compression
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;"""
.as(contractRowParser.singleOpt)(connection)
}
private val contractWithoutValueRowParser: RowParser[String] =
str("template_id")
def activeContractWithoutArgument(
treeEventWitnessesWhereClause: String,
contractId: ContractId,
)(connection: Connection): Option[String] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
WITH archival_event AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause -- only use visible archivals
LIMIT 1
),
create_event AS (
SELECT contract_id, template_id
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
-- no visibility check, as it is used to backfill missing template_id and create_arguments for divulged contracts
create_event_unrestricted AS (
SELECT contract_id, template_id
FROM participant_events, parameters
WHERE contract_id = $contractId
AND event_kind = 10 -- create
AND event_sequential_id <= parameters.ledger_end_sequential_id
LIMIT 1 -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgance_event.template_id can be NULL
-- for certain integrations. For example, the KV integration exploits that
-- every participant node knows about all create events. The integration
-- therefore only communicates the change in visibility to the IndexDB, but
-- does not include a full divulgence event.
COALESCE(divulgence_events.template_id, create_event_unrestricted.template_id)
FROM participant_events AS divulgence_events LEFT OUTER JOIN create_event_unrestricted USING (contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_kind = 0 -- divulgence
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND #$treeEventWitnessesWhereClause
ORDER BY divulgence_events.event_sequential_id
-- prudent engineering: make results more stable by preferring earlier divulgence events
-- Results might still change due to pruning.
LIMIT 1
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, template_id
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
LIMIT 1;
""".as(contractWithoutValueRowParser.singleOpt)(connection)
}
def contractKey(flatEventWitnesses: String => String, key: Key)(
connection: Connection
): Option[ContractId] = {
import com.daml.platform.store.Conversions.HashToStatement
SQL"""
WITH last_contract_key_create AS (
SELECT participant_events.*
FROM participant_events, parameters
WHERE event_kind = 10 -- create
AND create_key_hash = ${key.hash}
AND event_sequential_id <= parameters.ledger_end_sequential_id
-- do NOT check visibility here, as otherwise we do not abort the scan early
ORDER BY event_sequential_id DESC
LIMIT 1
)
SELECT contract_id
FROM last_contract_key_create -- creation only, as divulged contracts cannot be fetched by key
WHERE #${flatEventWitnesses("last_contract_key_create")} -- check visibility only here
AND NOT EXISTS -- check no archival visible
(SELECT 1
FROM participant_events, parameters
WHERE event_kind = 20 -- consuming exercise
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND #${flatEventWitnesses("participant_events")}
AND contract_id = last_contract_key_create.contract_id
);
""".as(contractId("contract_id").singleOpt)(connection)
}
private val selectColumnsForTransactions =
Seq(
"event_offset",
"transaction_id",
"node_index",
"event_sequential_id",
"ledger_effective_time",
"workflow_id",
"participant_events.event_id",
"contract_id",
"template_id",
"create_argument",
"create_argument_compression",
"create_signatories",
"create_observers",
"create_agreement_text",
"create_key_value",
"create_key_value_compression",
).mkString(", ")
private val selectColumnsForACS =
Seq(
"active_cs.event_offset",
"active_cs.transaction_id",
"active_cs.node_index",
"active_cs.event_sequential_id",
"active_cs.ledger_effective_time",
"active_cs.workflow_id",
"active_cs.event_id",
"active_cs.contract_id",
"active_cs.template_id",
"active_cs.create_argument",
"active_cs.create_argument_compression",
"active_cs.create_signatories",
"active_cs.create_observers",
"active_cs.create_agreement_text",
"active_cs.create_key_value",
"active_cs.create_key_value_compression",
).mkString(", ")
private type SharedRow =
Offset ~ String ~ Int ~ Long ~ String ~ String ~ Instant ~ Identifier ~ Option[String] ~
Option[String] ~ Array[String]
private val sharedRow: RowParser[SharedRow] =
offset("event_offset") ~
str("transaction_id") ~
int("node_index") ~
long("event_sequential_id") ~
str("event_id") ~
str("contract_id") ~
instant("ledger_effective_time") ~
identifier("template_id") ~
str("command_id").? ~
str("workflow_id").? ~
array[String]("event_witnesses")
private type CreatedEventRow =
SharedRow ~ InputStream ~ Option[Int] ~ Array[String] ~ Array[String] ~ Option[String] ~
Option[InputStream] ~ Option[Int]
private val createdEventRow: RowParser[CreatedEventRow] =
sharedRow ~
binaryStream("create_argument") ~
int("create_argument_compression").? ~
array[String]("create_signatories") ~
array[String]("create_observers") ~
str("create_agreement_text").? ~
binaryStream("create_key_value").? ~
int("create_key_value_compression").?
private type ExercisedEventRow =
SharedRow ~ Boolean ~ String ~ InputStream ~ Option[Int] ~ Option[InputStream] ~ Option[Int] ~
Array[String] ~ Array[String]
private val exercisedEventRow: RowParser[ExercisedEventRow] =
sharedRow ~
bool("exercise_consuming") ~
str("exercise_choice") ~
binaryStream("exercise_argument") ~
int("exercise_argument_compression").? ~
binaryStream("exercise_result").? ~
int("exercise_result_compression").? ~
array[String]("exercise_actors") ~
array[String]("exercise_child_event_ids")
private type ArchiveEventRow = SharedRow
private val archivedEventRow: RowParser[ArchiveEventRow] = sharedRow
private val createdFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent.Created]] =
createdEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~
templateId ~ commandId ~ workflowId ~ eventWitnesses ~ createArgument ~ createArgumentCompression ~
createSignatories ~ createObservers ~ createAgreementText ~ createKeyValue ~ createKeyValueCompression =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.FlatEvent.Created(
eventId = eventId,
contractId = contractId,
templateId = templateId,
createArgument = createArgument,
createArgumentCompression = createArgumentCompression,
createSignatories = ArraySeq.unsafeWrapArray(createSignatories),
createObservers = ArraySeq.unsafeWrapArray(createObservers),
createAgreementText = createAgreementText,
createKeyValue = createKeyValue,
createKeyValueCompression = createKeyValueCompression,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val archivedFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent.Archived]] =
archivedEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.FlatEvent.Archived(
eventId = eventId,
contractId = contractId,
templateId = templateId,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val rawFlatEventParser: RowParser[EventsTable.Entry[Raw.FlatEvent]] =
createdFlatEventParser | archivedFlatEventParser
def transactionsEventsSingleWildcardParty(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
SQL"""
select #$selectColumnsForTransactions, array[$party] as event_witnesses,
case when submitters = array[$party]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def transactionsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
witnessesWhereClause: String,
templateIds: Set[Ref.Identifier],
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
import com.daml.platform.store.Conversions.IdentifierToStatement
SQL"""
select #$selectColumnsForTransactions, array[$party] as event_witnesses,
case when submitters = array[$party]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
and template_id in ($templateIds)
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def transactionsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusive: Long,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
SQL"""
select #$selectColumnsForTransactions, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def transactionsEventsSameTemplates(
startExclusive: Long,
endInclusive: Long,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
templateIds: Set[Ref.Identifier],
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.IdentifierToStatement
SQL"""
select #$selectColumnsForTransactions, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
and template_id in ($templateIds)
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def transactionsEventsMixedTemplates(
startExclusive: Long,
endInclusive: Long,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
partiesAndTemplatesCondition: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
SQL"""
select #$selectColumnsForTransactions, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$partiesAndTemplatesCondition
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def transactionsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusive: Long,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
partiesAndTemplatesCondition: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
SQL"""
select #$selectColumnsForTransactions, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and (#$witnessesWhereClause or #$partiesAndTemplatesCondition)
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsSingleWildcardParty(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""select #$selectColumnsForACS, array[$party] as event_witnesses,
case when active_cs.submitters = array[$party]::text[] then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and #$witnessesWhereClause
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
templateIds: Set[Ref.Identifier],
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
import com.daml.platform.store.Conversions.OffsetToStatement
import com.daml.platform.store.Conversions.IdentifierToStatement
SQL"""select #$selectColumnsForACS, array[$party] as event_witnesses,
case when active_cs.submitters = array[$party]::text[] then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and #$witnessesWhereClause
and active_cs.template_id in ($templateIds)
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""select #$selectColumnsForACS, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and #$witnessesWhereClause
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsSameTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
templateIds: Set[Ref.Identifier],
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.OffsetToStatement
import com.daml.platform.store.Conversions.IdentifierToStatement
SQL"""select #$selectColumnsForACS, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and #$witnessesWhereClause
and active_cs.template_id in ($templateIds)
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsMixedTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
partiesAndTemplatesCondition: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""select #$selectColumnsForACS, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and #$partiesAndTemplatesCondition
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def activeContractsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
witnessesWhereClause: String,
partiesAndTemplatesCondition: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""select #$selectColumnsForACS, #$filteredWitnessesClause as event_witnesses,
case when #$submittersInPartiesClause then active_cs.command_id else '' end as command_id
from participant_events as active_cs
where active_cs.event_kind = 10 -- create
and active_cs.event_sequential_id > $startExclusive
and active_cs.event_sequential_id <= $endInclusiveSeq
and not exists (
select 1
from participant_events as archived_cs
where
archived_cs.contract_id = active_cs.contract_id and
archived_cs.event_kind = 20 and -- consuming
archived_cs.event_offset <= $endInclusiveOffset
)
and (#$witnessesWhereClause or #$partiesAndTemplatesCondition)
order by active_cs.event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawFlatEventParser)(connection)
}
def flatTransactionSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
witnessesWhereClause: String,
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
import com.daml.platform.store.Conversions.ledgerStringToStatement
SQL"""select #$selectColumnsForTransactions, array[$requestingParty] as event_witnesses,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id"""
.asVectorOf(rawFlatEventParser)(connection)
}
def flatTransactionMultiParty(
transactionId: TransactionId,
witnessesWhereClause: String,
submittersInPartiesClause: String,
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
import com.daml.platform.store.Conversions.ledgerStringToStatement
SQL"""select #$selectColumnsForTransactions, flat_event_witnesses as event_witnesses,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id"""
.asVectorOf(rawFlatEventParser)(connection)
}
private val createdTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent.Created]] =
createdEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses ~ createArgument ~ createArgumentCompression ~ createSignatories ~ createObservers ~ createAgreementText ~ createKeyValue ~ createKeyValueCompression =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.TreeEvent.Created(
eventId = eventId,
contractId = contractId,
templateId = templateId,
createArgument = createArgument,
createArgumentCompression = createArgumentCompression,
createSignatories = ArraySeq.unsafeWrapArray(createSignatories),
createObservers = ArraySeq.unsafeWrapArray(createObservers),
createAgreementText = createAgreementText,
createKeyValue = createKeyValue,
createKeyValueCompression = createKeyValueCompression,
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val exercisedTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent.Exercised]] =
exercisedEventRow map {
case eventOffset ~ transactionId ~ nodeIndex ~ eventSequentialId ~ eventId ~ contractId ~ ledgerEffectiveTime ~ templateId ~ commandId ~ workflowId ~ eventWitnesses ~ exerciseConsuming ~ exerciseChoice ~ exerciseArgument ~ exerciseArgumentCompression ~ exerciseResult ~ exerciseResultCompression ~ exerciseActors ~ exerciseChildEventIds =>
// ArraySeq.unsafeWrapArray is safe here
// since we get the Array from parsing and don't let it escape anywhere.
EventsTable.Entry(
eventOffset = eventOffset,
transactionId = transactionId,
nodeIndex = nodeIndex,
eventSequentialId = eventSequentialId,
ledgerEffectiveTime = ledgerEffectiveTime,
commandId = commandId.getOrElse(""),
workflowId = workflowId.getOrElse(""),
event = Raw.TreeEvent.Exercised(
eventId = eventId,
contractId = contractId,
templateId = templateId,
exerciseConsuming = exerciseConsuming,
exerciseChoice = exerciseChoice,
exerciseArgument = exerciseArgument,
exerciseArgumentCompression = exerciseArgumentCompression,
exerciseResult = exerciseResult,
exerciseResultCompression = exerciseResultCompression,
exerciseActors = ArraySeq.unsafeWrapArray(exerciseActors),
exerciseChildEventIds = ArraySeq.unsafeWrapArray(exerciseChildEventIds),
eventWitnesses = ArraySeq.unsafeWrapArray(eventWitnesses),
),
)
}
private val rawTreeEventParser: RowParser[EventsTable.Entry[Raw.TreeEvent]] =
createdTreeEventParser | exercisedTreeEventParser
private val selectColumnsForTransactionTree = Seq(
"event_offset",
"transaction_id",
"node_index",
"event_sequential_id",
"participant_events.event_id",
"contract_id",
"ledger_effective_time",
"template_id",
"workflow_id",
"create_argument",
"create_argument_compression",
"create_signatories",
"create_observers",
"create_agreement_text",
"create_key_value",
"create_key_value_compression",
"exercise_choice",
"exercise_argument",
"exercise_argument_compression",
"exercise_result",
"exercise_result_compression",
"exercise_actors",
"exercise_child_event_ids",
).mkString(", ")
def transactionTreeSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
witnessesWhereClause: String,
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
import com.daml.platform.store.Conversions.ledgerStringToStatement
SQL"""select #$selectColumnsForTransactionTree, array[$requestingParty] as event_witnesses,
event_kind = 20 as exercise_consuming,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by node_index asc"""
.asVectorOf(rawTreeEventParser)(connection)
}
def transactionTreeMultiParty(
transactionId: TransactionId,
witnessesWhereClause: String,
submittersInPartiesClause: String,
filteredWitnessesClause: String,
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] = {
import com.daml.platform.store.Conversions.ledgerStringToStatement
SQL"""select #$selectColumnsForTransactionTree, #$filteredWitnessesClause as event_witnesses,
event_kind = 20 as exercise_consuming,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
join parameters on
(participant_pruned_up_to_inclusive is null or event_offset > participant_pruned_up_to_inclusive)
and event_offset <= ledger_end
where transaction_id = $transactionId and #$witnessesWhereClause and event_kind != 0 -- we do not want to fetch divulgence events
order by node_index asc"""
.asVectorOf(rawTreeEventParser)(connection)
}
def transactionTreeEventsSingleParty(
startExclusive: Long,
endInclusive: Long,
requestingParty: Ref.Party,
witnessesWhereClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] = {
import com.daml.platform.store.Conversions.partyToStatement
SQL"""
select #$selectColumnsForTransactionTree, array[$requestingParty] as event_witnesses,
event_kind = 20 as exercise_consuming,
case when submitters = array[$requestingParty]::text[] then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawTreeEventParser)(connection)
}
def transactionTreeEventsMultiParty(
startExclusive: Long,
endInclusive: Long,
witnessesWhereClause: String,
filteredWitnessesClause: String,
submittersInPartiesClause: String,
limitExpr: String,
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] = {
SQL"""
select #$selectColumnsForTransactionTree, #$filteredWitnessesClause as event_witnesses,
event_kind = 20 as exercise_consuming,
case when #$submittersInPartiesClause then command_id else '' end as command_id
from participant_events
where event_sequential_id > $startExclusive
and event_sequential_id <= $endInclusive
and #$witnessesWhereClause
and event_kind != 0 -- we do not want to fetch divulgence events
order by event_sequential_id #$limitExpr"""
.withFetchSize(fetchSizeHint)
.asVectorOf(rawTreeEventParser)(connection)
}
}

View File

@ -3,15 +3,23 @@
package com.daml.platform.store.backend.postgresql
import java.sql.{Connection, PreparedStatement, ResultSet}
import java.sql.{Connection, PreparedStatement}
import java.time.Instant
import anorm.SQL
import anorm.SqlStringInterpolation
import anorm.SqlParser.get
import com.daml.ledger.{ApplicationId, TransactionId}
import com.daml.ledger.api.v1.command_completion_service.CompletionStreamResponse
import com.daml.ledger.participant.state.v1.Offset
import com.daml.lf.data.Ref
import com.daml.platform.store.appendonlydao.events.{ContractId, EventsTable, Key, Party, Raw}
import com.daml.platform.store.backend.common.{CommonStorageBackend, TemplatedStorageBackend}
import com.daml.platform.store.backend.{DbDto, StorageBackend}
import scala.collection.mutable
private[backend] object PostgresStorageBackend extends StorageBackend[PostgresDbBatch] {
private[backend] object PostgresStorageBackend
extends StorageBackend[PostgresDbBatch]
with CommonStorageBackend[PostgresDbBatch] {
private val preparedDeleteCommandSubmissions =
"""
@ -63,26 +71,6 @@ private[backend] object PostgresStorageBackend extends StorageBackend[PostgresDb
)
}
private val preparedUpdateLedgerEnd: Connection => PreparedStatement = _.prepareStatement(
"""
|UPDATE
| parameters
|SET
| ledger_end = ?,
| ledger_end_sequential_id = ?
|
|""".stripMargin
)
override def updateParams(connection: Connection, params: StorageBackend.Params): Unit = {
val preparedStatement = preparedUpdateLedgerEnd(connection)
preparedStatement.setString(1, params.ledgerEnd.toHexString)
preparedStatement.setLong(2, params.eventSeqId)
preparedStatement.execute()
preparedStatement.close()
()
}
override def initialize(connection: Connection): StorageBackend.LedgerEnd = {
val result @ StorageBackend.LedgerEnd(offset, _) = ledgerEnd(connection)
@ -98,32 +86,6 @@ private[backend] object PostgresStorageBackend extends StorageBackend[PostgresDb
result
}
override def ledgerEnd(connection: Connection): StorageBackend.LedgerEnd = {
val queryStatement = connection.createStatement()
val params = fetch(
queryStatement.executeQuery(
"""
|SELECT
| ledger_end,
| ledger_end_sequential_id
|FROM
| parameters
|
|""".stripMargin
)
)(rs =>
StorageBackend.LedgerEnd(
lastOffset =
if (rs.getString(1) == null) None
else Some(Offset.fromHexString(Ref.HexString.assertFromString(rs.getString(1)))),
lastEventSeqId = Option(rs.getLong(2)),
)
)
queryStatement.close()
assert(params.size == 1)
params.head
}
private val preparedDeleteIngestionOverspillEntries: Connection => PreparedStatement =
_.prepareStatement(
"""
@ -170,14 +132,448 @@ private[backend] object PostgresStorageBackend extends StorageBackend[PostgresDb
|""".stripMargin
)
private def fetch[T](resultSet: ResultSet)(parse: ResultSet => T): Vector[T] = {
val buffer = mutable.ArrayBuffer.empty[T]
while (resultSet.next()) {
buffer += parse(resultSet)
}
resultSet.close()
buffer.toVector
override def batch(dbDtos: Vector[DbDto]): PostgresDbBatch = PostgresDbBatch(dbDtos)
val SQL_INSERT_COMMAND: String =
"""insert into participant_command_submissions as pcs (deduplication_key, deduplicate_until)
|values ({deduplicationKey}, {deduplicateUntil})
|on conflict (deduplication_key)
| do update
| set deduplicate_until={deduplicateUntil}
| where pcs.deduplicate_until < {submittedAt}""".stripMargin
def upsertDeduplicationEntry(
key: String,
submittedAt: Instant,
deduplicateUntil: Instant,
)(connection: Connection): Int =
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt,
"deduplicateUntil" -> deduplicateUntil,
)
.executeUpdate()(connection)
def reset(connection: Connection): Unit = {
SQL("""truncate table configuration_entries cascade;
|truncate table package_entries cascade;
|truncate table parameters cascade;
|truncate table participant_command_completions cascade;
|truncate table participant_command_submissions cascade;
|truncate table participant_events_divulgence cascade;
|truncate table participant_events_create cascade;
|truncate table participant_events_consuming_exercise cascade;
|truncate table participant_events_non_consuming_exercise cascade;
|truncate table parties cascade;
|truncate table party_entries cascade;
|""".stripMargin)
.execute()(connection)
()
}
override def batch(dbDtos: Vector[DbDto]): PostgresDbBatch = PostgresDbBatch(dbDtos)
def enforceSynchronousCommit(connnection: Connection): Unit = {
val statement =
connnection.prepareStatement("SET LOCAL synchronous_commit = 'on'")
try {
statement.execute()
()
} finally {
statement.close()
}
}
val duplicateKeyError: String = "duplicate key"
def commandCompletions(
startExclusive: Offset,
endInclusive: Offset,
applicationId: ApplicationId,
parties: Set[Ref.Party],
)(connection: Connection): List[CompletionStreamResponse] =
TemplatedStorageBackend.commandCompletions(
startExclusive = startExclusive,
endInclusive = endInclusive,
applicationId = applicationId,
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", parties),
)(connection)
def activeContractWithArgument(readers: Set[Ref.Party], contractId: ContractId)(
connection: Connection
): Option[StorageBackend.RawContract] =
TemplatedStorageBackend.activeContractWithArgument(
treeEventWitnessesWhereClause = arrayIntersectionWhereClause("tree_event_witnesses", readers),
contractId = contractId,
)(connection)
def activeContractWithoutArgument(readers: Set[Ref.Party], contractId: ContractId)(
connection: Connection
): Option[String] =
TemplatedStorageBackend.activeContractWithoutArgument(
treeEventWitnessesWhereClause = arrayIntersectionWhereClause("tree_event_witnesses", readers),
contractId = contractId,
)(connection)
def contractKey(readers: Set[Ref.Party], key: Key)(
connection: Connection
): Option[ContractId] =
TemplatedStorageBackend.contractKey(
flatEventWitnesses = columnPrefix =>
arrayIntersectionWhereClause(s"$columnPrefix.flat_event_witnesses", readers),
key = key,
)(connection)
def transactionsEventsSingleWildcardParty(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.transactionsEventsSingleWildcardParty(
startExclusive = startExclusive,
endInclusive = endInclusive,
party = party,
witnessesWhereClause = arrayIntersectionWhereClause("flat_event_witnesses", Set(party)),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def transactionsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusive: Long,
party: Ref.Party,
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.transactionsEventsSinglePartyWithTemplates(
startExclusive = startExclusive,
endInclusive = endInclusive,
party = party,
witnessesWhereClause = arrayIntersectionWhereClause("flat_event_witnesses", Set(party)),
templateIds = templateIds,
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def transactionsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusive: Long,
parties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.transactionsEventsOnlyWildcardParties(
startExclusive = startExclusive,
endInclusive = endInclusive,
filteredWitnessesClause = arrayIntersectionValues("flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", parties),
witnessesWhereClause = arrayIntersectionWhereClause("flat_event_witnesses", parties),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def transactionsEventsSameTemplates(
startExclusive: Long,
endInclusive: Long,
parties: Set[Ref.Party],
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.transactionsEventsSameTemplates(
startExclusive = startExclusive,
endInclusive = endInclusive,
filteredWitnessesClause = arrayIntersectionValues("flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", parties),
witnessesWhereClause = arrayIntersectionWhereClause("flat_event_witnesses", parties),
templateIds = templateIds,
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def transactionsEventsMixedTemplates(
startExclusive: Long,
endInclusive: Long,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
val parties = partiesAndTemplateIds.map(_._1)
TemplatedStorageBackend.transactionsEventsMixedTemplates(
startExclusive = startExclusive,
endInclusive = endInclusive,
filteredWitnessesClause = arrayIntersectionValues("flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", parties),
partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause("flat_event_witnesses", partiesAndTemplateIds),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
}
def transactionsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusive: Long,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
wildcardParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
val parties = wildcardParties ++ partiesAndTemplateIds.map(_._1)
TemplatedStorageBackend.transactionsEventsMixedTemplatesWithWildcardParties(
startExclusive = startExclusive,
endInclusive = endInclusive,
filteredWitnessesClause = arrayIntersectionValues("flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", parties),
witnessesWhereClause = arrayIntersectionWhereClause("flat_event_witnesses", wildcardParties),
partiesAndTemplatesCondition =
formatPartiesAndTemplatesWhereClause("flat_event_witnesses", partiesAndTemplateIds),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
}
def activeContractsEventsSingleWildcardParty(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.activeContractsEventsSingleWildcardParty(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
party = party,
witnessesWhereClause =
arrayIntersectionWhereClause("active_cs.flat_event_witnesses", Set(party)),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def activeContractsEventsSinglePartyWithTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
party: Ref.Party,
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.activeContractsEventsSinglePartyWithTemplates(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
party = party,
templateIds = templateIds,
witnessesWhereClause =
arrayIntersectionWhereClause("active_cs.flat_event_witnesses", Set(party)),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def activeContractsEventsOnlyWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
parties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.activeContractsEventsOnlyWildcardParties(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
filteredWitnessesClause = arrayIntersectionValues("active_cs.flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("active_cs.submitters", parties),
witnessesWhereClause =
arrayIntersectionWhereClause("active_cs.flat_event_witnesses", parties),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def activeContractsEventsSameTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
parties: Set[Ref.Party],
templateIds: Set[Ref.Identifier],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.activeContractsEventsSameTemplates(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
templateIds = templateIds,
filteredWitnessesClause = arrayIntersectionValues("active_cs.flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("active_cs.submitters", parties),
witnessesWhereClause =
arrayIntersectionWhereClause("active_cs.flat_event_witnesses", parties),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def activeContractsEventsMixedTemplates(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
val parties = partiesAndTemplateIds.map(_._1)
TemplatedStorageBackend.activeContractsEventsMixedTemplates(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
filteredWitnessesClause = arrayIntersectionValues("active_cs.flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("active_cs.submitters", parties),
partiesAndTemplatesCondition = formatPartiesAndTemplatesWhereClause(
"active_cs.flat_event_witnesses",
partiesAndTemplateIds,
),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
}
def activeContractsEventsMixedTemplatesWithWildcardParties(
startExclusive: Long,
endInclusiveSeq: Long,
endInclusiveOffset: Offset,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
wildcardParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] = {
val parties = wildcardParties ++ partiesAndTemplateIds.map(_._1)
TemplatedStorageBackend.activeContractsEventsMixedTemplatesWithWildcardParties(
startExclusive = startExclusive,
endInclusiveSeq = endInclusiveSeq,
endInclusiveOffset = endInclusiveOffset,
filteredWitnessesClause = arrayIntersectionValues("active_cs.flat_event_witnesses", parties),
submittersInPartiesClause = arrayIntersectionWhereClause("active_cs.submitters", parties),
witnessesWhereClause =
arrayIntersectionWhereClause("active_cs.flat_event_witnesses", wildcardParties),
partiesAndTemplatesCondition = formatPartiesAndTemplatesWhereClause(
"active_cs.flat_event_witnesses",
partiesAndTemplateIds,
),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
}
def flatTransactionSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.flatTransactionSingleParty(
transactionId = transactionId,
requestingParty = requestingParty,
witnessesWhereClause =
arrayIntersectionWhereClause("flat_event_witnesses", Set(requestingParty)),
)(connection)
def flatTransactionMultiParty(
transactionId: TransactionId,
requestingParties: Set[Ref.Party],
)(connection: Connection): Vector[EventsTable.Entry[Raw.FlatEvent]] =
TemplatedStorageBackend.flatTransactionMultiParty(
transactionId = transactionId,
witnessesWhereClause =
arrayIntersectionWhereClause("flat_event_witnesses", requestingParties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", requestingParties),
)(connection)
def transactionTreeSingleParty(
transactionId: TransactionId,
requestingParty: Ref.Party,
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] =
TemplatedStorageBackend.transactionTreeSingleParty(
transactionId = transactionId,
requestingParty = requestingParty,
witnessesWhereClause =
arrayIntersectionWhereClause("tree_event_witnesses", Set(requestingParty)),
)(connection)
def transactionTreeMultiParty(
transactionId: TransactionId,
requestingParties: Set[Ref.Party],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] =
TemplatedStorageBackend.transactionTreeMultiParty(
transactionId = transactionId,
witnessesWhereClause =
arrayIntersectionWhereClause("tree_event_witnesses", requestingParties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", requestingParties),
filteredWitnessesClause = arrayIntersectionValues("tree_event_witnesses", requestingParties),
)(connection)
def transactionTreeEventsSingleParty(
startExclusive: Long,
endInclusive: Long,
requestingParty: Ref.Party,
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] =
TemplatedStorageBackend.transactionTreeEventsSingleParty(
startExclusive = startExclusive,
endInclusive = endInclusive,
requestingParty = requestingParty,
witnessesWhereClause =
arrayIntersectionWhereClause("tree_event_witnesses", Set(requestingParty)),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def transactionTreeEventsMultiParty(
startExclusive: Long,
endInclusive: Long,
requestingParties: Set[Ref.Party],
limit: Option[Int],
fetchSizeHint: Option[Int],
)(connection: Connection): Vector[EventsTable.Entry[Raw.TreeEvent]] =
TemplatedStorageBackend.transactionTreeEventsMultiParty(
startExclusive = startExclusive,
endInclusive = endInclusive,
witnessesWhereClause =
arrayIntersectionWhereClause("tree_event_witnesses", requestingParties),
filteredWitnessesClause = arrayIntersectionValues("tree_event_witnesses", requestingParties),
submittersInPartiesClause = arrayIntersectionWhereClause("submitters", requestingParties),
limitExpr = limitClause(limit),
fetchSizeHint = fetchSizeHint,
)(connection)
def maxEventSeqIdForOffset(offset: Offset)(connection: Connection): Option[Long] = {
import com.daml.platform.store.Conversions.OffsetToStatement
// This query could be: "select max(event_sequential_id) from participant_events where event_offset <= ${range.endInclusive}"
// however tests using PostgreSQL 12 with tens of millions of events have shown that the index
// on `event_offset` is not used unless we _hint_ at it by specifying `order by event_offset`
SQL"select max(event_sequential_id) from participant_events where event_offset <= $offset group by event_offset order by event_offset desc limit 1"
.as(get[Long](1).singleOpt)(connection)
}
private def format(parties: Set[Party]): String = parties.view.map(p => s"'$p'").mkString(",")
private def limitClause(to: Option[Int]): String = to.map(to => s"limit $to").getOrElse("")
private def arrayIntersectionWhereClause(arrayColumn: String, parties: Set[Ref.Party]): String =
s"$arrayColumn::text[] && array[${format(parties)}]::text[]"
private def arrayIntersectionValues(arrayColumn: String, parties: Set[Party]): String =
s"array(select unnest($arrayColumn) intersect select unnest(array[${format(parties)}]))"
private def formatPartiesAndTemplatesWhereClause(
witnessesAggregationColumn: String,
partiesAndTemplateIds: Set[(Ref.Party, Ref.Identifier)],
): String =
partiesAndTemplateIds.view
.map { case (p, i) =>
s"(${arrayIntersectionWhereClause(witnessesAggregationColumn, Set(p))} and template_id = '$i')"
}
.mkString("(", " or ", ")")
}

View File

@ -15,7 +15,6 @@ import com.daml.ledger.resources.{Resource, ResourceContext, ResourceOwner}
import com.daml.lf.transaction.GlobalKey
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.metrics.{Metrics, Timed}
import com.daml.platform.store.appendonlydao.EventSequentialId
import com.daml.platform.store.cache.ContractKeyStateValue._
import com.daml.platform.store.cache.ContractStateValue._
import com.daml.platform.store.cache.MutableCacheBackedContractStore._
@ -390,7 +389,9 @@ object MutableCacheBackedContractStore {
private[cache] class CacheIndex {
private val offsetRef =
new AtomicReference(Offset.beforeBegin -> EventSequentialId.beforeBegin)
new AtomicReference(
Offset.beforeBegin -> 0L
) // TODO append-only: FIXME parameters table consolidation
def set(offset: Offset, sequentialId: EventSequentialId): Unit =
offsetRef.set(offset -> sequentialId)

View File

@ -9,6 +9,7 @@ import com.daml.lf.data.Ref
import java.security.MessageDigest
import scalaz.syntax.tag._
// TODO append-only: move to store
object DeduplicationKeyMaker {
def make(commandId: domain.CommandId, submitters: List[Ref.Party]): String =
commandId.unwrap + "%" + hashSubmitters(submitters.sorted(Ordering.String).distinct)

View File

@ -17,7 +17,6 @@ import com.daml.lf.transaction.TransactionVersion
import com.daml.lf.value.Value.{ContractId, ValueInt64, ValueText, VersionedValue}
import com.daml.logging.LoggingContext
import com.daml.platform.index.BuffersUpdaterSpec.{contractStateEventMock, transactionLogUpdateMock}
import com.daml.platform.store.appendonlydao.EventSequentialId
import com.daml.platform.store.appendonlydao.events.{Contract, Key, Party}
import com.daml.platform.store.cache.MutableCacheBackedContractStore.EventSequentialId
import com.daml.platform.store.dao.events.ContractStateEvent
@ -95,7 +94,7 @@ final class BuffersUpdaterSpec
Offset,
EventSequentialId,
) => Source[((Offset, Long), TransactionLogUpdate), NotUsed] = {
case (Offset.beforeBegin, EventSequentialId.beforeBegin) =>
case (Offset.beforeBegin, 0L) => // TODO: append-only: FIXME consolidating parameters table
Source(
immutable.Iterable(
offset1 -> update1,

View File

@ -20,7 +20,6 @@ import com.daml.lf.transaction.test.TransactionBuilder
import com.daml.lf.value.Value.{ContractInst, ValueRecord, ValueText}
import com.daml.logging.LoggingContext
import com.daml.metrics.Metrics
import com.daml.platform.store.appendonlydao.EventSequentialId
import com.daml.platform.store.cache.ContractKeyStateValue.{Assigned, Unassigned}
import com.daml.platform.store.cache.ContractStateValue.{Active, Archived}
import com.daml.platform.store.cache.MutableCacheBackedContractStore.{
@ -143,7 +142,7 @@ class MutableCacheBackedContractStoreSpec
val sourceSubscriptionFixture
: (Offset, EventSequentialId) => Source[ContractStateEvent, NotUsed] = {
case (Offset.beforeBegin, EventSequentialId.beforeBegin) =>
case (Offset.beforeBegin, 0L) => // TODO append-only: FIXME parameters table consolidation
Source
.fromIterator(() => Iterator(created, archived, dummy))
// Simulate the source failure at the last event