Switching to composition at StorageBackend [DPP-704] (#11520)

* Switch to StorageBackendFactory
* Fixing tests

changelog_begin
changelog_end
This commit is contained in:
Marton Nagy 2021-11-03 23:53:05 +01:00 committed by GitHub
parent d006ad0e8b
commit 6c9416492d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
68 changed files with 2052 additions and 1421 deletions

View File

@ -20,7 +20,11 @@ import com.daml.platform.store.DbType.{
} }
import com.daml.platform.store.appendonlydao.events.{CompressionStrategy, LfValueTranslation} import com.daml.platform.store.appendonlydao.events.{CompressionStrategy, LfValueTranslation}
import com.daml.platform.store.backend.DataSourceStorageBackend.DataSourceConfig import com.daml.platform.store.backend.DataSourceStorageBackend.DataSourceConfig
import com.daml.platform.store.backend.StorageBackend import com.daml.platform.store.backend.{
DataSourceStorageBackend,
ResetStorageBackend,
StorageBackendFactory,
}
import com.daml.platform.store.backend.postgresql.PostgresDataSourceConfig import com.daml.platform.store.backend.postgresql.PostgresDataSourceConfig
import com.daml.platform.store.{DbType, LfValueTranslationCache} import com.daml.platform.store.{DbType, LfValueTranslationCache}
@ -37,7 +41,12 @@ object JdbcIndexer {
)(implicit materializer: Materializer, loggingContext: LoggingContext) { )(implicit materializer: Materializer, loggingContext: LoggingContext) {
def initialized(resetSchema: Boolean = false): ResourceOwner[Indexer] = { def initialized(resetSchema: Boolean = false): ResourceOwner[Indexer] = {
val storageBackend = StorageBackend.of(DbType.jdbcType(config.jdbcUrl)) val factory = StorageBackendFactory.of(DbType.jdbcType(config.jdbcUrl))
val dataSourceStorageBackend = factory.createDataSourceStorageBackend
val ingestionStorageBackend = factory.createIngestionStorageBackend
val parameterStorageBackend = factory.createParameterStorageBackend
val DBLockStorageBackend = factory.createDBLockStorageBackend
val resetStorageBackend = factory.createResetStorageBackend
val indexer = ParallelIndexerFactory( val indexer = ParallelIndexerFactory(
jdbcUrl = config.jdbcUrl, jdbcUrl = config.jdbcUrl,
inputMappingParallelism = config.inputMappingParallelism, inputMappingParallelism = config.inputMappingParallelism,
@ -58,14 +67,17 @@ object JdbcIndexer {
), ),
haConfig = config.haConfig, haConfig = config.haConfig,
metrics = metrics, metrics = metrics,
storageBackend = storageBackend, dbLockStorageBackend = DBLockStorageBackend,
dataSourceStorageBackend = dataSourceStorageBackend,
initializeParallelIngestion = InitializeParallelIngestion( initializeParallelIngestion = InitializeParallelIngestion(
providedParticipantId = config.participantId, providedParticipantId = config.participantId,
storageBackend = storageBackend, parameterStorageBackend = parameterStorageBackend,
ingestionStorageBackend = ingestionStorageBackend,
metrics = metrics, metrics = metrics,
), ),
parallelIndexerSubscription = ParallelIndexerSubscription( parallelIndexerSubscription = ParallelIndexerSubscription(
storageBackend = storageBackend, parameterStorageBackend = parameterStorageBackend,
ingestionStorageBackend = ingestionStorageBackend,
participantId = config.participantId, participantId = config.participantId,
translation = new LfValueTranslation( translation = new LfValueTranslation(
cache = lfValueTranslationCache, cache = lfValueTranslationCache,
@ -89,17 +101,20 @@ object JdbcIndexer {
readService = readService, readService = readService,
) )
if (resetSchema) { if (resetSchema) {
reset(storageBackend).flatMap(_ => indexer) reset(resetStorageBackend, dataSourceStorageBackend).flatMap(_ => indexer)
} else { } else {
indexer indexer
} }
} }
private def reset(storageBackend: StorageBackend[_]): ResourceOwner[Unit] = private def reset(
resetStorageBackend: ResetStorageBackend,
dataSourceStorageBackend: DataSourceStorageBackend,
): ResourceOwner[Unit] =
ResourceOwner.forFuture(() => ResourceOwner.forFuture(() =>
Future( Future(
Using.resource(storageBackend.createDataSource(config.jdbcUrl).getConnection)( Using.resource(dataSourceStorageBackend.createDataSource(config.jdbcUrl).getConnection)(
storageBackend.reset resetStorageBackend.reset
) )
)(servicesExecutionContext) )(servicesExecutionContext)
) )

View File

@ -20,7 +20,8 @@ import scala.concurrent.{ExecutionContext, Future}
private[platform] case class InitializeParallelIngestion( private[platform] case class InitializeParallelIngestion(
providedParticipantId: Ref.ParticipantId, providedParticipantId: Ref.ParticipantId,
storageBackend: IngestionStorageBackend[_] with ParameterStorageBackend, ingestionStorageBackend: IngestionStorageBackend[_],
parameterStorageBackend: ParameterStorageBackend,
metrics: Metrics, metrics: Metrics,
) { ) {
@ -40,7 +41,7 @@ private[platform] case class InitializeParallelIngestion(
s"Attempting to initialize with ledger ID $providedLedgerId and participant ID $providedParticipantId" s"Attempting to initialize with ledger ID $providedLedgerId and participant ID $providedParticipantId"
) )
_ <- dbDispatcher.executeSql(metrics.daml.index.db.initializeLedgerParameters)( _ <- dbDispatcher.executeSql(metrics.daml.index.db.initializeLedgerParameters)(
storageBackend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = providedLedgerId, ledgerId = providedLedgerId,
participantId = domain.ParticipantId(providedParticipantId), participantId = domain.ParticipantId(providedParticipantId),
@ -48,10 +49,10 @@ private[platform] case class InitializeParallelIngestion(
) )
) )
ledgerEnd <- dbDispatcher.executeSql(metrics.daml.index.db.getLedgerEnd)( ledgerEnd <- dbDispatcher.executeSql(metrics.daml.index.db.getLedgerEnd)(
storageBackend.ledgerEnd parameterStorageBackend.ledgerEnd
) )
_ <- dbDispatcher.executeSql(metrics.daml.parallelIndexer.initialization)( _ <- dbDispatcher.executeSql(metrics.daml.parallelIndexer.initialization)(
storageBackend.deletePartiallyIngestedData(ledgerEnd) ingestionStorageBackend.deletePartiallyIngestedData(ledgerEnd)
) )
} yield InitializeParallelIngestion.Initialized( } yield InitializeParallelIngestion.Initialized(
initialEventSeqId = ledgerEnd.map(_.lastEventSeqId).getOrElse(EventSequentialId.beforeBegin), initialEventSeqId = ledgerEnd.map(_.lastEventSeqId).getOrElse(EventSequentialId.beforeBegin),

View File

@ -35,7 +35,8 @@ object ParallelIndexerFactory {
dataSourceConfig: DataSourceConfig, dataSourceConfig: DataSourceConfig,
haConfig: HaConfig, haConfig: HaConfig,
metrics: Metrics, metrics: Metrics,
storageBackend: DBLockStorageBackend with DataSourceStorageBackend, dbLockStorageBackend: DBLockStorageBackend,
dataSourceStorageBackend: DataSourceStorageBackend,
initializeParallelIngestion: InitializeParallelIngestion, initializeParallelIngestion: InitializeParallelIngestion,
parallelIndexerSubscription: ParallelIndexerSubscription[_], parallelIndexerSubscription: ParallelIndexerSubscription[_],
mat: Materializer, mat: Materializer,
@ -53,7 +54,7 @@ object ParallelIndexerFactory {
Some(metrics.daml.parallelIndexer.batching.executor -> metrics.registry), Some(metrics.daml.parallelIndexer.batching.executor -> metrics.registry),
) )
haCoordinator <- haCoordinator <-
if (storageBackend.dbLockSupported) { if (dbLockStorageBackend.dbLockSupported) {
for { for {
executionContext <- ResourceOwner executionContext <- ResourceOwner
.forExecutorService(() => .forExecutorService(() =>
@ -67,10 +68,10 @@ object ParallelIndexerFactory {
timer <- ResourceOwner.forTimer(() => new Timer) timer <- ResourceOwner.forTimer(() => new Timer)
// this DataSource will be used to spawn the main connection where we keep the Indexer Main Lock // this DataSource will be used to spawn the main connection where we keep the Indexer Main Lock
// The life-cycle of such connections matches the life-cycle of a protectedExecution // The life-cycle of such connections matches the life-cycle of a protectedExecution
dataSource = storageBackend.createDataSource(jdbcUrl, dataSourceConfig) dataSource = dataSourceStorageBackend.createDataSource(jdbcUrl, dataSourceConfig)
} yield HaCoordinator.databaseLockBasedHaCoordinator( } yield HaCoordinator.databaseLockBasedHaCoordinator(
connectionFactory = () => dataSource.getConnection, connectionFactory = () => dataSource.getConnection,
storageBackend = storageBackend, storageBackend = dbLockStorageBackend,
executionContext = executionContext, executionContext = executionContext,
timer = timer, timer = timer,
haConfig = haConfig, haConfig = haConfig,
@ -85,7 +86,7 @@ object ParallelIndexerFactory {
.owner( .owner(
// this is the DataSource which will be wrapped by HikariCP, and which will drive the ingestion // this is the DataSource which will be wrapped by HikariCP, and which will drive the ingestion
// therefore this needs to be configured with the connection-init-hook, what we get from HaCoordinator // therefore this needs to be configured with the connection-init-hook, what we get from HaCoordinator
dataSource = storageBackend.createDataSource( dataSource = dataSourceStorageBackend.createDataSource(
jdbcUrl = jdbcUrl, jdbcUrl = jdbcUrl,
dataSourceConfig = dataSourceConfig, dataSourceConfig = dataSourceConfig,
connectionInitHook = Some(connectionInitializer.initialize), connectionInitHook = Some(connectionInitializer.initialize),

View File

@ -30,7 +30,8 @@ import com.daml.platform.store.backend.{
import scala.concurrent.Future import scala.concurrent.Future
private[platform] case class ParallelIndexerSubscription[DB_BATCH]( private[platform] case class ParallelIndexerSubscription[DB_BATCH](
storageBackend: IngestionStorageBackend[DB_BATCH] with ParameterStorageBackend, ingestionStorageBackend: IngestionStorageBackend[DB_BATCH],
parameterStorageBackend: ParameterStorageBackend,
participantId: Ref.ParticipantId, participantId: Ref.ParticipantId,
translation: LfValueTranslation, translation: LfValueTranslation,
compressionStrategy: CompressionStrategy, compressionStrategy: CompressionStrategy,
@ -69,12 +70,13 @@ private[platform] case class ParallelIndexerSubscription[DB_BATCH](
seqMapperZero = seqMapperZero(initialized.initialEventSeqId), seqMapperZero = seqMapperZero(initialized.initialEventSeqId),
seqMapper = seqMapper(metrics), seqMapper = seqMapper(metrics),
batchingParallelism = batchingParallelism, batchingParallelism = batchingParallelism,
batcher = batcherExecutor.execute(batcher(storageBackend.batch, metrics)), batcher = batcherExecutor.execute(batcher(ingestionStorageBackend.batch, metrics)),
ingestingParallelism = ingestionParallelism, ingestingParallelism = ingestionParallelism,
ingester = ingester(storageBackend.insertBatch, dbDispatcher, metrics), ingester = ingester(ingestionStorageBackend.insertBatch, dbDispatcher, metrics),
tailer = tailer(storageBackend.batch(Vector.empty)), tailer = tailer(ingestionStorageBackend.batch(Vector.empty)),
tailingRateLimitPerSecond = tailingRateLimitPerSecond, tailingRateLimitPerSecond = tailingRateLimitPerSecond,
ingestTail = ingestTail[DB_BATCH](storageBackend.updateLedgerEnd, dbDispatcher, metrics), ingestTail =
ingestTail[DB_BATCH](parameterStorageBackend.updateLedgerEnd, dbDispatcher, metrics),
)( )(
InstrumentedSource InstrumentedSource
.bufferedSource( .bufferedSource(

View File

@ -35,7 +35,14 @@ import com.daml.platform.server.api.validation.ErrorFactories
import com.daml.platform.store.Conversions._ import com.daml.platform.store.Conversions._
import com.daml.platform.store._ import com.daml.platform.store._
import com.daml.platform.store.appendonlydao.events._ import com.daml.platform.store.appendonlydao.events._
import com.daml.platform.store.backend.{ParameterStorageBackend, StorageBackend, UpdateToDbDto} import com.daml.platform.store.backend.{
DeduplicationStorageBackend,
ParameterStorageBackend,
ReadStorageBackend,
ResetStorageBackend,
StorageBackendFactory,
UpdateToDbDto,
}
import com.daml.platform.store.cache.MutableLedgerEndCache import com.daml.platform.store.cache.MutableLedgerEndCache
import com.daml.platform.store.entries.{ import com.daml.platform.store.entries.{
ConfigurationEntry, ConfigurationEntry,
@ -60,7 +67,10 @@ private class JdbcLedgerDao(
enricher: Option[ValueEnricher], enricher: Option[ValueEnricher],
sequentialIndexer: SequentialWriteDao, sequentialIndexer: SequentialWriteDao,
participantId: Ref.ParticipantId, participantId: Ref.ParticipantId,
storageBackend: StorageBackend[_], readStorageBackend: ReadStorageBackend,
parameterStorageBackend: ParameterStorageBackend,
deduplicationStorageBackend: DeduplicationStorageBackend,
resetStorageBackend: ResetStorageBackend,
errorFactories: ErrorFactories, errorFactories: ErrorFactories,
) extends LedgerDao { ) extends LedgerDao {
@ -73,7 +83,7 @@ private class JdbcLedgerDao(
override def lookupLedgerId()(implicit loggingContext: LoggingContext): Future[Option[LedgerId]] = override def lookupLedgerId()(implicit loggingContext: LoggingContext): Future[Option[LedgerId]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.getLedgerId)( .executeSql(metrics.daml.index.db.getLedgerId)(
storageBackend.ledgerIdentity(_).map(_.ledgerId) parameterStorageBackend.ledgerIdentity(_).map(_.ledgerId)
) )
override def lookupParticipantId()(implicit override def lookupParticipantId()(implicit
@ -81,7 +91,7 @@ private class JdbcLedgerDao(
): Future[Option[ParticipantId]] = ): Future[Option[ParticipantId]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.getParticipantId)( .executeSql(metrics.daml.index.db.getParticipantId)(
storageBackend.ledgerIdentity(_).map(_.participantId) parameterStorageBackend.ledgerIdentity(_).map(_.participantId)
) )
/** Defaults to Offset.begin if ledger_end is unset /** Defaults to Offset.begin if ledger_end is unset
@ -89,7 +99,7 @@ private class JdbcLedgerDao(
override def lookupLedgerEnd()(implicit loggingContext: LoggingContext): Future[Offset] = override def lookupLedgerEnd()(implicit loggingContext: LoggingContext): Future[Offset] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.getLedgerEnd)( .executeSql(metrics.daml.index.db.getLedgerEnd)(
storageBackend.ledgerEndOrBeforeBegin(_).lastOffset parameterStorageBackend.ledgerEndOrBeforeBegin(_).lastOffset
) )
case class InvalidLedgerEnd(msg: String) extends RuntimeException(msg) case class InvalidLedgerEnd(msg: String) extends RuntimeException(msg)
@ -99,7 +109,7 @@ private class JdbcLedgerDao(
): Future[(Offset, Long)] = ): Future[(Offset, Long)] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.getLedgerEndOffsetAndSequentialId) { connection => .executeSql(metrics.daml.index.db.getLedgerEndOffsetAndSequentialId) { connection =>
val end = storageBackend.ledgerEndOrBeforeBegin(connection) val end = parameterStorageBackend.ledgerEndOrBeforeBegin(connection)
end.lastOffset -> end.lastEventSeqId end.lastOffset -> end.lastEventSeqId
} }
@ -108,7 +118,7 @@ private class JdbcLedgerDao(
): Future[Option[Offset]] = ): Future[Option[Offset]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.getInitialLedgerEnd)( .executeSql(metrics.daml.index.db.getInitialLedgerEnd)(
storageBackend.ledgerEnd(_).map(_.lastOffset) parameterStorageBackend.ledgerEnd(_).map(_.lastOffset)
) )
override def initialize( override def initialize(
@ -117,7 +127,7 @@ private class JdbcLedgerDao(
)(implicit loggingContext: LoggingContext): Future[Unit] = )(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.initializeLedgerParameters)( .executeSql(metrics.daml.index.db.initializeLedgerParameters)(
storageBackend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = ledgerId, ledgerId = ledgerId,
participantId = participantId, participantId = participantId,
@ -129,7 +139,7 @@ private class JdbcLedgerDao(
loggingContext: LoggingContext loggingContext: LoggingContext
): Future[Option[(Offset, Configuration)]] = ): Future[Option[(Offset, Configuration)]] =
dbDispatcher.executeSql(metrics.daml.index.db.lookupConfiguration)( dbDispatcher.executeSql(metrics.daml.index.db.lookupConfiguration)(
storageBackend.ledgerConfiguration readStorageBackend.configurationStorageBackend.ledgerConfiguration
) )
override def getConfigurationEntries( override def getConfigurationEntries(
@ -139,7 +149,7 @@ private class JdbcLedgerDao(
PaginatingAsyncStream(PageSize) { queryOffset => PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext => withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadConfigurationEntries) { dbDispatcher.executeSql(metrics.daml.index.db.loadConfigurationEntries) {
storageBackend.configurationEntries( readStorageBackend.configurationStorageBackend.configurationEntries(
startExclusive = startExclusive, startExclusive = startExclusive,
endInclusive = endInclusive, endInclusive = endInclusive,
pageSize = PageSize, pageSize = PageSize,
@ -161,7 +171,8 @@ private class JdbcLedgerDao(
dbDispatcher.executeSql( dbDispatcher.executeSql(
metrics.daml.index.db.storeConfigurationEntryDbMetrics metrics.daml.index.db.storeConfigurationEntryDbMetrics
) { implicit conn => ) { implicit conn =>
val optCurrentConfig = storageBackend.ledgerConfiguration(conn) val optCurrentConfig =
readStorageBackend.configurationStorageBackend.ledgerConfiguration(conn)
val optExpectedGeneration: Option[Long] = val optExpectedGeneration: Option[Long] =
optCurrentConfig.map { case (_, c) => c.generation + 1 } optCurrentConfig.map { case (_, c) => c.generation + 1 }
val finalRejectionReason: Option[String] = val finalRejectionReason: Option[String] =
@ -263,7 +274,7 @@ private class JdbcLedgerDao(
PaginatingAsyncStream(PageSize) { queryOffset => PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext => withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadPartyEntries)( dbDispatcher.executeSql(metrics.daml.index.db.loadPartyEntries)(
storageBackend.partyEntries( readStorageBackend.partyStorageBackend.partyEntries(
startExclusive = startExclusive, startExclusive = startExclusive,
endInclusive = endInclusive, endInclusive = endInclusive,
pageSize = PageSize, pageSize = PageSize,
@ -391,25 +402,33 @@ private class JdbcLedgerDao(
Future.successful(List.empty) Future.successful(List.empty)
else else
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.loadParties)(storageBackend.parties(parties)) .executeSql(metrics.daml.index.db.loadParties)(
readStorageBackend.partyStorageBackend.parties(parties)
)
override def listKnownParties()(implicit override def listKnownParties()(implicit
loggingContext: LoggingContext loggingContext: LoggingContext
): Future[List[PartyDetails]] = ): Future[List[PartyDetails]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.loadAllParties)(storageBackend.knownParties) .executeSql(metrics.daml.index.db.loadAllParties)(
readStorageBackend.partyStorageBackend.knownParties
)
override def listLfPackages()(implicit override def listLfPackages()(implicit
loggingContext: LoggingContext loggingContext: LoggingContext
): Future[Map[Ref.PackageId, PackageDetails]] = ): Future[Map[Ref.PackageId, PackageDetails]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.loadPackages)(storageBackend.lfPackages) .executeSql(metrics.daml.index.db.loadPackages)(
readStorageBackend.packageStorageBackend.lfPackages
)
override def getLfArchive( override def getLfArchive(
packageId: Ref.PackageId packageId: Ref.PackageId
)(implicit loggingContext: LoggingContext): Future[Option[Archive]] = )(implicit loggingContext: LoggingContext): Future[Option[Archive]] =
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.loadArchive)(storageBackend.lfArchive(packageId)) .executeSql(metrics.daml.index.db.loadArchive)(
readStorageBackend.packageStorageBackend.lfArchive(packageId)
)
.map(_.map(data => ArchiveParser.assertFromByteArray(data)))( .map(_.map(data => ArchiveParser.assertFromByteArray(data)))(
servicesExecutionContext servicesExecutionContext
) )
@ -480,7 +499,7 @@ private class JdbcLedgerDao(
PaginatingAsyncStream(PageSize) { queryOffset => PaginatingAsyncStream(PageSize) { queryOffset =>
withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext => withEnrichedLoggingContext("queryOffset" -> queryOffset) { implicit loggingContext =>
dbDispatcher.executeSql(metrics.daml.index.db.loadPackageEntries)( dbDispatcher.executeSql(metrics.daml.index.db.loadPackageEntries)(
storageBackend.packageEntries( readStorageBackend.packageStorageBackend.packageEntries(
startExclusive = startExclusive, startExclusive = startExclusive,
endInclusive = endInclusive, endInclusive = endInclusive,
pageSize = PageSize, pageSize = PageSize,
@ -499,7 +518,7 @@ private class JdbcLedgerDao(
dbDispatcher.executeSql(metrics.daml.index.db.deduplicateCommandDbMetrics) { conn => dbDispatcher.executeSql(metrics.daml.index.db.deduplicateCommandDbMetrics) { conn =>
val key = DeduplicationKeyMaker.make(commandId, submitters) val key = DeduplicationKeyMaker.make(commandId, submitters)
// Insert a new deduplication entry, or update an expired entry // Insert a new deduplication entry, or update an expired entry
val updated = storageBackend.upsertDeduplicationEntry( val updated = deduplicationStorageBackend.upsertDeduplicationEntry(
key = key, key = key,
submittedAt = submittedAt, submittedAt = submittedAt,
deduplicateUntil = deduplicateUntil, deduplicateUntil = deduplicateUntil,
@ -510,7 +529,7 @@ private class JdbcLedgerDao(
CommandDeduplicationNew CommandDeduplicationNew
} else { } else {
// Deduplication row already exists // Deduplication row already exists
CommandDeduplicationDuplicate(storageBackend.deduplicatedUntil(key)(conn)) CommandDeduplicationDuplicate(deduplicationStorageBackend.deduplicatedUntil(key)(conn))
} }
} }
@ -518,7 +537,7 @@ private class JdbcLedgerDao(
currentTime: Timestamp currentTime: Timestamp
)(implicit loggingContext: LoggingContext): Future[Unit] = )(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.removeExpiredDeduplicationDataDbMetrics)( dbDispatcher.executeSql(metrics.daml.index.db.removeExpiredDeduplicationDataDbMetrics)(
storageBackend.removeExpiredDeduplicationData(currentTime) deduplicationStorageBackend.removeExpiredDeduplicationData(currentTime)
) )
override def stopDeduplicatingCommand( override def stopDeduplicatingCommand(
@ -527,7 +546,7 @@ private class JdbcLedgerDao(
)(implicit loggingContext: LoggingContext): Future[Unit] = { )(implicit loggingContext: LoggingContext): Future[Unit] = {
val key = DeduplicationKeyMaker.make(commandId, submitters) val key = DeduplicationKeyMaker.make(commandId, submitters)
dbDispatcher.executeSql(metrics.daml.index.db.stopDeduplicatingCommandDbMetrics)( dbDispatcher.executeSql(metrics.daml.index.db.stopDeduplicatingCommandDbMetrics)(
storageBackend.stopDeduplicatingCommand(key) deduplicationStorageBackend.stopDeduplicatingCommand(key)
) )
} }
@ -581,7 +600,7 @@ private class JdbcLedgerDao(
dbDispatcher dbDispatcher
.executeSql(metrics.daml.index.db.pruneDbMetrics) { conn => .executeSql(metrics.daml.index.db.pruneDbMetrics) { conn =>
if ( if (
!storageBackend.isPruningOffsetValidAgainstMigration( !readStorageBackend.eventStorageBackend.isPruningOffsetValidAgainstMigration(
pruneUpToInclusive, pruneUpToInclusive,
pruneAllDivulgedContracts, pruneAllDivulgedContracts,
conn, conn,
@ -592,16 +611,24 @@ private class JdbcLedgerDao(
)(new DamlContextualizedErrorLogger(logger, loggingContext, None)) )(new DamlContextualizedErrorLogger(logger, loggingContext, None))
} }
storageBackend.pruneEvents(pruneUpToInclusive, pruneAllDivulgedContracts)( readStorageBackend.eventStorageBackend.pruneEvents(
pruneUpToInclusive,
pruneAllDivulgedContracts,
)(
conn, conn,
loggingContext, loggingContext,
) )
storageBackend.pruneCompletions(pruneUpToInclusive)(conn, loggingContext) readStorageBackend.completionStorageBackend.pruneCompletions(pruneUpToInclusive)(
storageBackend.updatePrunedUptoInclusive(pruneUpToInclusive)(conn) conn,
loggingContext,
)
parameterStorageBackend.updatePrunedUptoInclusive(pruneUpToInclusive)(conn)
if (pruneAllDivulgedContracts) { if (pruneAllDivulgedContracts) {
storageBackend.updatePrunedAllDivulgedContractsUpToInclusive(pruneUpToInclusive)(conn) parameterStorageBackend.updatePrunedAllDivulgedContractsUpToInclusive(pruneUpToInclusive)(
conn
)
} }
} }
.andThen { .andThen {
@ -615,7 +642,7 @@ private class JdbcLedgerDao(
} }
override def reset()(implicit loggingContext: LoggingContext): Future[Unit] = override def reset()(implicit loggingContext: LoggingContext): Future[Unit] =
dbDispatcher.executeSql(metrics.daml.index.db.truncateAllTables)(storageBackend.reset) dbDispatcher.executeSql(metrics.daml.index.db.truncateAllTables)(resetStorageBackend.reset)
private val translation: LfValueTranslation = private val translation: LfValueTranslation =
new LfValueTranslation( new LfValueTranslation(
@ -625,13 +652,14 @@ private class JdbcLedgerDao(
loadPackage = (packageId, loggingContext) => this.getLfArchive(packageId)(loggingContext), loadPackage = (packageId, loggingContext) => this.getLfArchive(packageId)(loggingContext),
) )
private val queryNonPruned = QueryNonPrunedImpl(storageBackend, errorFactories) private val queryNonPruned = QueryNonPrunedImpl(parameterStorageBackend, errorFactories)
override val transactionsReader: TransactionsReader = override val transactionsReader: TransactionsReader =
new TransactionsReader( new TransactionsReader(
dispatcher = dbDispatcher, dispatcher = dbDispatcher,
queryNonPruned = queryNonPruned, queryNonPruned = queryNonPruned,
storageBackend = storageBackend, eventStorageBackend = readStorageBackend.eventStorageBackend,
contractStorageBackend = readStorageBackend.contractStorageBackend,
pageSize = eventsPageSize, pageSize = eventsPageSize,
eventProcessingParallelism = eventsProcessingParallelism, eventProcessingParallelism = eventsProcessingParallelism,
metrics = metrics, metrics = metrics,
@ -641,21 +669,25 @@ private class JdbcLedgerDao(
) )
override val contractsReader: ContractsReader = override val contractsReader: ContractsReader =
ContractsReader(dbDispatcher, metrics, storageBackend)( ContractsReader(dbDispatcher, metrics, readStorageBackend.contractStorageBackend)(
servicesExecutionContext servicesExecutionContext
) )
override val completions: CommandCompletionsReader = override val completions: CommandCompletionsReader =
new CommandCompletionsReader( new CommandCompletionsReader(
dbDispatcher, dbDispatcher,
storageBackend, readStorageBackend.completionStorageBackend,
queryNonPruned, queryNonPruned,
metrics, metrics,
) )
private val postCommitValidation = private val postCommitValidation =
if (performPostCommitValidation) if (performPostCommitValidation)
new PostCommitValidation.BackedBy(storageBackend, validatePartyAllocation) new PostCommitValidation.BackedBy(
readStorageBackend.partyStorageBackend,
readStorageBackend.contractStorageBackend,
validatePartyAllocation,
)
else else
PostCommitValidation.Skip PostCommitValidation.Skip
@ -740,9 +772,7 @@ private[platform] object JdbcLedgerDao {
enricher: Option[ValueEnricher], enricher: Option[ValueEnricher],
participantId: Ref.ParticipantId, participantId: Ref.ParticipantId,
errorFactories: ErrorFactories, errorFactories: ErrorFactories,
)(implicit loggingContext: LoggingContext): ResourceOwner[LedgerReadDao] = { )(implicit loggingContext: LoggingContext): ResourceOwner[LedgerReadDao] =
val dbType = DbType.jdbcType(jdbcUrl)
val storageBackend = StorageBackend.of(dbType)
owner( owner(
serverRole, serverRole,
jdbcUrl, jdbcUrl,
@ -757,10 +787,8 @@ private[platform] object JdbcLedgerDao {
enricher = enricher, enricher = enricher,
participantId = participantId, participantId = participantId,
sequentialWriteDao = NoopSequentialWriteDao, sequentialWriteDao = NoopSequentialWriteDao,
storageBackend = storageBackend,
errorFactories = errorFactories, errorFactories = errorFactories,
).map(new MeteredLedgerReadDao(_, metrics)) ).map(new MeteredLedgerReadDao(_, metrics))
}
def writeOwner( def writeOwner(
serverRole: ServerRole, serverRole: ServerRole,
@ -778,7 +806,6 @@ private[platform] object JdbcLedgerDao {
errorFactories: ErrorFactories, errorFactories: ErrorFactories,
)(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = { )(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = {
val dbType = DbType.jdbcType(jdbcUrl) val dbType = DbType.jdbcType(jdbcUrl)
val storageBackend = StorageBackend.of(dbType)
owner( owner(
serverRole, serverRole,
jdbcUrl, jdbcUrl,
@ -797,10 +824,9 @@ private[platform] object JdbcLedgerDao {
lfValueTranslationCache, lfValueTranslationCache,
metrics, metrics,
CompressionStrategy.none(metrics), CompressionStrategy.none(metrics),
storageBackend, DbType.jdbcType(jdbcUrl),
ledgerEndCache, ledgerEndCache,
), ),
storageBackend = storageBackend,
errorFactories = errorFactories, errorFactories = errorFactories,
).map(new MeteredLedgerDao(_, metrics)) ).map(new MeteredLedgerDao(_, metrics))
} }
@ -823,7 +849,6 @@ private[platform] object JdbcLedgerDao {
errorFactories: ErrorFactories, errorFactories: ErrorFactories,
)(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = { )(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = {
val dbType = DbType.jdbcType(jdbcUrl) val dbType = DbType.jdbcType(jdbcUrl)
val storageBackend = StorageBackend.of(dbType)
owner( owner(
serverRole, serverRole,
jdbcUrl, jdbcUrl,
@ -843,10 +868,9 @@ private[platform] object JdbcLedgerDao {
lfValueTranslationCache, lfValueTranslationCache,
metrics, metrics,
compressionStrategy, compressionStrategy,
storageBackend, dbType,
ledgerEndCache, ledgerEndCache,
), ),
storageBackend = storageBackend,
errorFactories = errorFactories, errorFactories = errorFactories,
).map(new MeteredLedgerDao(_, metrics)) ).map(new MeteredLedgerDao(_, metrics))
} }
@ -856,11 +880,13 @@ private[platform] object JdbcLedgerDao {
lfValueTranslationCache: LfValueTranslationCache.Cache, lfValueTranslationCache: LfValueTranslationCache.Cache,
metrics: Metrics, metrics: Metrics,
compressionStrategy: CompressionStrategy, compressionStrategy: CompressionStrategy,
storageBackend: StorageBackend[_], dbType: DbType,
ledgerEndCache: MutableLedgerEndCache, ledgerEndCache: MutableLedgerEndCache,
): SequentialWriteDao = ): SequentialWriteDao = {
val factory = StorageBackendFactory.of(dbType)
SequentialWriteDaoImpl( SequentialWriteDaoImpl(
storageBackend = storageBackend, ingestionStorageBackend = factory.createIngestionStorageBackend,
parameterStorageBackend = factory.createParameterStorageBackend,
updateToDbDtos = UpdateToDbDto( updateToDbDtos = UpdateToDbDto(
participantId = participantId, participantId = participantId,
translation = new LfValueTranslation( translation = new LfValueTranslation(
@ -873,6 +899,7 @@ private[platform] object JdbcLedgerDao {
), ),
ledgerEndCache = ledgerEndCache, ledgerEndCache = ledgerEndCache,
) )
}
private def owner( private def owner(
serverRole: ServerRole, serverRole: ServerRole,
@ -889,12 +916,13 @@ private[platform] object JdbcLedgerDao {
enricher: Option[ValueEnricher], enricher: Option[ValueEnricher],
participantId: Ref.ParticipantId, participantId: Ref.ParticipantId,
sequentialWriteDao: SequentialWriteDao, sequentialWriteDao: SequentialWriteDao,
storageBackend: StorageBackend[_],
errorFactories: ErrorFactories, errorFactories: ErrorFactories,
)(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = { )(implicit loggingContext: LoggingContext): ResourceOwner[LedgerDao] = {
val dbType = DbType.jdbcType(jdbcUrl)
val factory = StorageBackendFactory.of(dbType)
for { for {
dbDispatcher <- DbDispatcher.owner( dbDispatcher <- DbDispatcher.owner(
storageBackend.createDataSource(jdbcUrl), factory.createDataSourceStorageBackend.createDataSource(jdbcUrl),
serverRole, serverRole,
connectionPoolSize, connectionPoolSize,
connectionTimeout, connectionTimeout,
@ -912,7 +940,10 @@ private[platform] object JdbcLedgerDao {
enricher, enricher,
sequentialWriteDao, sequentialWriteDao,
participantId, participantId,
storageBackend, StorageBackendFactory.readStorageBackendFor(dbType),
factory.createParameterStorageBackend,
factory.createDeduplicationStorageBackend,
factory.createResetStorageBackend,
errorFactories, errorFactories,
) )
} }

View File

@ -23,7 +23,8 @@ object NoopSequentialWriteDao extends SequentialWriteDao {
} }
case class SequentialWriteDaoImpl[DB_BATCH]( case class SequentialWriteDaoImpl[DB_BATCH](
storageBackend: IngestionStorageBackend[DB_BATCH] with ParameterStorageBackend, ingestionStorageBackend: IngestionStorageBackend[DB_BATCH],
parameterStorageBackend: ParameterStorageBackend,
updateToDbDtos: Offset => state.Update => Iterator[DbDto], updateToDbDtos: Offset => state.Update => Iterator[DbDto],
ledgerEndCache: MutableLedgerEndCache, ledgerEndCache: MutableLedgerEndCache,
) extends SequentialWriteDao { ) extends SequentialWriteDao {
@ -33,7 +34,7 @@ case class SequentialWriteDaoImpl[DB_BATCH](
private def lazyInit(connection: Connection): Unit = private def lazyInit(connection: Connection): Unit =
if (!lastEventSeqIdInitialized) { if (!lastEventSeqIdInitialized) {
lastEventSeqId = storageBackend.ledgerEndOrBeforeBegin(connection).lastEventSeqId lastEventSeqId = parameterStorageBackend.ledgerEndOrBeforeBegin(connection).lastEventSeqId
lastEventSeqIdInitialized = true lastEventSeqIdInitialized = true
} }
@ -60,10 +61,10 @@ case class SequentialWriteDaoImpl[DB_BATCH](
.getOrElse(Vector.empty) .getOrElse(Vector.empty)
dbDtos dbDtos
.pipe(storageBackend.batch) .pipe(ingestionStorageBackend.batch)
.pipe(storageBackend.insertBatch(connection, _)) .pipe(ingestionStorageBackend.insertBatch(connection, _))
storageBackend.updateLedgerEnd( parameterStorageBackend.updateLedgerEnd(
ParameterStorageBackend.LedgerEnd( ParameterStorageBackend.LedgerEnd(
lastOffset = offset, lastOffset = offset,
lastEventSeqId = lastEventSeqId, lastEventSeqId = lastEventSeqId,

View File

@ -53,7 +53,8 @@ private[appendonlydao] object PostCommitValidation {
} }
final class BackedBy( final class BackedBy(
dao: PartyStorageBackend with ContractStorageBackend, partyStorageBackend: PartyStorageBackend,
contractStorageBackend: ContractStorageBackend,
validatePartyAllocation: Boolean, validatePartyAllocation: Boolean,
) extends PostCommitValidation { ) extends PostCommitValidation {
@ -90,7 +91,7 @@ private[appendonlydao] object PostCommitValidation {
if (referredContracts.isEmpty) { if (referredContracts.isEmpty) {
None None
} else { } else {
dao contractStorageBackend
.maximumLedgerTime(referredContracts)(connection) .maximumLedgerTime(referredContracts)(connection)
.map(validateCausalMonotonicity(_, transactionLedgerEffectiveTime)) .map(validateCausalMonotonicity(_, transactionLedgerEffectiveTime))
.getOrElse(Some(Rejection.UnknownContract)) .getOrElse(Some(Rejection.UnknownContract))
@ -116,7 +117,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction transaction: CommittedTransaction
)(implicit connection: Connection): Option[Rejection] = { )(implicit connection: Connection): Option[Rejection] = {
val informees = transaction.informees val informees = transaction.informees
val allocatedInformees = dao.parties(informees.toSeq)(connection).map(_.party) val allocatedInformees = partyStorageBackend.parties(informees.toSeq)(connection).map(_.party)
if (allocatedInformees.toSet == informees) if (allocatedInformees.toSet == informees)
None None
else else
@ -134,7 +135,7 @@ private[appendonlydao] object PostCommitValidation {
transaction: CommittedTransaction transaction: CommittedTransaction
)(implicit connection: Connection): Option[Rejection] = )(implicit connection: Connection): Option[Rejection] =
transaction transaction
.foldInExecutionOrder[Result](Right(State.empty(dao)))( .foldInExecutionOrder[Result](Right(State.empty(contractStorageBackend)))(
exerciseBegin = (acc, _, exe) => { exerciseBegin = (acc, _, exe) => {
val newAcc = acc.flatMap(validateKeyUsages(exe, _)) val newAcc = acc.flatMap(validateKeyUsages(exe, _))
(newAcc, true) (newAcc, true)
@ -206,13 +207,13 @@ private[appendonlydao] object PostCommitValidation {
* @param rollbackStack Stack of states at the beginning of rollback nodes so we can * @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 * restore the state at the end of the rollback. The most recent rollback
* comes first. * comes first.
* @param dao Dao about committed contracts for post-commit validation purposes. * @param contractStorageBackend For getting committed contracts for post-commit validation purposes.
* This is never changed during the traversal of the transaction. * This is never changed during the traversal of the transaction.
*/ */
private final case class State( private final case class State(
private val currentState: ActiveState, private val currentState: ActiveState,
private val rollbackStack: List[ActiveState], private val rollbackStack: List[ActiveState],
private val dao: PartyStorageBackend with ContractStorageBackend, private val contractStorageBackend: ContractStorageBackend,
) { ) {
def validateCreate( def validateCreate(
@ -264,14 +265,16 @@ private[appendonlydao] object PostCommitValidation {
private def lookup(key: Key)(implicit connection: Connection): Option[ContractId] = private def lookup(key: Key)(implicit connection: Connection): Option[ContractId] =
currentState.contracts.get(key.hash).orElse { currentState.contracts.get(key.hash).orElse {
if (currentState.removed(key.hash)) None if (currentState.removed(key.hash)) None
else dao.contractKeyGlobally(key)(connection) else contractStorageBackend.contractKeyGlobally(key)(connection)
} }
} }
private object State { private object State {
def empty(dao: PartyStorageBackend with ContractStorageBackend): State = def empty(
State(ActiveState(Map.empty, Set.empty), Nil, dao) contractStorageBackend: ContractStorageBackend
): State =
State(ActiveState(Map.empty, Set.empty), Nil, contractStorageBackend)
} }
sealed trait Rejection { sealed trait Rejection {

View File

@ -4,6 +4,7 @@
package com.daml.platform.store.appendonlydao.events package com.daml.platform.store.appendonlydao.events
import java.sql.Connection import java.sql.Connection
import akka.stream.OverflowStrategy import akka.stream.OverflowStrategy
import akka.stream.scaladsl.Source import akka.stream.scaladsl.Source
import akka.{Done, NotUsed} import akka.{Done, NotUsed}
@ -29,7 +30,7 @@ import com.daml.platform.store.appendonlydao.{
PaginatingAsyncStream, PaginatingAsyncStream,
} }
import com.daml.platform.store.backend.EventStorageBackend.{FilterParams, RangeParams} import com.daml.platform.store.backend.EventStorageBackend.{FilterParams, RangeParams}
import com.daml.platform.store.backend.StorageBackend import com.daml.platform.store.backend.{ContractStorageBackend, EventStorageBackend}
import com.daml.platform.store.interfaces.TransactionLogUpdate import com.daml.platform.store.interfaces.TransactionLogUpdate
import com.daml.platform.store.utils.Telemetry import com.daml.platform.store.utils.Telemetry
import com.daml.telemetry import com.daml.telemetry
@ -49,7 +50,8 @@ import scala.util.{Failure, Success}
private[appendonlydao] final class TransactionsReader( private[appendonlydao] final class TransactionsReader(
dispatcher: DbDispatcher, dispatcher: DbDispatcher,
queryNonPruned: QueryNonPruned, queryNonPruned: QueryNonPruned,
storageBackend: StorageBackend[_], eventStorageBackend: EventStorageBackend,
contractStorageBackend: ContractStorageBackend,
pageSize: Int, pageSize: Int,
eventProcessingParallelism: Int, eventProcessingParallelism: Int,
metrics: Metrics, metrics: Metrics,
@ -59,11 +61,11 @@ private[appendonlydao] final class TransactionsReader(
private val dbMetrics = metrics.daml.index.db private val dbMetrics = metrics.daml.index.db
private val eventSeqIdReader = private val eventSeqIdReader =
new EventsRange.EventSeqIdReader(storageBackend.maxEventSequentialIdOfAnObservableEvent) new EventsRange.EventSeqIdReader(eventStorageBackend.maxEventSequentialIdOfAnObservableEvent)
private val getTransactions = private val getTransactions =
new EventsTableFlatEventsRangeQueries.GetTransactions(storageBackend) new EventsTableFlatEventsRangeQueries.GetTransactions(eventStorageBackend)
private val getActiveContracts = private val getActiveContracts =
new EventsTableFlatEventsRangeQueries.GetActiveContracts(storageBackend) new EventsTableFlatEventsRangeQueries.GetActiveContracts(eventStorageBackend)
private val logger = ContextualizedLogger.get(this.getClass) private val logger = ContextualizedLogger.get(this.getClass)
@ -156,7 +158,7 @@ private[appendonlydao] final class TransactionsReader(
)(implicit loggingContext: LoggingContext): Future[Option[GetFlatTransactionResponse]] = )(implicit loggingContext: LoggingContext): Future[Option[GetFlatTransactionResponse]] =
dispatcher dispatcher
.executeSql(dbMetrics.lookupFlatTransactionById)( .executeSql(dbMetrics.lookupFlatTransactionById)(
storageBackend.flatTransaction( eventStorageBackend.flatTransaction(
transactionId, transactionId,
FilterParams( FilterParams(
wildCardParties = requestingParties, wildCardParties = requestingParties,
@ -193,7 +195,7 @@ private[appendonlydao] final class TransactionsReader(
queryNonPruned.executeSql( queryNonPruned.executeSql(
EventsRange.readPage( EventsRange.readPage(
read = (range, limit, fetchSizeHint) => read = (range, limit, fetchSizeHint) =>
storageBackend.transactionTreeEvents( eventStorageBackend.transactionTreeEvents(
rangeParams = RangeParams( rangeParams = RangeParams(
startExclusive = range.startExclusive, startExclusive = range.startExclusive,
endInclusive = range.endInclusive, endInclusive = range.endInclusive,
@ -256,7 +258,7 @@ private[appendonlydao] final class TransactionsReader(
)(implicit loggingContext: LoggingContext): Future[Option[GetTransactionResponse]] = )(implicit loggingContext: LoggingContext): Future[Option[GetTransactionResponse]] =
dispatcher dispatcher
.executeSql(dbMetrics.lookupTransactionTreeById)( .executeSql(dbMetrics.lookupTransactionTreeById)(
storageBackend.transactionTree( eventStorageBackend.transactionTree(
transactionId, transactionId,
FilterParams( FilterParams(
wildCardParties = requestingParties, wildCardParties = requestingParties,
@ -306,7 +308,7 @@ private[appendonlydao] final class TransactionsReader(
.mapAsync(eventProcessingParallelism) { range => .mapAsync(eventProcessingParallelism) { range =>
dispatcher.executeSql(dbMetrics.getTransactionLogUpdates) { implicit conn => dispatcher.executeSql(dbMetrics.getTransactionLogUpdates) { implicit conn =>
queryNonPruned.executeSql( queryNonPruned.executeSql(
query = storageBackend.rawEvents( query = eventStorageBackend.rawEvents(
startExclusive = range.startExclusive, startExclusive = range.startExclusive,
endInclusive = range.endInclusive, endInclusive = range.endInclusive,
)(conn), )(conn),
@ -436,7 +438,7 @@ private[appendonlydao] final class TransactionsReader(
.mapAsync(eventProcessingParallelism) { range => .mapAsync(eventProcessingParallelism) { range =>
dispatcher.executeSql(dbMetrics.getContractStateEvents) { implicit conn => dispatcher.executeSql(dbMetrics.getContractStateEvents) { implicit conn =>
queryNonPruned.executeSql( queryNonPruned.executeSql(
storageBackend contractStorageBackend
.contractStateEvents(range.startExclusive, range.endInclusive)(conn), .contractStateEvents(range.startExclusive, range.endInclusive)(conn),
startExclusive._1, startExclusive._1,
pruned => pruned =>

View File

@ -14,13 +14,11 @@ import com.daml.lf.data.Time.Timestamp
import com.daml.lf.ledger.EventId import com.daml.lf.ledger.EventId
import com.daml.logging.LoggingContext import com.daml.logging.LoggingContext
import com.daml.platform import com.daml.platform
import com.daml.platform.store.{DbType, EventSequentialId} import com.daml.platform.store.EventSequentialId
import com.daml.platform.store.appendonlydao.events.{ContractId, EventsTable, Key, Raw} import com.daml.platform.store.appendonlydao.events.{ContractId, EventsTable, Key, Raw}
import com.daml.platform.store.backend.EventStorageBackend.{FilterParams, RangeParams} import com.daml.platform.store.backend.EventStorageBackend.{FilterParams, RangeParams}
import com.daml.platform.store.backend.StorageBackend.RawTransactionEvent import com.daml.platform.store.backend.StorageBackend.RawTransactionEvent
import com.daml.platform.store.backend.h2.H2StorageBackend import com.daml.platform.store.backend.postgresql.PostgresDataSourceConfig
import com.daml.platform.store.backend.oracle.OracleStorageBackend
import com.daml.platform.store.backend.postgresql.{PostgresDataSourceConfig, PostgresStorageBackend}
import com.daml.platform.store.entries.{ConfigurationEntry, PackageLedgerEntry, PartyLedgerEntry} import com.daml.platform.store.entries.{ConfigurationEntry, PackageLedgerEntry, PartyLedgerEntry}
import com.daml.platform.store.interfaces.LedgerDaoContractsReader.KeyState import com.daml.platform.store.interfaces.LedgerDaoContractsReader.KeyState
import com.daml.scalautil.NeverEqualsOverride import com.daml.scalautil.NeverEqualsOverride
@ -48,7 +46,10 @@ trait StorageBackend[DB_BATCH]
with EventStorageBackend with EventStorageBackend
with DataSourceStorageBackend with DataSourceStorageBackend
with DBLockStorageBackend with DBLockStorageBackend
with IntegrityStorageBackend { with IntegrityStorageBackend
with ResetStorageBackend
trait ResetStorageBackend {
/** Truncates all storage backend tables, EXCEPT the packages table. /** Truncates all storage backend tables, EXCEPT the packages table.
* Does not touch other tables, like the Flyway history table. * Does not touch other tables, like the Flyway history table.
@ -411,11 +412,4 @@ object StorageBackend {
eventSequentialId: Long, eventSequentialId: Long,
offset: Offset, offset: Offset,
) extends NeverEqualsOverride ) extends NeverEqualsOverride
def of(dbType: DbType): StorageBackend[_] =
dbType match {
case DbType.H2Database => H2StorageBackend
case DbType.Postgres => PostgresStorageBackend
case DbType.Oracle => OracleStorageBackend
}
} }

View File

@ -0,0 +1,55 @@
// 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
import com.daml.platform.store.DbType
import com.daml.platform.store.backend.h2.H2StorageBackendFactory
import com.daml.platform.store.backend.oracle.OracleStorageBackendFactory
import com.daml.platform.store.backend.postgresql.PostgresStorageBackendFactory
trait StorageBackendFactory {
def createIngestionStorageBackend: IngestionStorageBackend[_]
def createParameterStorageBackend: ParameterStorageBackend
def createConfigurationStorageBackend: ConfigurationStorageBackend
def createPartyStorageBackend: PartyStorageBackend
def createPackageStorageBackend: PackageStorageBackend
def createDeduplicationStorageBackend: DeduplicationStorageBackend
def createCompletionStorageBackend: CompletionStorageBackend
def createContractStorageBackend: ContractStorageBackend
def createEventStorageBackend: EventStorageBackend
def createDataSourceStorageBackend: DataSourceStorageBackend
def createDBLockStorageBackend: DBLockStorageBackend
def createIntegrityStorageBackend: IntegrityStorageBackend
def createResetStorageBackend: ResetStorageBackend
}
object StorageBackendFactory {
def of(dbType: DbType): StorageBackendFactory =
dbType match {
case DbType.H2Database => H2StorageBackendFactory
case DbType.Postgres => PostgresStorageBackendFactory
case DbType.Oracle => OracleStorageBackendFactory
}
def readStorageBackendFor(dbType: DbType): ReadStorageBackend = {
val factory = of(dbType)
ReadStorageBackend(
configurationStorageBackend = factory.createConfigurationStorageBackend,
partyStorageBackend = factory.createPartyStorageBackend,
packageStorageBackend = factory.createPackageStorageBackend,
completionStorageBackend = factory.createCompletionStorageBackend,
contractStorageBackend = factory.createContractStorageBackend,
eventStorageBackend = factory.createEventStorageBackend,
)
}
}
case class ReadStorageBackend(
configurationStorageBackend: ConfigurationStorageBackend,
partyStorageBackend: PartyStorageBackend,
packageStorageBackend: PackageStorageBackend,
completionStorageBackend: CompletionStorageBackend,
contractStorageBackend: ContractStorageBackend,
eventStorageBackend: EventStorageBackend,
)

View File

@ -23,19 +23,20 @@ object VerifiedDataSource {
executionContext: ExecutionContext, executionContext: ExecutionContext,
loggingContext: LoggingContext, loggingContext: LoggingContext,
): Future[DataSource] = { ): Future[DataSource] = {
val storageBackend = StorageBackend.of(DbType.jdbcType(jdbcUrl)) val dataSourceStorageBackend =
StorageBackendFactory.of(DbType.jdbcType(jdbcUrl)).createDataSourceStorageBackend
for { for {
dataSource <- RetryStrategy.constant( dataSource <- RetryStrategy.constant(
attempts = MaxInitialConnectRetryAttempts, attempts = MaxInitialConnectRetryAttempts,
waitTime = 1.second, waitTime = 1.second,
) { (i, _) => ) { (i, _) =>
Future { Future {
val createdDatasource = storageBackend.createDataSource(jdbcUrl) val createdDatasource = dataSourceStorageBackend.createDataSource(jdbcUrl)
logger.info( logger.info(
s"Attempting to connect to the database (attempt $i/$MaxInitialConnectRetryAttempts)" s"Attempting to connect to the database (attempt $i/$MaxInitialConnectRetryAttempts)"
) )
Using.resource(createdDatasource.getConnection)( Using.resource(createdDatasource.getConnection)(
storageBackend.checkDatabaseAvailable dataSourceStorageBackend.checkDatabaseAvailable
) )
createdDatasource createdDatasource
}.andThen { case Failure(exception) => }.andThen { case Failure(exception) =>
@ -44,7 +45,7 @@ object VerifiedDataSource {
} }
_ <- Future { _ <- Future {
Using.resource(dataSource.getConnection)( Using.resource(dataSource.getConnection)(
storageBackend.checkCompatibility dataSourceStorageBackend.checkCompatibility
) )
} }
} yield dataSource } yield dataSource

View File

@ -20,12 +20,11 @@ import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpola
import com.google.protobuf.any import com.google.protobuf.any
import com.google.rpc.status.{Status => StatusProto} import com.google.rpc.status.{Status => StatusProto}
trait CompletionStorageBackendTemplate extends CompletionStorageBackend { class CompletionStorageBackendTemplate(queryStrategy: QueryStrategy)
extends CompletionStorageBackend {
private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass) private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass)
def queryStrategy: QueryStrategy
override def commandCompletions( override def commandCompletions(
startExclusive: Offset, startExclusive: Offset,
endInclusive: Offset, endInclusive: Offset,
@ -154,7 +153,7 @@ trait CompletionStorageBackendTemplate extends CompletionStorageBackend {
.map(_.details) .map(_.details)
.getOrElse(Seq.empty) .getOrElse(Seq.empty)
def pruneCompletions( override def pruneCompletions(
pruneUpToInclusive: Offset pruneUpToInclusive: Offset
)(connection: Connection, loggingContext: LoggingContext): Unit = { )(connection: Connection, loggingContext: LoggingContext): Unit = {
pruneWithLogging(queryDescription = "Command completions pruning") { pruneWithLogging(queryDescription = "Command completions pruning") {

View File

@ -15,7 +15,7 @@ import com.daml.platform.store.appendonlydao.JdbcLedgerDao.{acceptType, rejectTy
import com.daml.platform.store.backend.ConfigurationStorageBackend import com.daml.platform.store.backend.ConfigurationStorageBackend
import com.daml.platform.store.entries.ConfigurationEntry import com.daml.platform.store.entries.ConfigurationEntry
private[backend] trait ConfigurationStorageBackendTemplate extends ConfigurationStorageBackend { private[backend] object ConfigurationStorageBackendTemplate extends ConfigurationStorageBackend {
private val SQL_GET_CONFIGURATION_ENTRIES = SQL( private val SQL_GET_CONFIGURATION_ENTRIES = SQL(
"""select """select

View File

@ -27,9 +27,7 @@ import com.daml.platform.store.interfaces.LedgerDaoContractsReader.{
import scala.util.{Failure, Success, Try} import scala.util.{Failure, Success, Try}
trait ContractStorageBackendTemplate extends ContractStorageBackend { class ContractStorageBackendTemplate(queryStrategy: QueryStrategy) extends ContractStorageBackend {
def queryStrategy: QueryStrategy
override def contractKeyGlobally(key: Key)(connection: Connection): Option[ContractId] = override def contractKeyGlobally(key: Key)(connection: Connection): Option[ContractId] =
contractKey( contractKey(

View File

@ -6,21 +6,15 @@ package com.daml.platform.store.backend.common
import java.sql.Connection import java.sql.Connection
import anorm.SqlParser.get import anorm.SqlParser.get
import com.daml.platform.store.backend.DataSourceStorageBackend
import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation
private[backend] trait DataSourceStorageBackendTemplate extends DataSourceStorageBackend { private[backend] object DataSourceStorageBackendTemplate {
protected def exe(statement: String): Connection => Unit = { connection => def exe(statement: String): Connection => Unit = { implicit connection =>
val stmnt = connection.createStatement() SQL"#$statement".execute()
try { ()
stmnt.execute(statement)
()
} finally {
stmnt.close()
}
} }
override def checkDatabaseAvailable(connection: Connection): Unit = def checkDatabaseAvailable(connection: Connection): Unit =
assert(SQL"SELECT 1".as(get[Int](1).single)(connection) == 1) assert(SQL"SELECT 1".as(get[Int](1).single)(connection) == 1)
} }

View File

@ -26,19 +26,17 @@ import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, Sql
import scala.collection.compat.immutable.ArraySeq import scala.collection.compat.immutable.ArraySeq
trait EventStorageBackendTemplate extends EventStorageBackend { abstract class EventStorageBackendTemplate(
eventStrategy: EventStrategy,
queryStrategy: QueryStrategy,
// TODO Refactoring: This method is needed in pruneEvents, but belongs to [[ParameterStorageBackend]].
// Remove with the break-out of pruneEvents.
participantAllDivulgedContractsPrunedUpToInclusive: Connection => Option[Offset],
) extends EventStorageBackend {
import com.daml.platform.store.Conversions.ArrayColumnToStringArray.arrayColumnToStringArray import com.daml.platform.store.Conversions.ArrayColumnToStringArray.arrayColumnToStringArray
private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass) private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass)
def eventStrategy: EventStrategy
def queryStrategy: QueryStrategy
// TODO Refactoring: This method is needed in pruneEvents, but belongs to [[ParameterStorageBackend]].
// Remove with the break-out of pruneEvents.
def participantAllDivulgedContractsPrunedUpToInclusive(
connection: Connection
): Option[Offset]
private val selectColumnsForFlatTransactions = private val selectColumnsForFlatTransactions =
Seq( Seq(
"event_offset", "event_offset",

View File

@ -6,10 +6,10 @@ package com.daml.platform.store.backend.common
import java.sql.Connection import java.sql.Connection
import anorm.{SQL, SqlQuery} import anorm.{SQL, SqlQuery}
import com.daml.platform.store.backend.{IngestionStorageBackend, ParameterStorageBackend} import com.daml.platform.store.backend.{DbDto, IngestionStorageBackend, ParameterStorageBackend}
private[backend] trait IngestionStorageBackendTemplate[DB_BATCH] private[backend] class IngestionStorageBackendTemplate(schema: Schema[DbDto])
extends IngestionStorageBackend[DB_BATCH] { extends IngestionStorageBackend[AppendOnlySchema.Batch] {
private val SQL_DELETE_OVERSPILL_ENTRIES: List[SqlQuery] = private val SQL_DELETE_OVERSPILL_ENTRIES: List[SqlQuery] =
List( List(
@ -39,4 +39,13 @@ private[backend] trait IngestionStorageBackendTemplate[DB_BATCH]
} }
} }
} }
override def insertBatch(
connection: Connection,
dbBatch: AppendOnlySchema.Batch,
): Unit =
schema.executeUpdate(dbBatch, connection)
override def batch(dbDtos: Vector[DbDto]): AppendOnlySchema.Batch =
schema.prepareData(dbDtos)
} }

View File

@ -10,7 +10,7 @@ import anorm.SqlParser.{long, str}
import anorm.~ import anorm.~
import com.daml.platform.store.backend.IntegrityStorageBackend import com.daml.platform.store.backend.IntegrityStorageBackend
private[backend] trait IntegrityStorageBackendTemplate extends IntegrityStorageBackend { private[backend] object IntegrityStorageBackendTemplate extends IntegrityStorageBackend {
private val allSequentialIds: String = private val allSequentialIds: String =
s""" s"""

View File

@ -17,7 +17,7 @@ import com.daml.platform.store.appendonlydao.JdbcLedgerDao.{acceptType, rejectTy
import com.daml.platform.store.backend.PackageStorageBackend import com.daml.platform.store.backend.PackageStorageBackend
import com.daml.platform.store.entries.PackageLedgerEntry import com.daml.platform.store.entries.PackageLedgerEntry
private[backend] trait PackageStorageBackendTemplate extends PackageStorageBackend { private[backend] object PackageStorageBackendTemplate extends PackageStorageBackend {
private val SQL_SELECT_PACKAGES = private val SQL_SELECT_PACKAGES =
SQL( SQL(

View File

@ -18,7 +18,7 @@ import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpola
import com.daml.scalautil.Statement.discard import com.daml.scalautil.Statement.discard
import scalaz.syntax.tag._ import scalaz.syntax.tag._
private[backend] trait ParameterStorageBackendTemplate extends ParameterStorageBackend { private[backend] object ParameterStorageBackendTemplate extends ParameterStorageBackend {
private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass) private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass)

View File

@ -17,9 +17,7 @@ import com.daml.platform.store.backend.PartyStorageBackend
import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation
import com.daml.platform.store.entries.PartyLedgerEntry import com.daml.platform.store.entries.PartyLedgerEntry
trait PartyStorageBackendTemplate extends PartyStorageBackend { class PartyStorageBackendTemplate(queryStrategy: QueryStrategy) extends PartyStorageBackend {
def queryStrategy: QueryStrategy
private val SQL_GET_PARTY_ENTRIES = SQL( private val SQL_GET_PARTY_ENTRIES = SQL(
"""select * from party_entries """select * from party_entries

View File

@ -0,0 +1,110 @@
// 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.h2
import anorm.{Row, SimpleSql}
import com.daml.platform.store.appendonlydao.events.ContractId
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.ContractStorageBackendTemplate
object H2ContractStorageBackend extends ContractStorageBackendTemplate(H2QueryStrategy) {
override def maximumLedgerTimeSqlLiteral(id: ContractId): SimpleSql[Row] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
WITH archival_event AS (
SELECT 1
FROM participant_events_consuming_exercise, parameters
WHERE contract_id = $id
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY
),
create_event AS (
SELECT ledger_effective_time
FROM participant_events_create, parameters
WHERE contract_id = $id
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT NULL::BIGINT
FROM participant_events_divulgence, parameters
WHERE contract_id = $id
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.
FETCH NEXT 1 ROW ONLY
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT ledger_effective_time
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
FETCH NEXT 1 ROW ONLY"""
}
override def activeContractSqlLiteral(
contractId: ContractId,
treeEventWitnessesClause: CompositeSql,
resultColumns: List[String],
coalescedColumns: String,
): SimpleSql[Row] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL""" WITH archival_event AS (
SELECT 1
FROM participant_events_consuming_exercise, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause -- only use visible archivals
FETCH NEXT 1 ROW ONLY
),
create_event AS (
SELECT contract_id, #${resultColumns.mkString(", ")}
FROM participant_events_create, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause
FETCH NEXT 1 ROW ONLY -- 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, #${resultColumns.mkString(", ")}
FROM participant_events_create, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgence_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.
#$coalescedColumns
FROM participant_events_divulgence divulgence_events LEFT OUTER JOIN create_event_unrestricted ON (divulgence_events.contract_id = create_event_unrestricted.contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause
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.
FETCH NEXT 1 ROW ONLY
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, #${resultColumns.mkString(", ")}
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
FETCH NEXT 1 ROW ONLY"""
}
}

View File

@ -0,0 +1,24 @@
// 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.h2
import java.sql.Connection
import com.daml.platform.store.backend.DBLockStorageBackend
object H2DBLockStorageBackend extends DBLockStorageBackend {
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def lock(id: Int): DBLockStorageBackend.LockId =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def dbLockSupported: Boolean = false
}

View File

@ -0,0 +1,56 @@
// 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.h2
import java.sql.Connection
import com.daml.logging.LoggingContext
import com.daml.platform.store.backend.DataSourceStorageBackend
import com.daml.platform.store.backend.common.{
DataSourceStorageBackendTemplate,
InitHookDataSourceProxy,
}
import javax.sql.DataSource
object H2DataSourceStorageBackend extends DataSourceStorageBackend {
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
val h2DataSource = new org.h2.jdbcx.JdbcDataSource()
// H2 (org.h2.jdbcx.JdbcDataSource) does not support setting the user/password within the jdbcUrl, so remove
// those properties from the url if present and set them separately. Note that Postgres and Oracle support
// user/password in the URLs, so we don't bother exposing user/password configs separately from the url just for h2
// which is anyway not supported for production. (This also helps run canton h2 participants that set user and
// password.)
val (urlNoUserNoPassword, user, password) = extractUserPasswordAndRemoveFromUrl(jdbcUrl)
user.foreach(h2DataSource.setUser)
password.foreach(h2DataSource.setPassword)
h2DataSource.setUrl(urlNoUserNoPassword)
InitHookDataSourceProxy(h2DataSource, connectionInitHook.toList)
}
def extractUserPasswordAndRemoveFromUrl(
jdbcUrl: String
): (String, Option[String], Option[String]) = {
def setKeyValueAndRemoveFromUrl(url: String, key: String): (String, Option[String]) = {
val regex = s".*(;(?i)${key}=([^;]*)).*".r
url match {
case regex(keyAndValue, value) =>
(url.replace(keyAndValue, ""), Some(value))
case _ => (url, None)
}
}
val (urlNoUser, user) = setKeyValueAndRemoveFromUrl(jdbcUrl, "user")
val (urlNoUserNoPassword, password) = setKeyValueAndRemoveFromUrl(urlNoUser, "password")
(urlNoUserNoPassword, user, password)
}
override def checkDatabaseAvailable(connection: Connection): Unit =
DataSourceStorageBackendTemplate.checkDatabaseAvailable(connection)
}

View File

@ -0,0 +1,55 @@
// 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.h2
import java.sql.Connection
import anorm.SQL
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.backend.common.DeduplicationStorageBackendTemplate
import scala.util.control.NonFatal
object H2DeduplicationStorageBackend extends DeduplicationStorageBackendTemplate {
private val logger = ContextualizedLogger.get(this.getClass)
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 def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int = {
// Under the default READ_COMMITTED isolation level used for the indexdb, when a deduplication
// upsert is performed simultaneously from multiple threads, the query fails with
// JdbcSQLIntegrityConstraintViolationException: Unique index or primary key violation
// Simple retry helps
def retry[T](op: => T): T =
try {
op
} catch {
case NonFatal(e) =>
logger.debug(s"Caught exception while upserting a deduplication entry: $e")
op
}
retry(
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
)
}
}

View File

@ -0,0 +1,47 @@
// 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.h2
import java.sql.Connection
import anorm.SqlParser.get
import anorm.SqlStringInterpolation
import com.daml.ledger.offset.Offset
import com.daml.platform.store.backend.common.{
EventStorageBackendTemplate,
ParameterStorageBackendTemplate,
}
object H2EventStorageBackend
extends EventStorageBackendTemplate(
queryStrategy = H2QueryStrategy,
eventStrategy = H2EventStrategy,
participantAllDivulgedContractsPrunedUpToInclusive =
ParameterStorageBackendTemplate.participantAllDivulgedContractsPrunedUpToInclusive,
) {
override def maxEventSequentialIdOfAnObservableEvent(
offset: Offset
)(connection: Connection): Option[Long] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""
SELECT max_esi FROM (
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_consuming_exercise WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
UNION ALL
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_non_consuming_exercise WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
UNION ALL
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_create WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
) AS t
ORDER BY max_esi DESC
FETCH NEXT 1 ROW ONLY;
""".as(get[Long](1).singleOpt)(connection)
}
// Migration from mutable schema is not supported for H2
override def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean = true
}

View File

@ -0,0 +1,52 @@
// 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.h2
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.EventStrategy
object H2EventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Ref.Party],
): CompositeSql = {
val partiesArray = parties.view.map(_.toString).toArray
cSQL"array_intersection(#$witnessesColumnName, $partiesArray)"
}
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
H2QueryStrategy.arrayIntersectionNonEmptyClause(
columnName = submittersColumnName,
parties = parties,
)
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
cSQL"(${H2QueryStrategy.arrayIntersectionNonEmptyClause(witnessesColumnName, wildCardParties)})" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val clause =
H2QueryStrategy.arrayIntersectionNonEmptyClause(
witnessesColumnName,
parties,
)
val templateIdsArray = templateIds.view.map(_.toString).toArray
cSQL"( ($clause) AND (template_id = ANY($templateIdsArray)) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}

View File

@ -0,0 +1,27 @@
// 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.h2
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.QueryStrategy
object H2QueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql =
if (parties.isEmpty)
cSQL"false"
else
parties.view
.map(p => cSQL"array_contains(#$columnName, '#${p.toString}')")
.mkComposite("(", " or ", ")")
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"array_contains($arrayColumnName, $elementColumnName)"
override def isTrue(booleanColumnName: String): String = booleanColumnName
}

View File

@ -0,0 +1,47 @@
// 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.h2
import java.sql.Connection
import anorm.SQL
import com.daml.platform.store.backend.ResetStorageBackend
object H2ResetStorageBackend extends ResetStorageBackend {
override def reset(connection: Connection): Unit = {
SQL("""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_divulgence;
|truncate table participant_events_create;
|truncate table participant_events_consuming_exercise;
|truncate table participant_events_non_consuming_exercise;
|truncate table party_entries;
|set referential_integrity true;""".stripMargin)
.execute()(connection)
()
}
override def resetAll(connection: Connection): Unit = {
SQL("""set referential_integrity false;
|truncate table configuration_entries;
|truncate table packages;
|truncate table package_entries;
|truncate table parameters;
|truncate table participant_command_completions;
|truncate table participant_command_submissions;
|truncate table participant_events_divulgence;
|truncate table participant_events_create;
|truncate table participant_events_consuming_exercise;
|truncate table participant_events_non_consuming_exercise;
|truncate table party_entries;
|set referential_integrity true;""".stripMargin)
.execute()(connection)
()
}
}

View File

@ -1,378 +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.backend.h2
import java.sql.Connection
import anorm.{Row, SQL, SimpleSql}
import anorm.SqlParser.get
import com.daml.ledger.offset.Offset
import com.daml.lf.data.Ref
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.appendonlydao.events.ContractId
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.{
AppendOnlySchema,
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
ContractStorageBackendTemplate,
DataSourceStorageBackendTemplate,
DeduplicationStorageBackendTemplate,
EventStorageBackendTemplate,
EventStrategy,
IngestionStorageBackendTemplate,
InitHookDataSourceProxy,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
QueryStrategy,
}
import com.daml.platform.store.backend.{
DBLockStorageBackend,
DataSourceStorageBackend,
DbDto,
StorageBackend,
common,
}
import javax.sql.DataSource
import scala.util.control.NonFatal
private[backend] object H2StorageBackend
extends StorageBackend[AppendOnlySchema.Batch]
with DataSourceStorageBackendTemplate
with IngestionStorageBackendTemplate[AppendOnlySchema.Batch]
with ParameterStorageBackendTemplate
with ConfigurationStorageBackendTemplate
with PackageStorageBackendTemplate
with DeduplicationStorageBackendTemplate
with EventStorageBackendTemplate
with ContractStorageBackendTemplate
with CompletionStorageBackendTemplate
with PartyStorageBackendTemplate
with IntegrityStorageBackendTemplate {
private val logger = ContextualizedLogger.get(this.getClass)
override def reset(connection: Connection): Unit = {
SQL("""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_divulgence;
|truncate table participant_events_create;
|truncate table participant_events_consuming_exercise;
|truncate table participant_events_non_consuming_exercise;
|truncate table party_entries;
|set referential_integrity true;""".stripMargin)
.execute()(connection)
()
}
override def resetAll(connection: Connection): Unit = {
SQL("""set referential_integrity false;
|truncate table configuration_entries;
|truncate table packages;
|truncate table package_entries;
|truncate table parameters;
|truncate table participant_command_completions;
|truncate table participant_command_submissions;
|truncate table participant_events_divulgence;
|truncate table participant_events_create;
|truncate table participant_events_consuming_exercise;
|truncate table participant_events_non_consuming_exercise;
|truncate table party_entries;
|set referential_integrity true;""".stripMargin)
.execute()(connection)
()
}
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 def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int = {
// Under the default READ_COMMITTED isolation level used for the indexdb, when a deduplication
// upsert is performed simultaneously from multiple threads, the query fails with
// JdbcSQLIntegrityConstraintViolationException: Unique index or primary key violation
// Simple retry helps
def retry[T](op: => T): T =
try {
op
} catch {
case NonFatal(e) =>
logger.debug(s"Caught exception while upserting a deduplication entry: $e")
op
}
retry(
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
)
}
override def batch(dbDtos: Vector[DbDto]): AppendOnlySchema.Batch =
H2Schema.schema.prepareData(dbDtos)
override def insertBatch(connection: Connection, batch: AppendOnlySchema.Batch): Unit =
H2Schema.schema.executeUpdate(batch, connection)
def maxEventSequentialIdOfAnObservableEvent(
offset: Offset
)(connection: Connection): Option[Long] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""
SELECT max_esi FROM (
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_consuming_exercise WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
UNION ALL
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_non_consuming_exercise WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
UNION ALL
(SELECT max(event_sequential_id) AS max_esi FROM participant_events_create WHERE event_offset <= $offset GROUP BY event_offset ORDER BY event_offset DESC FETCH NEXT 1 ROW ONLY)
) AS t
ORDER BY max_esi DESC
FETCH NEXT 1 ROW ONLY;
""".as(get[Long](1).singleOpt)(connection)
}
object H2QueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql =
if (parties.isEmpty)
cSQL"false"
else
parties.view
.map(p => cSQL"array_contains(#$columnName, '#${p.toString}')")
.mkComposite("(", " or ", ")")
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"array_contains($arrayColumnName, $elementColumnName)"
override def isTrue(booleanColumnName: String): String = booleanColumnName
}
override def queryStrategy: QueryStrategy = H2QueryStrategy
object H2EventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Ref.Party],
): CompositeSql = {
val partiesArray = parties.view.map(_.toString).toArray
cSQL"array_intersection(#$witnessesColumnName, $partiesArray)"
}
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
H2QueryStrategy.arrayIntersectionNonEmptyClause(
columnName = submittersColumnName,
parties = parties,
)
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
cSQL"(${H2QueryStrategy.arrayIntersectionNonEmptyClause(witnessesColumnName, wildCardParties)})" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val clause =
H2QueryStrategy.arrayIntersectionNonEmptyClause(
witnessesColumnName,
parties,
)
val templateIdsArray = templateIds.view.map(_.toString).toArray
cSQL"( ($clause) AND (template_id = ANY($templateIdsArray)) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}
override def eventStrategy: common.EventStrategy = H2EventStrategy
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
val h2DataSource = new org.h2.jdbcx.JdbcDataSource()
// H2 (org.h2.jdbcx.JdbcDataSource) does not support setting the user/password within the jdbcUrl, so remove
// those properties from the url if present and set them separately. Note that Postgres and Oracle support
// user/password in the URLs, so we don't bother exposing user/password configs separately from the url just for h2
// which is anyway not supported for production. (This also helps run canton h2 participants that set user and
// password.)
val (urlNoUserNoPassword, user, password) = extractUserPasswordAndRemoveFromUrl(jdbcUrl)
user.foreach(h2DataSource.setUser)
password.foreach(h2DataSource.setPassword)
h2DataSource.setUrl(urlNoUserNoPassword)
InitHookDataSourceProxy(h2DataSource, connectionInitHook.toList)
}
def extractUserPasswordAndRemoveFromUrl(
jdbcUrl: String
): (String, Option[String], Option[String]) = {
def setKeyValueAndRemoveFromUrl(url: String, key: String): (String, Option[String]) = {
val regex = s".*(;(?i)${key}=([^;]*)).*".r
url match {
case regex(keyAndValue, value) =>
(url.replace(keyAndValue, ""), Some(value))
case _ => (url, None)
}
}
val (urlNoUser, user) = setKeyValueAndRemoveFromUrl(jdbcUrl, "user")
val (urlNoUserNoPassword, password) = setKeyValueAndRemoveFromUrl(urlNoUser, "password")
(urlNoUserNoPassword, user, password)
}
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def lock(id: Int): DBLockStorageBackend.LockId =
throw new UnsupportedOperationException("db level locks are not supported for H2")
override def dbLockSupported: Boolean = false
// Migration from mutable schema is not supported for H2
override def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean = true
override def maximumLedgerTimeSqlLiteral(id: ContractId): SimpleSql[Row] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL"""
WITH archival_event AS (
SELECT 1
FROM participant_events_consuming_exercise, parameters
WHERE contract_id = $id
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY
),
create_event AS (
SELECT ledger_effective_time
FROM participant_events_create, parameters
WHERE contract_id = $id
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT NULL::BIGINT
FROM participant_events_divulgence, parameters
WHERE contract_id = $id
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.
FETCH NEXT 1 ROW ONLY
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT ledger_effective_time
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
FETCH NEXT 1 ROW ONLY"""
}
override def activeContractSqlLiteral(
contractId: ContractId,
treeEventWitnessesClause: CompositeSql,
resultColumns: List[String],
coalescedColumns: String,
): SimpleSql[Row] = {
import com.daml.platform.store.Conversions.ContractIdToStatement
SQL""" WITH archival_event AS (
SELECT 1
FROM participant_events_consuming_exercise, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause -- only use visible archivals
FETCH NEXT 1 ROW ONLY
),
create_event AS (
SELECT contract_id, #${resultColumns.mkString(", ")}
FROM participant_events_create, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause
FETCH NEXT 1 ROW ONLY -- 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, #${resultColumns.mkString(", ")}
FROM participant_events_create, parameters
WHERE contract_id = $contractId
AND event_sequential_id <= parameters.ledger_end_sequential_id
FETCH NEXT 1 ROW ONLY -- limit here to guide planner wrt expected number of results
),
divulged_contract AS (
SELECT divulgence_events.contract_id,
-- Note: the divulgence_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.
#$coalescedColumns
FROM participant_events_divulgence divulgence_events LEFT OUTER JOIN create_event_unrestricted ON (divulgence_events.contract_id = create_event_unrestricted.contract_id),
parameters
WHERE divulgence_events.contract_id = $contractId -- restrict to aid query planner
AND divulgence_events.event_sequential_id <= parameters.ledger_end_sequential_id
AND $treeEventWitnessesClause
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.
FETCH NEXT 1 ROW ONLY
),
create_and_divulged_contracts AS (
(SELECT * FROM create_event) -- prefer create over divulgence events
UNION ALL
(SELECT * FROM divulged_contract)
)
SELECT contract_id, #${resultColumns.mkString(", ")}
FROM create_and_divulged_contracts
WHERE NOT EXISTS (SELECT 1 FROM archival_event)
FETCH NEXT 1 ROW ONLY"""
}
}

View File

@ -0,0 +1,71 @@
// 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.h2
import com.daml.platform.store.backend.common.{
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
IngestionStorageBackendTemplate,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
}
import com.daml.platform.store.backend.{
CompletionStorageBackend,
ConfigurationStorageBackend,
ContractStorageBackend,
DBLockStorageBackend,
DataSourceStorageBackend,
DeduplicationStorageBackend,
EventStorageBackend,
IngestionStorageBackend,
IntegrityStorageBackend,
PackageStorageBackend,
ParameterStorageBackend,
PartyStorageBackend,
ResetStorageBackend,
StorageBackendFactory,
}
object H2StorageBackendFactory extends StorageBackendFactory {
override val createIngestionStorageBackend: IngestionStorageBackend[_] =
new IngestionStorageBackendTemplate(H2Schema.schema)
override val createParameterStorageBackend: ParameterStorageBackend =
ParameterStorageBackendTemplate
override val createConfigurationStorageBackend: ConfigurationStorageBackend =
ConfigurationStorageBackendTemplate
override val createPartyStorageBackend: PartyStorageBackend =
new PartyStorageBackendTemplate(H2QueryStrategy)
override val createPackageStorageBackend: PackageStorageBackend =
PackageStorageBackendTemplate
override val createDeduplicationStorageBackend: DeduplicationStorageBackend =
H2DeduplicationStorageBackend
override val createCompletionStorageBackend: CompletionStorageBackend =
new CompletionStorageBackendTemplate(H2QueryStrategy)
override val createContractStorageBackend: ContractStorageBackend =
H2ContractStorageBackend
override val createEventStorageBackend: EventStorageBackend =
H2EventStorageBackend
override val createDataSourceStorageBackend: DataSourceStorageBackend =
H2DataSourceStorageBackend
override val createDBLockStorageBackend: DBLockStorageBackend =
H2DBLockStorageBackend
override val createIntegrityStorageBackend: IntegrityStorageBackend =
IntegrityStorageBackendTemplate
override val createResetStorageBackend: ResetStorageBackend =
H2ResetStorageBackend
}

View File

@ -0,0 +1,72 @@
// 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.oracle
import java.sql.Connection
import anorm.SqlParser.get
import anorm.SqlStringInterpolation
import com.daml.platform.store.backend.DBLockStorageBackend
object OracleDBLockStorageBackend extends DBLockStorageBackend {
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] = {
val oracleLockMode = lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "6" // "DBMS_LOCK.x_mode"
case DBLockStorageBackend.LockMode.Shared => "4" // "DBMS_LOCK.s_mode"
}
SQL"""
SELECT DBMS_LOCK.REQUEST(
id => ${oracleIntLockId(lockId)},
lockmode => #$oracleLockMode,
timeout => 0
) FROM DUAL"""
.as(get[Int](1).single)(connection) match {
case 0 => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case 1 => None
case 2 => throw new Exception("DBMS_LOCK.REQUEST Error 2: Acquiring lock caused a deadlock!")
case 3 => throw new Exception("DBMS_LOCK.REQUEST Error 3: Parameter error as acquiring lock")
case 4 => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case 5 =>
throw new Exception("DBMS_LOCK.REQUEST Error 5: Illegal lock handle as acquiring lock")
case unknown => throw new Exception(s"Invalid result from DBMS_LOCK.REQUEST: $unknown")
}
}
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean = {
SQL"""
SELECT DBMS_LOCK.RELEASE(
id => ${oracleIntLockId(lock.lockId)}
) FROM DUAL"""
.as(get[Int](1).single)(connection) match {
case 0 => true
case 3 => throw new Exception("DBMS_LOCK.RELEASE Error 3: Parameter error as releasing lock")
case 4 => false
case 5 =>
throw new Exception("DBMS_LOCK.RELEASE Error 5: Illegal lock handle as releasing lock")
case unknown => throw new Exception(s"Invalid result from DBMS_LOCK.RELEASE: $unknown")
}
}
case class OracleLockId(id: Int) extends DBLockStorageBackend.LockId {
// respecting Oracle limitations: https://docs.oracle.com/cd/B19306_01/appdev.102/b14258/d_lock.htm#ARPLS021
assert(id >= 0, s"Lock id $id is too small for Oracle")
assert(id <= 1073741823, s"Lock id $id is too large for Oracle")
}
private def oracleIntLockId(lockId: DBLockStorageBackend.LockId): Int =
lockId match {
case OracleLockId(id) => id
case unknown =>
throw new Exception(
s"LockId $unknown not supported. Probable cause: LockId was created by a different StorageBackend"
)
}
override def lock(id: Int): DBLockStorageBackend.LockId = OracleLockId(id)
override def dbLockSupported: Boolean = true
}

View File

@ -0,0 +1,28 @@
// 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.oracle
import java.sql.Connection
import anorm.SqlParser.get
import anorm.SqlStringInterpolation
import com.daml.logging.LoggingContext
import com.daml.platform.store.backend.DataSourceStorageBackend
import com.daml.platform.store.backend.common.InitHookDataSourceProxy
import javax.sql.DataSource
object OracleDataSourceStorageBackend extends DataSourceStorageBackend {
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
val oracleDataSource = new oracle.jdbc.pool.OracleDataSource
oracleDataSource.setURL(jdbcUrl)
InitHookDataSourceProxy(oracleDataSource, connectionInitHook.toList)
}
override def checkDatabaseAvailable(connection: Connection): Unit =
assert(SQL"SELECT 1 FROM DUAL".as(get[Int](1).single)(connection) == 1)
}

View File

@ -0,0 +1,58 @@
// 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.oracle
import java.sql.Connection
import anorm.SQL
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.backend.common.DeduplicationStorageBackendTemplate
import scala.util.control.NonFatal
object OracleDeduplicationStorageBackend extends DeduplicationStorageBackendTemplate {
private val logger = ContextualizedLogger.get(this.getClass)
val SQL_INSERT_COMMAND: String =
"""merge into participant_command_submissions pcs
|using dual
|on (pcs.deduplication_key ={deduplicationKey})
|when matched then
| update set pcs.deduplicate_until={deduplicateUntil}
| where pcs.deduplicate_until < {submittedAt}
|when not matched then
| insert (pcs.deduplication_key, pcs.deduplicate_until)
| values ({deduplicationKey}, {deduplicateUntil})""".stripMargin
override def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int = {
// Under the default READ_COMMITTED isolation level used for the indexdb, when a deduplication
// upsert is performed simultaneously from multiple threads, the query fails with
// SQLIntegrityConstraintViolationException: ORA-00001: unique constraint (INDEXDB.SYS_C007590) violated
// Simple retry helps
def retry[T](op: => T): T =
try {
op
} catch {
case NonFatal(e) =>
logger.debug(s"Caught exception while upserting a deduplication entry: $e")
op
}
retry(
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
)
}
}

View File

@ -0,0 +1,50 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
package com.daml.platform.store.backend.oracle
import java.sql.Connection
import anorm.SqlParser.get
import anorm.SqlStringInterpolation
import com.daml.ledger.offset.Offset
import com.daml.platform.store.backend.common.{
EventStorageBackendTemplate,
ParameterStorageBackendTemplate,
}
object OracleEventStorageBackend
extends EventStorageBackendTemplate(
eventStrategy = OracleEventStrategy,
queryStrategy = OracleQueryStrategy,
participantAllDivulgedContractsPrunedUpToInclusive =
ParameterStorageBackendTemplate.participantAllDivulgedContractsPrunedUpToInclusive,
) {
def maxEventSequentialIdOfAnObservableEvent(
offset: Offset
)(connection: Connection): Option[Long] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""SELECT max(max_esi) FROM (
(
SELECT max(event_sequential_id) AS max_esi FROM participant_events_consuming_exercise
WHERE event_offset = (select max(event_offset) from participant_events_consuming_exercise where event_offset <= $offset)
) UNION ALL (
SELECT max(event_sequential_id) AS max_esi FROM participant_events_create
WHERE event_offset = (select max(event_offset) from participant_events_create where event_offset <= $offset)
) UNION ALL (
SELECT max(event_sequential_id) AS max_esi FROM participant_events_non_consuming_exercise
WHERE event_offset = (select max(event_offset) from participant_events_non_consuming_exercise where event_offset <= $offset)
)
)"""
.as(get[Long](1).?.single)(connection)
}
// Migration from mutable schema is not supported for Oracle
override def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean = true
}

View File

@ -0,0 +1,54 @@
// 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.oracle
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.EventStrategy
object OracleEventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
if (parties.size == 1)
cSQL"(json_array(${parties.head.toString}))"
else
cSQL"""
(select json_arrayagg(value) from (select value
from json_table(#$witnessesColumnName, '$$[*]' columns (value PATH '$$'))
where value IN (${parties.map(_.toString)})))
"""
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
cSQL"(${OracleQueryStrategy.arrayIntersectionNonEmptyClause(submittersColumnName, parties)})"
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
cSQL"(${OracleQueryStrategy.arrayIntersectionNonEmptyClause(witnessesColumnName, wildCardParties)})" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val clause =
OracleQueryStrategy.arrayIntersectionNonEmptyClause(
witnessesColumnName,
parties,
)
cSQL"( ($clause) AND (template_id IN (${templateIds.map(_.toString)})) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}

View File

@ -0,0 +1,28 @@
// 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.oracle
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.QueryStrategy
object OracleQueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql =
cSQL"EXISTS (SELECT 1 FROM JSON_TABLE(#$columnName, '$$[*]' columns (value PATH '$$')) WHERE value IN (${parties
.map(_.toString)}))"
override def columnEqualityBoolean(column: String, value: String): String =
s"""case when ($column = $value) then 1 else 0 end"""
override def booleanOrAggregationFunction: String = "max"
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"EXISTS (SELECT 1 FROM JSON_TABLE($arrayColumnName, '$$[*]' columns (value PATH '$$')) WHERE value = $elementColumnName)"
override def isTrue(booleanColumnName: String): String = s"$booleanColumnName = 1"
}

View File

@ -0,0 +1,40 @@
// 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.oracle
import java.sql.Connection
import anorm.SQL
import com.daml.platform.store.backend.ResetStorageBackend
object OracleResetStorageBackend extends ResetStorageBackend {
override def reset(connection: Connection): Unit =
List(
"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 party_entries cascade",
).map(SQL(_)).foreach(_.execute()(connection))
override def resetAll(connection: Connection): Unit =
List(
"truncate table configuration_entries cascade",
"truncate table packages 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 party_entries cascade",
).map(SQL(_)).foreach(_.execute()(connection))
}

View File

@ -1,302 +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.backend.oracle
import anorm.SqlParser.get
import anorm.SQL
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.common.{
AppendOnlySchema,
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
ContractStorageBackendTemplate,
DataSourceStorageBackendTemplate,
DeduplicationStorageBackendTemplate,
EventStorageBackendTemplate,
EventStrategy,
IngestionStorageBackendTemplate,
InitHookDataSourceProxy,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
QueryStrategy,
}
import com.daml.platform.store.backend.{
DBLockStorageBackend,
DataSourceStorageBackend,
DbDto,
StorageBackend,
common,
}
import java.sql.Connection
import com.daml.ledger.offset.Offset
import com.daml.lf.data.Time.Timestamp
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import javax.sql.DataSource
import scala.util.control.NonFatal
private[backend] object OracleStorageBackend
extends StorageBackend[AppendOnlySchema.Batch]
with DataSourceStorageBackendTemplate
with IngestionStorageBackendTemplate[AppendOnlySchema.Batch]
with ParameterStorageBackendTemplate
with ConfigurationStorageBackendTemplate
with PackageStorageBackendTemplate
with DeduplicationStorageBackendTemplate
with EventStorageBackendTemplate
with ContractStorageBackendTemplate
with CompletionStorageBackendTemplate
with PartyStorageBackendTemplate
with IntegrityStorageBackendTemplate {
private val logger = ContextualizedLogger.get(this.getClass)
override def reset(connection: Connection): Unit =
List(
"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 party_entries cascade",
).map(SQL(_)).foreach(_.execute()(connection))
override def resetAll(connection: Connection): Unit =
List(
"truncate table configuration_entries cascade",
"truncate table packages 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 party_entries cascade",
).map(SQL(_)).foreach(_.execute()(connection))
val SQL_INSERT_COMMAND: String =
"""merge into participant_command_submissions pcs
|using dual
|on (pcs.deduplication_key ={deduplicationKey})
|when matched then
| update set pcs.deduplicate_until={deduplicateUntil}
| where pcs.deduplicate_until < {submittedAt}
|when not matched then
| insert (pcs.deduplication_key, pcs.deduplicate_until)
| values ({deduplicationKey}, {deduplicateUntil})""".stripMargin
override def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int = {
// Under the default READ_COMMITTED isolation level used for the indexdb, when a deduplication
// upsert is performed simultaneously from multiple threads, the query fails with
// SQLIntegrityConstraintViolationException: ORA-00001: unique constraint (INDEXDB.SYS_C007590) violated
// Simple retry helps
def retry[T](op: => T): T =
try {
op
} catch {
case NonFatal(e) =>
logger.debug(s"Caught exception while upserting a deduplication entry: $e")
op
}
retry(
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
)
}
override def batch(dbDtos: Vector[DbDto]): AppendOnlySchema.Batch =
OracleSchema.schema.prepareData(dbDtos)
override def insertBatch(connection: Connection, batch: AppendOnlySchema.Batch): Unit =
OracleSchema.schema.executeUpdate(batch, connection)
object OracleQueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql =
cSQL"EXISTS (SELECT 1 FROM JSON_TABLE(#$columnName, '$$[*]' columns (value PATH '$$')) WHERE value IN (${parties
.map(_.toString)}))"
override def columnEqualityBoolean(column: String, value: String): String =
s"""case when ($column = $value) then 1 else 0 end"""
override def booleanOrAggregationFunction: String = "max"
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"EXISTS (SELECT 1 FROM JSON_TABLE($arrayColumnName, '$$[*]' columns (value PATH '$$')) WHERE value = $elementColumnName)"
override def isTrue(booleanColumnName: String): String = s"$booleanColumnName = 1"
}
override def queryStrategy: QueryStrategy = OracleQueryStrategy
object OracleEventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
if (parties.size == 1)
cSQL"(json_array(${parties.head.toString}))"
else
cSQL"""
(select json_arrayagg(value) from (select value
from json_table(#$witnessesColumnName, '$$[*]' columns (value PATH '$$'))
where value IN (${parties.map(_.toString)})))
"""
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Ref.Party],
): CompositeSql =
cSQL"(${OracleQueryStrategy.arrayIntersectionNonEmptyClause(submittersColumnName, parties)})"
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
cSQL"(${OracleQueryStrategy.arrayIntersectionNonEmptyClause(witnessesColumnName, wildCardParties)})" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val clause =
OracleQueryStrategy.arrayIntersectionNonEmptyClause(
witnessesColumnName,
parties,
)
cSQL"( ($clause) AND (template_id IN (${templateIds.map(_.toString)})) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}
override def eventStrategy: common.EventStrategy = OracleEventStrategy
def maxEventSequentialIdOfAnObservableEvent(
offset: Offset
)(connection: Connection): Option[Long] = {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""SELECT max(max_esi) FROM (
(
SELECT max(event_sequential_id) AS max_esi FROM participant_events_consuming_exercise
WHERE event_offset = (select max(event_offset) from participant_events_consuming_exercise where event_offset <= $offset)
) UNION ALL (
SELECT max(event_sequential_id) AS max_esi FROM participant_events_create
WHERE event_offset = (select max(event_offset) from participant_events_create where event_offset <= $offset)
) UNION ALL (
SELECT max(event_sequential_id) AS max_esi FROM participant_events_non_consuming_exercise
WHERE event_offset = (select max(event_offset) from participant_events_non_consuming_exercise where event_offset <= $offset)
)
)"""
.as(get[Long](1).?.single)(connection)
}
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
val oracleDataSource = new oracle.jdbc.pool.OracleDataSource
oracleDataSource.setURL(jdbcUrl)
InitHookDataSourceProxy(oracleDataSource, connectionInitHook.toList)
}
override def checkDatabaseAvailable(connection: Connection): Unit =
assert(SQL"SELECT 1 FROM DUAL".as(get[Int](1).single)(connection) == 1)
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] = {
val oracleLockMode = lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "6" // "DBMS_LOCK.x_mode"
case DBLockStorageBackend.LockMode.Shared => "4" // "DBMS_LOCK.s_mode"
}
SQL"""
SELECT DBMS_LOCK.REQUEST(
id => ${oracleIntLockId(lockId)},
lockmode => #$oracleLockMode,
timeout => 0
) FROM DUAL"""
.as(get[Int](1).single)(connection) match {
case 0 => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case 1 => None
case 2 => throw new Exception("DBMS_LOCK.REQUEST Error 2: Acquiring lock caused a deadlock!")
case 3 => throw new Exception("DBMS_LOCK.REQUEST Error 3: Parameter error as acquiring lock")
case 4 => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case 5 =>
throw new Exception("DBMS_LOCK.REQUEST Error 5: Illegal lock handle as acquiring lock")
case unknown => throw new Exception(s"Invalid result from DBMS_LOCK.REQUEST: $unknown")
}
}
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean = {
SQL"""
SELECT DBMS_LOCK.RELEASE(
id => ${oracleIntLockId(lock.lockId)}
) FROM DUAL"""
.as(get[Int](1).single)(connection) match {
case 0 => true
case 3 => throw new Exception("DBMS_LOCK.RELEASE Error 3: Parameter error as releasing lock")
case 4 => false
case 5 =>
throw new Exception("DBMS_LOCK.RELEASE Error 5: Illegal lock handle as releasing lock")
case unknown => throw new Exception(s"Invalid result from DBMS_LOCK.RELEASE: $unknown")
}
}
case class OracleLockId(id: Int) extends DBLockStorageBackend.LockId {
// respecting Oracle limitations: https://docs.oracle.com/cd/B19306_01/appdev.102/b14258/d_lock.htm#ARPLS021
assert(id >= 0, s"Lock id $id is too small for Oracle")
assert(id <= 1073741823, s"Lock id $id is too large for Oracle")
}
private def oracleIntLockId(lockId: DBLockStorageBackend.LockId): Int =
lockId match {
case OracleLockId(id) => id
case unknown =>
throw new Exception(
s"LockId $unknown not supported. Probable cause: LockId was created by a different StorageBackend"
)
}
override def lock(id: Int): DBLockStorageBackend.LockId = OracleLockId(id)
override def dbLockSupported: Boolean = true
// Migration from mutable schema is not supported for Oracle
override def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean = true
}

View File

@ -0,0 +1,72 @@
// 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.oracle
import com.daml.platform.store.backend.common.{
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
ContractStorageBackendTemplate,
IngestionStorageBackendTemplate,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
}
import com.daml.platform.store.backend.{
CompletionStorageBackend,
ConfigurationStorageBackend,
ContractStorageBackend,
DBLockStorageBackend,
DataSourceStorageBackend,
DeduplicationStorageBackend,
EventStorageBackend,
IngestionStorageBackend,
IntegrityStorageBackend,
PackageStorageBackend,
ParameterStorageBackend,
PartyStorageBackend,
ResetStorageBackend,
StorageBackendFactory,
}
object OracleStorageBackendFactory extends StorageBackendFactory {
override val createIngestionStorageBackend: IngestionStorageBackend[_] =
new IngestionStorageBackendTemplate(OracleSchema.schema)
override val createParameterStorageBackend: ParameterStorageBackend =
ParameterStorageBackendTemplate
override val createConfigurationStorageBackend: ConfigurationStorageBackend =
ConfigurationStorageBackendTemplate
override val createPartyStorageBackend: PartyStorageBackend =
new PartyStorageBackendTemplate(OracleQueryStrategy)
override val createPackageStorageBackend: PackageStorageBackend =
PackageStorageBackendTemplate
override val createDeduplicationStorageBackend: DeduplicationStorageBackend =
OracleDeduplicationStorageBackend
override val createCompletionStorageBackend: CompletionStorageBackend =
new CompletionStorageBackendTemplate(OracleQueryStrategy)
override val createContractStorageBackend: ContractStorageBackend =
new ContractStorageBackendTemplate(OracleQueryStrategy)
override val createEventStorageBackend: EventStorageBackend =
OracleEventStorageBackend
override val createDataSourceStorageBackend: DataSourceStorageBackend =
OracleDataSourceStorageBackend
override val createDBLockStorageBackend: DBLockStorageBackend =
OracleDBLockStorageBackend
override val createIntegrityStorageBackend: IntegrityStorageBackend =
IntegrityStorageBackendTemplate
override val createResetStorageBackend: ResetStorageBackend =
OracleResetStorageBackend
}

View File

@ -0,0 +1,52 @@
// 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.postgresql
import java.sql.Connection
import anorm.SqlStringInterpolation
import anorm.SqlParser.get
import com.daml.platform.store.backend.DBLockStorageBackend
object PostgresDBLockStorageBackend extends DBLockStorageBackend {
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] = {
val lockFunction = lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "pg_try_advisory_lock"
case DBLockStorageBackend.LockMode.Shared => "pg_try_advisory_lock_shared"
}
SQL"SELECT #$lockFunction(${pgBigintLockId(lockId)})"
.as(get[Boolean](1).single)(connection) match {
case true => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case false => None
}
}
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean = {
val lockFunction = lock.lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "pg_advisory_unlock"
case DBLockStorageBackend.LockMode.Shared => "pg_advisory_unlock_shared"
}
SQL"SELECT #$lockFunction(${pgBigintLockId(lock.lockId)})"
.as(get[Boolean](1).single)(connection)
}
case class PGLockId(id: Long) extends DBLockStorageBackend.LockId
private def pgBigintLockId(lockId: DBLockStorageBackend.LockId): Long =
lockId match {
case PGLockId(id) => id
case unknown =>
throw new Exception(
s"LockId $unknown not supported. Probable cause: LockId was created by a different StorageBackend"
)
}
override def lock(id: Int): DBLockStorageBackend.LockId = PGLockId(id.toLong)
override def dbLockSupported: Boolean = true
}

View File

@ -1,25 +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.backend.postgresql
import com.daml.platform.store.backend.postgresql.PostgresDataSourceConfig.SynchronousCommitValue
case class PostgresDataSourceConfig(
synchronousCommit: Option[SynchronousCommitValue] = None,
// TCP keepalive configuration for postgres. See https://www.postgresql.org/docs/13/runtime-config-connection.html#RUNTIME-CONFIG-CONNECTION-SETTINGS for details
tcpKeepalivesIdle: Option[Int] = None, // corresponds to: tcp_keepalives_idle
tcpKeepalivesInterval: Option[Int] = None, // corresponds to: tcp_keepalives_interval
tcpKeepalivesCount: Option[Int] = None, // corresponds to: tcp_keepalives_count
)
object PostgresDataSourceConfig {
sealed abstract class SynchronousCommitValue(val pgSqlName: String)
object SynchronousCommitValue {
case object On extends SynchronousCommitValue("on")
case object Off extends SynchronousCommitValue("off")
case object RemoteWrite extends SynchronousCommitValue("remote_write")
case object RemoteApply extends SynchronousCommitValue("remote_apply")
case object Local extends SynchronousCommitValue("local")
}
}

View File

@ -0,0 +1,105 @@
// 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.postgresql
import java.sql.Connection
import anorm.SqlParser.get
import anorm.SqlStringInterpolation
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.backend.DataSourceStorageBackend
import com.daml.platform.store.backend.common.{
DataSourceStorageBackendTemplate,
InitHookDataSourceProxy,
}
import javax.sql.DataSource
import org.postgresql.ds.PGSimpleDataSource
import com.daml.platform.store.backend.postgresql.PostgresDataSourceConfig.SynchronousCommitValue
case class PostgresDataSourceConfig(
synchronousCommit: Option[SynchronousCommitValue] = None,
// TCP keepalive configuration for postgres. See https://www.postgresql.org/docs/13/runtime-config-connection.html#RUNTIME-CONFIG-CONNECTION-SETTINGS for details
tcpKeepalivesIdle: Option[Int] = None, // corresponds to: tcp_keepalives_idle
tcpKeepalivesInterval: Option[Int] = None, // corresponds to: tcp_keepalives_interval
tcpKeepalivesCount: Option[Int] = None, // corresponds to: tcp_keepalives_count
)
object PostgresDataSourceConfig {
sealed abstract class SynchronousCommitValue(val pgSqlName: String)
object SynchronousCommitValue {
case object On extends SynchronousCommitValue("on")
case object Off extends SynchronousCommitValue("off")
case object RemoteWrite extends SynchronousCommitValue("remote_write")
case object RemoteApply extends SynchronousCommitValue("remote_apply")
case object Local extends SynchronousCommitValue("local")
}
}
object PostgresDataSourceStorageBackend extends DataSourceStorageBackend {
private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass)
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
import DataSourceStorageBackendTemplate.exe
val pgSimpleDataSource = new PGSimpleDataSource()
pgSimpleDataSource.setUrl(jdbcUrl)
val hookFunctions = List(
dataSourceConfig.postgresConfig.synchronousCommit.toList
.map(synchCommitValue => exe(s"SET synchronous_commit TO ${synchCommitValue.pgSqlName}")),
dataSourceConfig.postgresConfig.tcpKeepalivesIdle.toList
.map(i => exe(s"SET tcp_keepalives_idle TO $i")),
dataSourceConfig.postgresConfig.tcpKeepalivesInterval.toList
.map(i => exe(s"SET tcp_keepalives_interval TO $i")),
dataSourceConfig.postgresConfig.tcpKeepalivesCount.toList
.map(i => exe(s"SET tcp_keepalives_count TO $i")),
connectionInitHook.toList,
).flatten
InitHookDataSourceProxy(pgSimpleDataSource, hookFunctions)
}
override def checkCompatibility(
connection: Connection
)(implicit loggingContext: LoggingContext): Unit = {
getPostgresVersion(connection) match {
case Some((major, minor)) =>
if (major < 10) {
logger.error(
"Deprecated Postgres version. " +
s"Found Postgres version $major.$minor, minimum required Postgres version is 10. " +
"This application will continue running but is at risk of data loss, as Postgres < 10 does not support crash-fault tolerant hash indices. " +
"Please upgrade your Postgres database to version 10 or later to fix this issue."
)
}
case None =>
logger.warn(
s"Could not determine the version of the Postgres database. Please verify that this application is compatible with this Postgres version."
)
}
()
}
private[backend] def getPostgresVersion(
connection: Connection
)(implicit loggingContext: LoggingContext): Option[(Int, Int)] = {
val version = SQL"SHOW server_version".as(get[String](1).single)(connection)
logger.debug(s"Found Postgres version $version")
parsePostgresVersion(version)
}
private[backend] def parsePostgresVersion(version: String): Option[(Int, Int)] = {
val versionPattern = """(\d+)[.](\d+).*""".r
version match {
case versionPattern(major, minor) => Some((major.toInt, minor.toInt))
case _ => None
}
}
override def checkDatabaseAvailable(connection: Connection): Unit =
DataSourceStorageBackendTemplate.checkDatabaseAvailable(connection)
}

View File

@ -0,0 +1,35 @@
// 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.postgresql
import java.sql.Connection
import anorm.SQL
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.LoggingContext
import com.daml.platform.store.backend.common.DeduplicationStorageBackendTemplate
object PostgresDeduplicationStorageBackend extends DeduplicationStorageBackendTemplate {
private 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 def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int =
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
}

View File

@ -0,0 +1,62 @@
// 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.postgresql
import java.sql.Connection
import anorm.SqlParser.{get, int}
import com.daml.ledger.offset.Offset
import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation
import com.daml.platform.store.backend.common.{
EventStorageBackendTemplate,
ParameterStorageBackendTemplate,
}
object PostgresEventStorageBackend
extends EventStorageBackendTemplate(
eventStrategy = PostgresEventStrategy,
queryStrategy = PostgresQueryStrategy,
participantAllDivulgedContractsPrunedUpToInclusive =
ParameterStorageBackendTemplate.participantAllDivulgedContractsPrunedUpToInclusive,
) {
// TODO FIXME: Use tables directly instead of the participant_events view.
override def maxEventSequentialIdOfAnObservableEvent(
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)
}
/** If `pruneAllDivulgedContracts` is set, validate that the pruning offset is after
* the last event offset that was ingested before the migration to append-only schema (if such event offset exists).
* (see [[com.daml.platform.store.appendonlydao.JdbcLedgerDao.prune]])
*/
override def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean =
if (pruneAllDivulgedContracts) {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""
with max_offset_before_migration as (
select max(event_offset) as max_event_offset
from participant_events, participant_migration_history_v100
where event_sequential_id <= ledger_end_sequential_id_before
)
select 1 as result
from max_offset_before_migration
where max_event_offset >= $pruneUpToInclusive
"""
.as(int("result").singleOpt)(connection)
.isEmpty
} else {
true
}
}

View File

@ -0,0 +1,51 @@
// 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.postgresql
import com.daml.platform.store.appendonlydao.events.Party
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.EventStrategy
object PostgresEventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Party],
): CompositeSql =
if (parties.size == 1)
cSQL"array[${parties.head.toString}]::text[]"
else {
val partiesArray: Array[String] = parties.view.map(_.toString).toArray
cSQL"array(select unnest(#$witnessesColumnName) intersect select unnest($partiesArray::text[]))"
}
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Party],
): CompositeSql = {
val partiesArray = parties.view.map(_.toString).toArray
cSQL"(#$submittersColumnName::text[] && $partiesArray::text[])"
}
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
val partiesArray = wildCardParties.view.map(_.toString).toArray
cSQL"(#$witnessesColumnName::text[] && $partiesArray::text[])" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val partiesArray = parties.view.map(_.toString).toArray
val templateIdsArray = templateIds.view.map(_.toString).toArray
cSQL"( (#$witnessesColumnName::text[] && $partiesArray::text[]) AND (template_id = ANY($templateIdsArray::text[])) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}

View File

@ -0,0 +1,25 @@
// 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.postgresql
import com.daml.lf.data.Ref
import com.daml.platform.store.backend.common.ComposableQuery.CompositeSql
import com.daml.platform.store.backend.common.QueryStrategy
object PostgresQueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql = {
import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation
val partiesArray: Array[String] = parties.map(_.toString).toArray
cSQL"#$columnName::text[] && $partiesArray::text[]"
}
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"$elementColumnName = any($arrayColumnName)"
override def isTrue(booleanColumnName: String): String = booleanColumnName
}

View File

@ -0,0 +1,44 @@
// 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.postgresql
import java.sql.Connection
import anorm.SQL
import com.daml.platform.store.backend.ResetStorageBackend
object PostgresResetStorageBackend extends ResetStorageBackend {
override 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 party_entries cascade;
|""".stripMargin)
.execute()(connection)
()
}
override def resetAll(connection: Connection): Unit = {
SQL("""truncate table configuration_entries cascade;
|truncate table packages 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 party_entries cascade;
|""".stripMargin)
.execute()(connection)
()
}
}

View File

@ -1,322 +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.backend.postgresql
import java.sql.Connection
import anorm.SQL
import anorm.SqlParser.{get, int}
import com.daml.ledger.offset.Offset
import com.daml.lf.data.Ref
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.platform.store.appendonlydao.events.Party
import com.daml.platform.store.backend.EventStorageBackend.FilterParams
import com.daml.platform.store.backend.common.ComposableQuery.{CompositeSql, SqlStringInterpolation}
import com.daml.platform.store.backend.common.{
AppendOnlySchema,
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
ContractStorageBackendTemplate,
DataSourceStorageBackendTemplate,
DeduplicationStorageBackendTemplate,
EventStorageBackendTemplate,
EventStrategy,
IngestionStorageBackendTemplate,
InitHookDataSourceProxy,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
QueryStrategy,
}
import com.daml.platform.store.backend.{
DBLockStorageBackend,
DataSourceStorageBackend,
DbDto,
StorageBackend,
common,
}
import javax.sql.DataSource
import org.postgresql.ds.PGSimpleDataSource
private[backend] object PostgresStorageBackend
extends StorageBackend[AppendOnlySchema.Batch]
with DataSourceStorageBackendTemplate
with IngestionStorageBackendTemplate[AppendOnlySchema.Batch]
with ParameterStorageBackendTemplate
with ConfigurationStorageBackendTemplate
with PackageStorageBackendTemplate
with DeduplicationStorageBackendTemplate
with EventStorageBackendTemplate
with ContractStorageBackendTemplate
with CompletionStorageBackendTemplate
with PartyStorageBackendTemplate
with IntegrityStorageBackendTemplate {
private val logger: ContextualizedLogger = ContextualizedLogger.get(this.getClass)
override def insertBatch(
connection: Connection,
postgresDbBatch: AppendOnlySchema.Batch,
): Unit =
PGSchema.schema.executeUpdate(postgresDbBatch, connection)
override def batch(dbDtos: Vector[DbDto]): AppendOnlySchema.Batch =
PGSchema.schema.prepareData(dbDtos)
private 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 def upsertDeduplicationEntry(
key: String,
submittedAt: Timestamp,
deduplicateUntil: Timestamp,
)(connection: Connection)(implicit loggingContext: LoggingContext): Int =
SQL(SQL_INSERT_COMMAND)
.on(
"deduplicationKey" -> key,
"submittedAt" -> submittedAt.micros,
"deduplicateUntil" -> deduplicateUntil.micros,
)
.executeUpdate()(connection)
override 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 party_entries cascade;
|""".stripMargin)
.execute()(connection)
()
}
override def resetAll(connection: Connection): Unit = {
SQL("""truncate table configuration_entries cascade;
|truncate table packages 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 party_entries cascade;
|""".stripMargin)
.execute()(connection)
()
}
/** If `pruneAllDivulgedContracts` is set, validate that the pruning offset is after
* the last event offset that was ingested before the migration to append-only schema (if such event offset exists).
* (see [[com.daml.platform.store.appendonlydao.JdbcLedgerDao.prune]])
*/
def isPruningOffsetValidAgainstMigration(
pruneUpToInclusive: Offset,
pruneAllDivulgedContracts: Boolean,
connection: Connection,
): Boolean =
if (pruneAllDivulgedContracts) {
import com.daml.platform.store.Conversions.OffsetToStatement
SQL"""
with max_offset_before_migration as (
select max(event_offset) as max_event_offset
from participant_events, participant_migration_history_v100
where event_sequential_id <= ledger_end_sequential_id_before
)
select 1 as result
from max_offset_before_migration
where max_event_offset >= $pruneUpToInclusive
"""
.as(int("result").singleOpt)(connection)
.isEmpty
} else {
true
}
object PostgresQueryStrategy extends QueryStrategy {
override def arrayIntersectionNonEmptyClause(
columnName: String,
parties: Set[Ref.Party],
): CompositeSql = {
import com.daml.platform.store.backend.common.ComposableQuery.SqlStringInterpolation
val partiesArray: Array[String] = parties.map(_.toString).toArray
cSQL"#$columnName::text[] && $partiesArray::text[]"
}
override def arrayContains(arrayColumnName: String, elementColumnName: String): String =
s"$elementColumnName = any($arrayColumnName)"
override def isTrue(booleanColumnName: String): String = booleanColumnName
}
override def queryStrategy: QueryStrategy = PostgresQueryStrategy
object PostgresEventStrategy extends EventStrategy {
override def filteredEventWitnessesClause(
witnessesColumnName: String,
parties: Set[Party],
): CompositeSql =
if (parties.size == 1)
cSQL"array[${parties.head.toString}]::text[]"
else {
val partiesArray: Array[String] = parties.view.map(_.toString).toArray
cSQL"array(select unnest(#$witnessesColumnName) intersect select unnest($partiesArray::text[]))"
}
override def submittersArePartiesClause(
submittersColumnName: String,
parties: Set[Party],
): CompositeSql = {
val partiesArray = parties.view.map(_.toString).toArray
cSQL"(#$submittersColumnName::text[] && $partiesArray::text[])"
}
override def witnessesWhereClause(
witnessesColumnName: String,
filterParams: FilterParams,
): CompositeSql = {
val wildCardClause = filterParams.wildCardParties match {
case wildCardParties if wildCardParties.isEmpty =>
Nil
case wildCardParties =>
val partiesArray = wildCardParties.view.map(_.toString).toArray
cSQL"(#$witnessesColumnName::text[] && $partiesArray::text[])" :: Nil
}
val partiesTemplatesClauses =
filterParams.partiesAndTemplates.iterator.map { case (parties, templateIds) =>
val partiesArray = parties.view.map(_.toString).toArray
val templateIdsArray = templateIds.view.map(_.toString).toArray
cSQL"( (#$witnessesColumnName::text[] && $partiesArray::text[]) AND (template_id = ANY($templateIdsArray::text[])) )"
}.toList
(wildCardClause ::: partiesTemplatesClauses).mkComposite("(", " OR ", ")")
}
}
override def eventStrategy: common.EventStrategy = PostgresEventStrategy
// TODO FIXME: Use tables directly instead of the participant_events view.
override def maxEventSequentialIdOfAnObservableEvent(
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)
}
override def createDataSource(
jdbcUrl: String,
dataSourceConfig: DataSourceStorageBackend.DataSourceConfig,
connectionInitHook: Option[Connection => Unit],
)(implicit loggingContext: LoggingContext): DataSource = {
val pgSimpleDataSource = new PGSimpleDataSource()
pgSimpleDataSource.setUrl(jdbcUrl)
val hookFunctions = List(
dataSourceConfig.postgresConfig.synchronousCommit.toList
.map(synchCommitValue => exe(s"SET synchronous_commit TO ${synchCommitValue.pgSqlName}")),
dataSourceConfig.postgresConfig.tcpKeepalivesIdle.toList
.map(i => exe(s"SET tcp_keepalives_idle TO $i")),
dataSourceConfig.postgresConfig.tcpKeepalivesInterval.toList
.map(i => exe(s"SET tcp_keepalives_interval TO $i")),
dataSourceConfig.postgresConfig.tcpKeepalivesCount.toList
.map(i => exe(s"SET tcp_keepalives_count TO $i")),
connectionInitHook.toList,
).flatten
InitHookDataSourceProxy(pgSimpleDataSource, hookFunctions)
}
override def checkCompatibility(
connection: Connection
)(implicit loggingContext: LoggingContext): Unit = {
getPostgresVersion(connection) match {
case Some((major, minor)) =>
if (major < 10) {
logger.error(
"Deprecated Postgres version. " +
s"Found Postgres version $major.$minor, minimum required Postgres version is 10. " +
"This application will continue running but is at risk of data loss, as Postgres < 10 does not support crash-fault tolerant hash indices. " +
"Please upgrade your Postgres database to version 10 or later to fix this issue."
)
}
case None =>
logger.warn(
s"Could not determine the version of the Postgres database. Please verify that this application is compatible with this Postgres version."
)
}
()
}
private[backend] def getPostgresVersion(
connection: Connection
)(implicit loggingContext: LoggingContext): Option[(Int, Int)] = {
val version = SQL"SHOW server_version".as(get[String](1).single)(connection)
logger.debug(s"Found Postgres version $version")
parsePostgresVersion(version)
}
private[backend] def parsePostgresVersion(version: String): Option[(Int, Int)] = {
val versionPattern = """(\d+)[.](\d+).*""".r
version match {
case versionPattern(major, minor) => Some((major.toInt, minor.toInt))
case _ => None
}
}
override def tryAcquire(
lockId: DBLockStorageBackend.LockId,
lockMode: DBLockStorageBackend.LockMode,
)(connection: Connection): Option[DBLockStorageBackend.Lock] = {
val lockFunction = lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "pg_try_advisory_lock"
case DBLockStorageBackend.LockMode.Shared => "pg_try_advisory_lock_shared"
}
SQL"SELECT #$lockFunction(${pgBigintLockId(lockId)})"
.as(get[Boolean](1).single)(connection) match {
case true => Some(DBLockStorageBackend.Lock(lockId, lockMode))
case false => None
}
}
override def release(lock: DBLockStorageBackend.Lock)(connection: Connection): Boolean = {
val lockFunction = lock.lockMode match {
case DBLockStorageBackend.LockMode.Exclusive => "pg_advisory_unlock"
case DBLockStorageBackend.LockMode.Shared => "pg_advisory_unlock_shared"
}
SQL"SELECT #$lockFunction(${pgBigintLockId(lock.lockId)})"
.as(get[Boolean](1).single)(connection)
}
case class PGLockId(id: Long) extends DBLockStorageBackend.LockId
private def pgBigintLockId(lockId: DBLockStorageBackend.LockId): Long =
lockId match {
case PGLockId(id) => id
case unknown =>
throw new Exception(
s"LockId $unknown not supported. Probable cause: LockId was created by a different StorageBackend"
)
}
override def lock(id: Int): DBLockStorageBackend.LockId = PGLockId(id.toLong)
override def dbLockSupported: Boolean = true
}

View File

@ -0,0 +1,72 @@
// 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.postgresql
import com.daml.platform.store.backend.common.{
CompletionStorageBackendTemplate,
ConfigurationStorageBackendTemplate,
ContractStorageBackendTemplate,
IngestionStorageBackendTemplate,
IntegrityStorageBackendTemplate,
PackageStorageBackendTemplate,
ParameterStorageBackendTemplate,
PartyStorageBackendTemplate,
}
import com.daml.platform.store.backend.{
CompletionStorageBackend,
ConfigurationStorageBackend,
ContractStorageBackend,
DBLockStorageBackend,
DataSourceStorageBackend,
DeduplicationStorageBackend,
EventStorageBackend,
IngestionStorageBackend,
IntegrityStorageBackend,
PackageStorageBackend,
ParameterStorageBackend,
PartyStorageBackend,
ResetStorageBackend,
StorageBackendFactory,
}
object PostgresStorageBackendFactory extends StorageBackendFactory {
override val createIngestionStorageBackend: IngestionStorageBackend[_] =
new IngestionStorageBackendTemplate(PGSchema.schema)
override val createParameterStorageBackend: ParameterStorageBackend =
ParameterStorageBackendTemplate
override val createConfigurationStorageBackend: ConfigurationStorageBackend =
ConfigurationStorageBackendTemplate
override val createPartyStorageBackend: PartyStorageBackend =
new PartyStorageBackendTemplate(PostgresQueryStrategy)
override val createPackageStorageBackend: PackageStorageBackend =
PackageStorageBackendTemplate
override val createDeduplicationStorageBackend: DeduplicationStorageBackend =
PostgresDeduplicationStorageBackend
override val createCompletionStorageBackend: CompletionStorageBackend =
new CompletionStorageBackendTemplate(PostgresQueryStrategy)
override val createContractStorageBackend: ContractStorageBackend =
new ContractStorageBackendTemplate(PostgresQueryStrategy)
override val createEventStorageBackend: EventStorageBackend =
PostgresEventStorageBackend
override val createDataSourceStorageBackend: DataSourceStorageBackend =
PostgresDataSourceStorageBackend
override val createDBLockStorageBackend: DBLockStorageBackend =
PostgresDBLockStorageBackend
override val createIntegrityStorageBackend: IntegrityStorageBackend =
IntegrityStorageBackendTemplate
override val createResetStorageBackend: ResetStorageBackend =
PostgresResetStorageBackend
}

View File

@ -7,14 +7,14 @@ import com.daml.ledger.api.testing.utils.AkkaBeforeAndAfterAll
import com.daml.ledger.resources.ResourceContext import com.daml.ledger.resources.ResourceContext
import com.daml.logging.LoggingContext import com.daml.logging.LoggingContext
import com.daml.platform.store.DbType import com.daml.platform.store.DbType
import com.daml.platform.store.backend.StorageBackend import com.daml.platform.store.backend.{ParameterStorageBackend, StorageBackendFactory}
import org.scalatest.Assertion import org.scalatest.Assertion
import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually
import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.flatspec.AsyncFlatSpec
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.time.{Millis, Seconds, Span} import org.scalatest.time.{Millis, Seconds, Span}
import java.sql.Connection import java.sql.Connection
import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.{ExecutionContext, Future}
trait IndexerStabilitySpec trait IndexerStabilitySpec
@ -50,8 +50,10 @@ trait IndexerStabilitySpec
materializer, materializer,
) )
.use[Unit] { indexers => .use[Unit] { indexers =>
val storageBackend = StorageBackend.of(DbType.jdbcType(jdbcUrl)) val factory = StorageBackendFactory.of(DbType.jdbcType(jdbcUrl))
val dataSource = storageBackend.createDataSource(jdbcUrl) val dataSource = factory.createDataSourceStorageBackend.createDataSource(jdbcUrl)
val parameterStorageBackend = factory.createParameterStorageBackend
val integrityStorageBackend = factory.createIntegrityStorageBackend
val connection = dataSource.getConnection() val connection = dataSource.getConnection()
Iterator Iterator
@ -61,7 +63,7 @@ trait IndexerStabilitySpec
info(s"Indexer ${activeIndexer.readService.name} is running") info(s"Indexer ${activeIndexer.readService.name} is running")
// Assert that state updates are being indexed // Assert that state updates are being indexed
assertLedgerEndHasMoved(storageBackend, connection) assertLedgerEndHasMoved(parameterStorageBackend, connection)
info("Ledger end has moved") info("Ledger end has moved")
// At this point, the indexer that was aborted by the previous iteration can be reset, // At this point, the indexer that was aborted by the previous iteration can be reset,
@ -94,7 +96,7 @@ trait IndexerStabilitySpec
Thread.sleep(1000L) Thread.sleep(1000L)
// Verify the integrity of the index database // Verify the integrity of the index database
storageBackend.verifyIntegrity()(connection) integrityStorageBackend.verifyIntegrity()(connection)
info(s"Integrity of the index database was checked") info(s"Integrity of the index database was checked")
connection.close() connection.close()
@ -120,7 +122,7 @@ trait IndexerStabilitySpec
// Asserts that the ledger end has moved at least the specified number of events within a short time // Asserts that the ledger end has moved at least the specified number of events within a short time
private def assertLedgerEndHasMoved( private def assertLedgerEndHasMoved(
storageBackend: StorageBackend[_], parameterStorageBackend: ParameterStorageBackend,
connection: Connection, connection: Connection,
)(implicit pos: org.scalactic.source.Position): Assertion = { )(implicit pos: org.scalactic.source.Position): Assertion = {
implicit val patienceConfig: PatienceConfig = PatienceConfig( implicit val patienceConfig: PatienceConfig = PatienceConfig(
@ -129,10 +131,10 @@ trait IndexerStabilitySpec
) )
// Note: we don't know exactly at which ledger end the current indexer has started. // Note: we don't know exactly at which ledger end the current indexer has started.
// We only observe that the ledger end is moving right now. // We only observe that the ledger end is moving right now.
val initialLedgerEnd = storageBackend.ledgerEndOrBeforeBegin(connection) val initialLedgerEnd = parameterStorageBackend.ledgerEndOrBeforeBegin(connection)
val minEvents = 2L val minEvents = 2L
eventually { eventually {
val ledgerEnd = storageBackend.ledgerEndOrBeforeBegin(connection) val ledgerEnd = parameterStorageBackend.ledgerEndOrBeforeBegin(connection)
assert(ledgerEnd.lastEventSeqId > initialLedgerEnd.lastEventSeqId + minEvents) assert(ledgerEnd.lastEventSeqId > initialLedgerEnd.lastEventSeqId + minEvents)
} }
} }

View File

@ -4,9 +4,10 @@
package com.daml.platform.store.backend package com.daml.platform.store.backend
import java.sql.Connection import java.sql.Connection
import com.daml.platform.store.backend.h2.H2StorageBackend
import com.daml.platform.store.backend.oracle.OracleStorageBackend import com.daml.platform.store.backend.h2.H2StorageBackendFactory
import com.daml.platform.store.backend.postgresql.PostgresStorageBackend import com.daml.platform.store.backend.oracle.OracleStorageBackendFactory
import com.daml.platform.store.backend.postgresql.PostgresStorageBackendFactory
import com.daml.testing.oracle.OracleAroundAll import com.daml.testing.oracle.OracleAroundAll
import com.daml.testing.postgresql.PostgresAroundAll import com.daml.testing.postgresql.PostgresAroundAll
import org.scalatest.Suite import org.scalatest.Suite
@ -16,11 +17,12 @@ import org.scalatest.Suite
*/ */
private[backend] trait StorageBackendProvider { private[backend] trait StorageBackendProvider {
protected def jdbcUrl: String protected def jdbcUrl: String
protected def backend: StorageBackend[_] protected def backendFactory: StorageBackendFactory
protected final def ingest(dbDtos: Vector[DbDto], connection: Connection): Unit = { protected final def ingest(dbDtos: Vector[DbDto], connection: Connection): Unit = {
def typeBoundIngest[T](backend: StorageBackend[T]): Unit = def typeBoundIngest[T](ingestionStorageBackend: IngestionStorageBackend[T]): Unit =
backend.insertBatch(connection, backend.batch(dbDtos)) ingestionStorageBackend.insertBatch(connection, ingestionStorageBackend.batch(dbDtos))
typeBoundIngest(backend) typeBoundIngest(backendFactory.createIngestionStorageBackend)
} }
} }
@ -28,12 +30,12 @@ private[backend] trait StorageBackendProviderPostgres
extends StorageBackendProvider extends StorageBackendProvider
with PostgresAroundAll { this: Suite => with PostgresAroundAll { this: Suite =>
override protected def jdbcUrl: String = postgresDatabase.url override protected def jdbcUrl: String = postgresDatabase.url
override protected val backend: StorageBackend[_] = PostgresStorageBackend override protected val backendFactory: StorageBackendFactory = PostgresStorageBackendFactory
} }
private[backend] trait StorageBackendProviderH2 extends StorageBackendProvider { this: Suite => private[backend] trait StorageBackendProviderH2 extends StorageBackendProvider { this: Suite =>
override protected def jdbcUrl: String = "jdbc:h2:mem:storage_backend_provider;db_close_delay=-1" override protected def jdbcUrl: String = "jdbc:h2:mem:storage_backend_provider;db_close_delay=-1"
override protected val backend: StorageBackend[_] = H2StorageBackend override protected val backendFactory: StorageBackendFactory = H2StorageBackendFactory
} }
private[backend] trait StorageBackendProviderOracle private[backend] trait StorageBackendProviderOracle
@ -41,5 +43,5 @@ private[backend] trait StorageBackendProviderOracle
with OracleAroundAll { this: Suite => with OracleAroundAll { this: Suite =>
override protected def jdbcUrl: String = override protected def jdbcUrl: String =
s"jdbc:oracle:thin:$oracleUser/$oraclePwd@localhost:$oraclePort/ORCLPDB1" s"jdbc:oracle:thin:$oracleUser/$oraclePwd@localhost:$oraclePort/ORCLPDB1"
override protected val backend: StorageBackend[_] = OracleStorageBackend override protected val backendFactory: StorageBackendFactory = OracleStorageBackendFactory
} }

View File

@ -49,7 +49,7 @@ private[backend] trait StorageBackendSpec
) )
dispatcher <- DbDispatcher dispatcher <- DbDispatcher
.owner( .owner(
dataSource = backend.createDataSource(jdbcUrl), dataSource = backendFactory.createDataSourceStorageBackend.createDataSource(jdbcUrl),
serverRole = ServerRole.Testing(this.getClass), serverRole = ServerRole.Testing(this.getClass),
connectionPoolSize = connectionPoolSize, connectionPoolSize = connectionPoolSize,
connectionTimeout = FiniteDuration(250, "millis"), connectionTimeout = FiniteDuration(250, "millis"),
@ -81,7 +81,7 @@ private[backend] trait StorageBackendSpec
runningTests.incrementAndGet() == 1, runningTests.incrementAndGet() == 1,
"StorageBackendSpec tests must not run in parallel, as they all run against the same database.", "StorageBackendSpec tests must not run in parallel, as they all run against the same database.",
) )
Await.result(executeSql(backend.resetAll), 60.seconds) Await.result(executeSql(backendFactory.createResetStorageBackend.resetAll), 60.seconds)
} }
override protected def afterEach(): Unit = { override protected def afterEach(): Unit = {

View File

@ -15,6 +15,11 @@ private[backend] trait StorageBackendTestsCompletions
with StorageBackendSpec { with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val completionStorageBackend: CompletionStorageBackend =
backendFactory.createCompletionStorageBackend
behavior of "StorageBackend (completions)" behavior of "StorageBackend (completions)"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -31,20 +36,27 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(4), 3L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(4), 3L))
)
completions0to3 <- executeSql( completions0to3 <- executeSql(
backend.commandCompletions(Offset.beforeBegin, offset(3), applicationId, Set(party)) completionStorageBackend.commandCompletions(
Offset.beforeBegin,
offset(3),
applicationId,
Set(party),
)
) )
completions1to3 <- executeSql( completions1to3 <- executeSql(
backend.commandCompletions(offset(1), offset(3), applicationId, Set(party)) completionStorageBackend.commandCompletions(offset(1), offset(3), applicationId, Set(party))
) )
completions2to3 <- executeSql( completions2to3 <- executeSql(
backend.commandCompletions(offset(2), offset(3), applicationId, Set(party)) completionStorageBackend.commandCompletions(offset(2), offset(3), applicationId, Set(party))
) )
completions1to9 <- executeSql( completions1to9 <- executeSql(
backend.commandCompletions(offset(1), offset(9), applicationId, Set(party)) completionStorageBackend.commandCompletions(offset(1), offset(9), applicationId, Set(party))
) )
} yield { } yield {
completions0to3 should have length 2 completions0to3 should have length 2
@ -64,11 +76,13 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))
)
completions <- executeSql( completions <- executeSql(
backend.commandCompletions(offset(1), offset(2), applicationId, Set(party)) completionStorageBackend.commandCompletions(offset(1), offset(2), applicationId, Set(party))
) )
} yield { } yield {
completions should have length 1 completions should have length 1
@ -88,11 +102,18 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))
)
completions <- executeSql( completions <- executeSql(
backend.commandCompletions(offset(1), offset(3), someApplicationId, Set(party)) completionStorageBackend.commandCompletions(
offset(1),
offset(3),
someApplicationId,
Set(party),
)
) )
} yield { } yield {
completions should have length 2 completions should have length 2
@ -119,11 +140,18 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))
)
completions <- executeSql( completions <- executeSql(
backend.commandCompletions(offset(1), offset(3), someApplicationId, Set(party)) completionStorageBackend.commandCompletions(
offset(1),
offset(3),
someApplicationId,
Set(party),
)
) )
} yield { } yield {
completions should have length 2 completions should have length 2
@ -161,11 +189,18 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))
)
completions <- executeSql( completions <- executeSql(
backend.commandCompletions(offset(1), offset(3), someApplicationId, Set(party)) completionStorageBackend.commandCompletions(
offset(1),
offset(3),
someApplicationId,
Set(party),
)
) )
} yield { } yield {
completions should have length 2 completions should have length 2
@ -200,11 +235,18 @@ private[backend] trait StorageBackendTestsCompletions
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos1, _)) _ <- executeSql(ingest(dtos1, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))
)
result <- executeSql( result <- executeSql(
backend.commandCompletions(offset(1), offset(2), someApplicationId, Set(party)) completionStorageBackend.commandCompletions(
offset(1),
offset(2),
someApplicationId,
Set(party),
)
).failed ).failed
} yield { } yield {
result shouldBe an[IllegalArgumentException] result shouldBe an[IllegalArgumentException]
@ -222,9 +264,16 @@ private[backend] trait StorageBackendTestsCompletions
for { for {
_ <- executeSql(ingest(dtos2, _)) _ <- executeSql(ingest(dtos2, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 2L))
)
result <- executeSql( result <- executeSql(
backend.commandCompletions(offset(2), offset(3), someApplicationId, Set(party)) completionStorageBackend.commandCompletions(
offset(2),
offset(3),
someApplicationId,
Set(party),
)
).failed ).failed
} yield { } yield {
result shouldBe an[IllegalArgumentException] result shouldBe an[IllegalArgumentException]

View File

@ -172,8 +172,10 @@ trait StorageBackendTestsDBLockForSuite
with StorageBackendProvider { with StorageBackendProvider {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
override def dbLock: DBLockStorageBackend = backend override val dbLock: DBLockStorageBackend = backendFactory.createDBLockStorageBackend
override def getConnection: Connection = override def getConnection: Connection =
backend.createDataSource(jdbcUrl)(LoggingContext.ForTesting).getConnection backendFactory.createDataSourceStorageBackend
.createDataSource(jdbcUrl)(LoggingContext.ForTesting)
.getConnection
} }

View File

@ -16,6 +16,11 @@ private[backend] trait StorageBackendTestsDeduplication
with StorageBackendSpec { with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val deduplicationStorageBackend: DeduplicationStorageBackend =
backendFactory.createDeduplicationStorageBackend
behavior of "DeduplicationStorageBackend" behavior of "DeduplicationStorageBackend"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -27,13 +32,15 @@ private[backend] trait StorageBackendTestsDeduplication
val n = 8 val n = 8
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
insertedRows <- Future.sequence( insertedRows <- Future.sequence(
Vector.fill(n)( Vector.fill(n)(
executeSql(backend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil)) executeSql(
deduplicationStorageBackend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil)
)
) )
) )
foundDeduplicateUntil <- executeSql(backend.deduplicatedUntil(key)) foundDeduplicateUntil <- executeSql(deduplicationStorageBackend.deduplicatedUntil(key))
} yield { } yield {
insertedRows.count(_ == 1) shouldBe 1 // One of the calls inserts a new row insertedRows.count(_ == 1) shouldBe 1 // One of the calls inserts a new row
insertedRows.count(_ == 0) shouldBe (n - 1) // All other calls don't write anything insertedRows.count(_ == 0) shouldBe (n - 1) // All other calls don't write anything
@ -52,17 +59,23 @@ private[backend] trait StorageBackendTestsDeduplication
val n = 8 val n = 8
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
insertedRows <- executeSql( insertedRows <- executeSql(
backend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil) deduplicationStorageBackend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil)
) )
foundDeduplicateUntil <- executeSql(backend.deduplicatedUntil(key)) foundDeduplicateUntil <- executeSql(deduplicationStorageBackend.deduplicatedUntil(key))
updatedRows <- Future.sequence( updatedRows <- Future.sequence(
Vector.fill(n)( Vector.fill(n)(
executeSql(backend.upsertDeduplicationEntry(key, submittedAt2, deduplicateUntil2)) executeSql(
deduplicationStorageBackend.upsertDeduplicationEntry(
key,
submittedAt2,
deduplicateUntil2,
)
)
) )
) )
foundDeduplicateUntil2 <- executeSql(backend.deduplicatedUntil(key)) foundDeduplicateUntil2 <- executeSql(deduplicationStorageBackend.deduplicatedUntil(key))
} yield { } yield {
insertedRows shouldBe 1 // First call inserts a new row insertedRows shouldBe 1 // First call inserts a new row
updatedRows.count( updatedRows.count(
@ -84,15 +97,15 @@ private[backend] trait StorageBackendTestsDeduplication
val deduplicateUntil2 = submittedAt2.addMicros(5000L) val deduplicateUntil2 = submittedAt2.addMicros(5000L)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
insertedRows <- executeSql( insertedRows <- executeSql(
backend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil) deduplicationStorageBackend.upsertDeduplicationEntry(key, submittedAt, deduplicateUntil)
) )
foundDeduplicateUntil <- executeSql(backend.deduplicatedUntil(key)) foundDeduplicateUntil <- executeSql(deduplicationStorageBackend.deduplicatedUntil(key))
updatedRows <- executeSql( updatedRows <- executeSql(
backend.upsertDeduplicationEntry(key, submittedAt2, deduplicateUntil2) deduplicationStorageBackend.upsertDeduplicationEntry(key, submittedAt2, deduplicateUntil2)
) )
foundDeduplicateUntil2 <- executeSql(backend.deduplicatedUntil(key)) foundDeduplicateUntil2 <- executeSql(deduplicationStorageBackend.deduplicatedUntil(key))
} yield { } yield {
insertedRows shouldBe 1 // First call inserts a new row insertedRows shouldBe 1 // First call inserts a new row
updatedRows shouldBe 0 // Second call doesn't write anything updatedRows shouldBe 0 // Second call doesn't write anything

View File

@ -13,6 +13,14 @@ private[backend] trait StorageBackendTestsIngestion
with StorageBackendSpec { with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val configurationStorageBackend: ConfigurationStorageBackend =
backendFactory.createConfigurationStorageBackend
private val partyStorageBackend: PartyStorageBackend = backendFactory.createPartyStorageBackend
private val packageStorageBackend: PackageStorageBackend =
backendFactory.createPackageStorageBackend
behavior of "StorageBackend (ingestion)" behavior of "StorageBackend (ingestion)"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -24,11 +32,13 @@ private[backend] trait StorageBackendTestsIngestion
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
configBeforeLedgerEndUpdate <- executeSql(backend.ledgerConfiguration) configBeforeLedgerEndUpdate <- executeSql(configurationStorageBackend.ledgerConfiguration)
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))) _ <- executeSql(
configAfterLedgerEndUpdate <- executeSql(backend.ledgerConfiguration) parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))
)
configAfterLedgerEndUpdate <- executeSql(configurationStorageBackend.ledgerConfiguration)
} yield { } yield {
// The first query is executed before the ledger end is updated. // The first query is executed before the ledger end is updated.
// It should not see the already ingested configuration change. // It should not see the already ingested configuration change.
@ -51,11 +61,13 @@ private[backend] trait StorageBackendTestsIngestion
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
packagesBeforeLedgerEndUpdate <- executeSql(backend.lfPackages) packagesBeforeLedgerEndUpdate <- executeSql(packageStorageBackend.lfPackages)
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))) _ <- executeSql(
packagesAfterLedgerEndUpdate <- executeSql(backend.lfPackages) parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))
)
packagesAfterLedgerEndUpdate <- executeSql(packageStorageBackend.lfPackages)
} yield { } yield {
// The first query is executed before the ledger end is updated. // The first query is executed before the ledger end is updated.
// It should not see the already ingested package upload. // It should not see the already ingested package upload.
@ -73,11 +85,13 @@ private[backend] trait StorageBackendTestsIngestion
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
partiesBeforeLedgerEndUpdate <- executeSql(backend.knownParties) partiesBeforeLedgerEndUpdate <- executeSql(partyStorageBackend.knownParties)
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))) _ <- executeSql(
partiesAfterLedgerEndUpdate <- executeSql(backend.knownParties) parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(someOffset, 0))
)
partiesAfterLedgerEndUpdate <- executeSql(partyStorageBackend.knownParties)
} yield { } yield {
// The first query is executed before the ledger end is updated. // The first query is executed before the ledger end is updated.
// It should not see the already ingested party allocation. // It should not see the already ingested party allocation.

View File

@ -12,6 +12,9 @@ import org.scalatest.matchers.should.Matchers
private[backend] trait StorageBackendTestsInitialization extends Matchers with StorageBackendSpec { private[backend] trait StorageBackendTestsInitialization extends Matchers with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
behavior of "StorageBackend (initialization)" behavior of "StorageBackend (initialization)"
it should "correctly handle repeated initialization" in { it should "correctly handle repeated initialization" in {
@ -22,7 +25,7 @@ private[backend] trait StorageBackendTestsInitialization extends Matchers with S
for { for {
_ <- executeSql( _ <- executeSql(
backend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = ledgerId, ledgerId = ledgerId,
participantId = participantId, participantId = participantId,
@ -30,7 +33,7 @@ private[backend] trait StorageBackendTestsInitialization extends Matchers with S
) )
) )
error1 <- executeSql( error1 <- executeSql(
backend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = otherLedgerId, ledgerId = otherLedgerId,
participantId = participantId, participantId = participantId,
@ -38,7 +41,7 @@ private[backend] trait StorageBackendTestsInitialization extends Matchers with S
) )
).failed ).failed
error2 <- executeSql( error2 <- executeSql(
backend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = ledgerId, ledgerId = ledgerId,
participantId = otherParticipantId, participantId = otherParticipantId,
@ -46,7 +49,7 @@ private[backend] trait StorageBackendTestsInitialization extends Matchers with S
) )
).failed ).failed
error3 <- executeSql( error3 <- executeSql(
backend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = otherLedgerId, ledgerId = otherLedgerId,
participantId = otherParticipantId, participantId = otherParticipantId,
@ -54,7 +57,7 @@ private[backend] trait StorageBackendTestsInitialization extends Matchers with S
) )
).failed ).failed
_ <- executeSql( _ <- executeSql(
backend.initializeParameters( parameterStorageBackend.initializeParameters(
ParameterStorageBackend.IdentityParams( ParameterStorageBackend.IdentityParams(
ledgerId = ledgerId, ledgerId = ledgerId,
participantId = participantId, participantId = participantId,

View File

@ -15,6 +15,18 @@ private[backend] trait StorageBackendTestsInitializeIngestion
with StorageBackendSpec { with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val configurationStorageBackend: ConfigurationStorageBackend =
backendFactory.createConfigurationStorageBackend
private val partyStorageBackend: PartyStorageBackend = backendFactory.createPartyStorageBackend
private val packageStorageBackend: PackageStorageBackend =
backendFactory.createPackageStorageBackend
private val ingestionStorageBackend: IngestionStorageBackend[_] =
backendFactory.createIngestionStorageBackend
private val contractStorageBackend: ContractStorageBackend =
backendFactory.createContractStorageBackend
behavior of "StorageBackend (initializeIngestion)" behavior of "StorageBackend (initializeIngestion)"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -59,46 +71,58 @@ private[backend] trait StorageBackendTestsInitializeIngestion
for { for {
// Initialize // Initialize
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Start the indexer (a no-op in this case) // Start the indexer (a no-op in this case)
end1 <- executeSql(backend.ledgerEnd) end1 <- executeSql(parameterStorageBackend.ledgerEnd)
_ <- executeSql(backend.deletePartiallyIngestedData(end1)) _ <- executeSql(ingestionStorageBackend.deletePartiallyIngestedData(end1))
// Fully insert first batch of updates // Fully insert first batch of updates
_ <- executeSql(ingest(dtos1, _)) _ <- executeSql(ingest(dtos1, _))
_ <- executeSql(backend.updateLedgerEnd(ledgerEnd(5, 3L))) _ <- executeSql(parameterStorageBackend.updateLedgerEnd(ledgerEnd(5, 3L)))
// Partially insert second batch of updates (indexer crashes before updating ledger end) // Partially insert second batch of updates (indexer crashes before updating ledger end)
_ <- executeSql(ingest(dtos2, _)) _ <- executeSql(ingest(dtos2, _))
// Check the contents // Check the contents
parties1 <- executeSql(backend.knownParties) parties1 <- executeSql(partyStorageBackend.knownParties)
config1 <- executeSql(backend.ledgerConfiguration) config1 <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages1 <- executeSql(backend.lfPackages) packages1 <- executeSql(packageStorageBackend.lfPackages)
contract41 <- executeSql( contract41 <- executeSql(
backend.activeContractWithoutArgument(readers, ContractId.V0.assertFromString("#4")) contractStorageBackend.activeContractWithoutArgument(
readers,
ContractId.V0.assertFromString("#4"),
)
) )
contract91 <- executeSql( contract91 <- executeSql(
backend.activeContractWithoutArgument(readers, ContractId.V0.assertFromString("#9")) contractStorageBackend.activeContractWithoutArgument(
readers,
ContractId.V0.assertFromString("#9"),
)
) )
// Restart the indexer - should delete data from the partial insert above // Restart the indexer - should delete data from the partial insert above
end2 <- executeSql(backend.ledgerEnd) end2 <- executeSql(parameterStorageBackend.ledgerEnd)
_ <- executeSql(backend.deletePartiallyIngestedData(end2)) _ <- executeSql(ingestionStorageBackend.deletePartiallyIngestedData(end2))
// Move the ledger end so that any non-deleted data would become visible // Move the ledger end so that any non-deleted data would become visible
_ <- executeSql(backend.updateLedgerEnd(ledgerEnd(10, 6L))) _ <- executeSql(parameterStorageBackend.updateLedgerEnd(ledgerEnd(10, 6L)))
// Check the contents // Check the contents
parties2 <- executeSql(backend.knownParties) parties2 <- executeSql(partyStorageBackend.knownParties)
config2 <- executeSql(backend.ledgerConfiguration) config2 <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages2 <- executeSql(backend.lfPackages) packages2 <- executeSql(packageStorageBackend.lfPackages)
contract42 <- executeSql( contract42 <- executeSql(
backend.activeContractWithoutArgument(readers, ContractId.V0.assertFromString("#4")) contractStorageBackend.activeContractWithoutArgument(
readers,
ContractId.V0.assertFromString("#4"),
)
) )
contract92 <- executeSql( contract92 <- executeSql(
backend.activeContractWithoutArgument(readers, ContractId.V0.assertFromString("#9")) contractStorageBackend.activeContractWithoutArgument(
readers,
ContractId.V0.assertFromString("#9"),
)
) )
} yield { } yield {
parties1 should have length 1 parties1 should have length 1

View File

@ -9,6 +9,11 @@ import org.scalatest.matchers.should.Matchers
private[backend] trait StorageBackendTestsIntegrity extends Matchers with StorageBackendSpec { private[backend] trait StorageBackendTestsIntegrity extends Matchers with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val integrityStorageBackend: IntegrityStorageBackend =
backendFactory.createIntegrityStorageBackend
import StorageBackendTestValues._ import StorageBackendTestValues._
behavior of "IntegrityStorageBackend" behavior of "IntegrityStorageBackend"
@ -20,10 +25,12 @@ private[backend] trait StorageBackendTestsIntegrity extends Matchers with Storag
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(updates, _)) _ <- executeSql(ingest(updates, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(7), 7L))) _ <- executeSql(
failure <- executeSql(backend.verifyIntegrity()).failed parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(7), 7L))
)
failure <- executeSql(integrityStorageBackend.verifyIntegrity()).failed
} yield { } yield {
// Error message should contain the duplicate event sequential id // Error message should contain the duplicate event sequential id
failure.getMessage should include("7") failure.getMessage should include("7")
@ -37,10 +44,12 @@ private[backend] trait StorageBackendTestsIntegrity extends Matchers with Storag
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(updates, _)) _ <- executeSql(ingest(updates, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 3L))) _ <- executeSql(
failure <- executeSql(backend.verifyIntegrity()).failed parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(3), 3L))
)
failure <- executeSql(integrityStorageBackend.verifyIntegrity()).failed
} yield { } yield {
failure.getMessage should include("consecutive") failure.getMessage should include("consecutive")
} }
@ -56,10 +65,12 @@ private[backend] trait StorageBackendTestsIntegrity extends Matchers with Storag
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(updates, _)) _ <- executeSql(ingest(updates, _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 2L))) _ <- executeSql(
_ <- executeSql(backend.verifyIntegrity()) parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 2L))
)
_ <- executeSql(integrityStorageBackend.verifyIntegrity())
} yield { } yield {
succeed succeed
} }

View File

@ -17,6 +17,12 @@ private[backend] trait StorageBackendTestsMigrationPruning
with StorageBackendSpec { with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val contractStorageBackend: ContractStorageBackend =
backendFactory.createContractStorageBackend
private val eventStorageBackend: EventStorageBackend = backendFactory.createEventStorageBackend
import StorageBackendTestValues._ import StorageBackendTestValues._
it should "prune all divulgence events if pruning offset is after migration offset" in { it should "prune all divulgence events if pruning offset is after migration offset" in {
@ -28,20 +34,25 @@ private[backend] trait StorageBackendTestsMigrationPruning
val archive = dtoExercise(offset(2), 3L, consuming = true, "#1", submitter) val archive = dtoExercise(offset(2), 3L, consuming = true, "#1", submitter)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(Vector(create, divulgence, archive), _)) _ <- executeSql(ingest(Vector(create, divulgence, archive), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 3L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 3L))
)
// Simulate that the archive happened after the migration to append-only schema // Simulate that the archive happened after the migration to append-only schema
_ <- executeSql(updateMigrationHistoryTable(ledgerSequentialIdBefore = 2)) _ <- executeSql(updateMigrationHistoryTable(ledgerSequentialIdBefore = 2))
beforePruning <- executeSql( beforePruning <- executeSql(
backend.activeContractWithoutArgument(Set(divulgee), ContractId.assertFromString("#1")) contractStorageBackend.activeContractWithoutArgument(
Set(divulgee),
ContractId.assertFromString("#1"),
)
) )
// Check that the divulgee can fetch the divulged event // Check that the divulgee can fetch the divulged event
_ <- Future.successful(beforePruning should not be empty) _ <- Future.successful(beforePruning should not be empty)
// Trying to prune all divulged contracts before the migration should fail // Trying to prune all divulged contracts before the migration should fail
_ <- _ <-
executeSql( executeSql(
backend.isPruningOffsetValidAgainstMigration( eventStorageBackend.isPruningOffsetValidAgainstMigration(
offset(1), offset(1),
pruneAllDivulgedContracts = true, pruneAllDivulgedContracts = true,
_, _,
@ -49,18 +60,24 @@ private[backend] trait StorageBackendTestsMigrationPruning
).map(_ shouldBe false) ).map(_ shouldBe false)
// Validation passes the pruning offset for all divulged contracts is after the migration // Validation passes the pruning offset for all divulged contracts is after the migration
_ <- executeSql( _ <- executeSql(
backend.isPruningOffsetValidAgainstMigration( eventStorageBackend.isPruningOffsetValidAgainstMigration(
offset(2), offset(2),
pruneAllDivulgedContracts = true, pruneAllDivulgedContracts = true,
_, _,
) )
).map(_ shouldBe true) ).map(_ shouldBe true)
_ <- executeSql( _ <- executeSql(
backend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(_, loggingContext) eventStorageBackend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(
_,
loggingContext,
)
) )
// Ensure the divulged contract is not visible anymore // Ensure the divulged contract is not visible anymore
afterPruning <- executeSql( afterPruning <- executeSql(
backend.activeContractWithoutArgument(Set(divulgee), ContractId.assertFromString("#1")) contractStorageBackend.activeContractWithoutArgument(
Set(divulgee),
ContractId.assertFromString("#1"),
)
) )
} yield { } yield {
// Pruning succeeded // Pruning succeeded

View File

@ -12,6 +12,14 @@ import org.scalatest.matchers.should.Matchers
private[backend] trait StorageBackendTestsPruning extends Matchers with StorageBackendSpec { private[backend] trait StorageBackendTestsPruning extends Matchers with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val contractStorageBackend: ContractStorageBackend =
backendFactory.createContractStorageBackend
private val eventStorageBackend: EventStorageBackend = backendFactory.createEventStorageBackend
private val completionStorageBackend: CompletionStorageBackend =
backendFactory.createCompletionStorageBackend
behavior of "StorageBackend (pruning)" behavior of "StorageBackend (pruning)"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -21,17 +29,17 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
val offset_2 = offset(2) val offset_2 = offset(2)
val offset_3 = offset(4) val offset_3 = offset(4)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
initialPruningOffset <- executeSql(backend.prunedUpToInclusive) initialPruningOffset <- executeSql(parameterStorageBackend.prunedUpToInclusive)
_ <- executeSql(backend.updatePrunedUptoInclusive(offset_1)) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset_1))
updatedPruningOffset_1 <- executeSql(backend.prunedUpToInclusive) updatedPruningOffset_1 <- executeSql(parameterStorageBackend.prunedUpToInclusive)
_ <- executeSql(backend.updatePrunedUptoInclusive(offset_2)) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset_2))
updatedPruningOffset_2 <- executeSql(backend.prunedUpToInclusive) updatedPruningOffset_2 <- executeSql(parameterStorageBackend.prunedUpToInclusive)
_ <- executeSql(backend.updatePrunedUptoInclusive(offset_3)) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset_3))
updatedPruningOffset_3 <- executeSql(backend.prunedUpToInclusive) updatedPruningOffset_3 <- executeSql(parameterStorageBackend.prunedUpToInclusive)
} yield { } yield {
initialPruningOffset shouldBe empty initialPruningOffset shouldBe empty
updatedPruningOffset_1 shouldBe Some(offset_1) updatedPruningOffset_1 shouldBe Some(offset_1)
@ -46,22 +54,30 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
val offset_2 = offset(2) val offset_2 = offset(2)
val offset_3 = offset(4) val offset_3 = offset(4)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
initialPruningOffset <- executeSql(backend.participantAllDivulgedContractsPrunedUpToInclusive) initialPruningOffset <- executeSql(
parameterStorageBackend.participantAllDivulgedContractsPrunedUpToInclusive
)
_ <- executeSql(backend.updatePrunedAllDivulgedContractsUpToInclusive(offset_1)) _ <- executeSql(
parameterStorageBackend.updatePrunedAllDivulgedContractsUpToInclusive(offset_1)
)
updatedPruningOffset_1 <- executeSql( updatedPruningOffset_1 <- executeSql(
backend.participantAllDivulgedContractsPrunedUpToInclusive parameterStorageBackend.participantAllDivulgedContractsPrunedUpToInclusive
) )
_ <- executeSql(backend.updatePrunedAllDivulgedContractsUpToInclusive(offset_2)) _ <- executeSql(
parameterStorageBackend.updatePrunedAllDivulgedContractsUpToInclusive(offset_2)
)
updatedPruningOffset_2 <- executeSql( updatedPruningOffset_2 <- executeSql(
backend.participantAllDivulgedContractsPrunedUpToInclusive parameterStorageBackend.participantAllDivulgedContractsPrunedUpToInclusive
) )
_ <- executeSql(backend.updatePrunedAllDivulgedContractsUpToInclusive(offset_3)) _ <- executeSql(
parameterStorageBackend.updatePrunedAllDivulgedContractsUpToInclusive(offset_3)
)
updatedPruningOffset_3 <- executeSql( updatedPruningOffset_3 <- executeSql(
backend.participantAllDivulgedContractsPrunedUpToInclusive parameterStorageBackend.participantAllDivulgedContractsPrunedUpToInclusive
) )
} yield { } yield {
initialPruningOffset shouldBe empty initialPruningOffset shouldBe empty
@ -91,29 +107,34 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
val range = RangeParams(0L, 2L, None, None) val range = RangeParams(0L, 2L, None, None)
val filter = FilterParams(Set(someParty), Set.empty) val filter = FilterParams(Set(someParty), Set.empty)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Ingest a create and archive event // Ingest a create and archive event
_ <- executeSql(ingest(Vector(create, archive), _)) _ <- executeSql(ingest(Vector(create, archive), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 2L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 2L))
)
// Make sure the events are visible // Make sure the events are visible
before1 <- executeSql(backend.transactionEvents(range, filter)) before1 <- executeSql(eventStorageBackend.transactionEvents(range, filter))
before2 <- executeSql(backend.activeContractEvents(range, filter, offset(1))) before2 <- executeSql(eventStorageBackend.activeContractEvents(range, filter, offset(1)))
before3 <- executeSql(backend.flatTransaction(createTransactionId, filter)) before3 <- executeSql(eventStorageBackend.flatTransaction(createTransactionId, filter))
before4 <- executeSql(backend.transactionTreeEvents(range, filter)) before4 <- executeSql(eventStorageBackend.transactionTreeEvents(range, filter))
before5 <- executeSql(backend.transactionTree(createTransactionId, filter)) before5 <- executeSql(eventStorageBackend.transactionTree(createTransactionId, filter))
before6 <- executeSql(backend.rawEvents(0, 2L)) before6 <- executeSql(eventStorageBackend.rawEvents(0, 2L))
// Prune // Prune
_ <- executeSql( _ <- executeSql(
backend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(_, loggingContext) eventStorageBackend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(
_,
loggingContext,
)
) )
_ <- executeSql(backend.updatePrunedUptoInclusive(offset(2))) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset(2)))
// Make sure the events are not visible anymore // Make sure the events are not visible anymore
after1 <- executeSql(backend.transactionEvents(range, filter)) after1 <- executeSql(eventStorageBackend.transactionEvents(range, filter))
after2 <- executeSql(backend.activeContractEvents(range, filter, offset(1))) after2 <- executeSql(eventStorageBackend.activeContractEvents(range, filter, offset(1)))
after3 <- executeSql(backend.flatTransaction(createTransactionId, filter)) after3 <- executeSql(eventStorageBackend.flatTransaction(createTransactionId, filter))
after4 <- executeSql(backend.transactionTreeEvents(range, filter)) after4 <- executeSql(eventStorageBackend.transactionTreeEvents(range, filter))
after5 <- executeSql(backend.transactionTree(createTransactionId, filter)) after5 <- executeSql(eventStorageBackend.transactionTree(createTransactionId, filter))
after6 <- executeSql(backend.rawEvents(0, 2L)) after6 <- executeSql(eventStorageBackend.rawEvents(0, 2L))
} yield { } yield {
before1 should not be empty before1 should not be empty
before2 should not be empty before2 should not be empty
@ -145,29 +166,34 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
val range = RangeParams(0L, 1L, None, None) val range = RangeParams(0L, 1L, None, None)
val filter = FilterParams(Set(someParty), Set.empty) val filter = FilterParams(Set(someParty), Set.empty)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Ingest a create and archive event // Ingest a create and archive event
_ <- executeSql(ingest(Vector(partyEntry, create), _)) _ <- executeSql(ingest(Vector(partyEntry, create), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(2), 1L))
)
// Make sure the events are visible // Make sure the events are visible
before1 <- executeSql(backend.transactionEvents(range, filter)) before1 <- executeSql(eventStorageBackend.transactionEvents(range, filter))
before2 <- executeSql(backend.activeContractEvents(range, filter, offset(2))) before2 <- executeSql(eventStorageBackend.activeContractEvents(range, filter, offset(2)))
before3 <- executeSql(backend.flatTransaction(createTransactionId, filter)) before3 <- executeSql(eventStorageBackend.flatTransaction(createTransactionId, filter))
before4 <- executeSql(backend.transactionTreeEvents(range, filter)) before4 <- executeSql(eventStorageBackend.transactionTreeEvents(range, filter))
before5 <- executeSql(backend.transactionTree(createTransactionId, filter)) before5 <- executeSql(eventStorageBackend.transactionTree(createTransactionId, filter))
before6 <- executeSql(backend.rawEvents(0, 1L)) before6 <- executeSql(eventStorageBackend.rawEvents(0, 1L))
// Prune // Prune
_ <- executeSql( _ <- executeSql(
backend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(_, loggingContext) eventStorageBackend.pruneEvents(offset(2), pruneAllDivulgedContracts = true)(
_,
loggingContext,
)
) )
_ <- executeSql(backend.updatePrunedUptoInclusive(offset(2))) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset(2)))
// Make sure the events are still visible - active contracts should not be pruned // Make sure the events are still visible - active contracts should not be pruned
after1 <- executeSql(backend.transactionEvents(range, filter)) after1 <- executeSql(eventStorageBackend.transactionEvents(range, filter))
after2 <- executeSql(backend.activeContractEvents(range, filter, offset(2))) after2 <- executeSql(eventStorageBackend.activeContractEvents(range, filter, offset(2)))
after3 <- executeSql(backend.flatTransaction(createTransactionId, filter)) after3 <- executeSql(eventStorageBackend.flatTransaction(createTransactionId, filter))
after4 <- executeSql(backend.transactionTreeEvents(range, filter)) after4 <- executeSql(eventStorageBackend.transactionTreeEvents(range, filter))
after5 <- executeSql(backend.transactionTree(createTransactionId, filter)) after5 <- executeSql(eventStorageBackend.transactionTree(createTransactionId, filter))
after6 <- executeSql(backend.rawEvents(0, 1L)) after6 <- executeSql(eventStorageBackend.rawEvents(0, 1L))
} yield { } yield {
before1 should not be empty before1 should not be empty
before2 should not be empty before2 should not be empty
@ -214,7 +240,7 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Ingest // Ingest
_ <- executeSql( _ <- executeSql(
ingest( ingest(
@ -227,30 +253,35 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
_, _,
) )
) )
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(4), 4L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(4), 4L))
)
contract1_beforePruning <- executeSql( contract1_beforePruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract1_id), ContractId.assertFromString(contract1_id),
) )
) )
contract2_beforePruning <- executeSql( contract2_beforePruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract2_id), ContractId.assertFromString(contract2_id),
) )
) )
_ <- executeSql( _ <- executeSql(
backend.pruneEvents(offset(3), pruneAllDivulgedContracts = true)(_, loggingContext) eventStorageBackend.pruneEvents(offset(3), pruneAllDivulgedContracts = true)(
_,
loggingContext,
)
) )
contract1_afterPruning <- executeSql( contract1_afterPruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract1_id), ContractId.assertFromString(contract1_id),
) )
) )
contract2_afterPruning <- executeSql( contract2_afterPruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract2_id), ContractId.assertFromString(contract2_id),
) )
@ -298,7 +329,7 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Ingest // Ingest
_ <- executeSql( _ <- executeSql(
ingest( ingest(
@ -312,30 +343,35 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
) )
) )
// Set the ledger end past the last ingested event so we can prune up to it inclusively // Set the ledger end past the last ingested event so we can prune up to it inclusively
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(5), 5L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(5), 5L))
)
contract1_beforePruning <- executeSql( contract1_beforePruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract1_id), ContractId.assertFromString(contract1_id),
) )
) )
contract2_beforePruning <- executeSql( contract2_beforePruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract2_id), ContractId.assertFromString(contract2_id),
) )
) )
_ <- executeSql( _ <- executeSql(
backend.pruneEvents(offset(4), pruneAllDivulgedContracts = false)(_, loggingContext) eventStorageBackend.pruneEvents(offset(4), pruneAllDivulgedContracts = false)(
_,
loggingContext,
)
) )
contract1_afterPruning <- executeSql( contract1_afterPruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract1_id), ContractId.assertFromString(contract1_id),
) )
) )
contract2_afterPruning <- executeSql( contract2_afterPruning <- executeSql(
backend.activeContractWithoutArgument( contractStorageBackend.activeContractWithoutArgument(
Set(divulgee), Set(divulgee),
ContractId.assertFromString(contract2_id), ContractId.assertFromString(contract2_id),
) )
@ -362,20 +398,32 @@ private[backend] trait StorageBackendTestsPruning extends Matchers with StorageB
) )
val applicationId = dtoApplicationId(completion) val applicationId = dtoApplicationId(completion)
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
// Ingest a completion // Ingest a completion
_ <- executeSql(ingest(Vector(completion), _)) _ <- executeSql(ingest(Vector(completion), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))
)
// Make sure the completion is visible // Make sure the completion is visible
before <- executeSql( before <- executeSql(
backend.commandCompletions(offset(0), offset(1), applicationId, Set(someParty)) completionStorageBackend.commandCompletions(
offset(0),
offset(1),
applicationId,
Set(someParty),
)
) )
// Prune // Prune
_ <- executeSql(backend.pruneCompletions(offset(1))(_, loggingContext)) _ <- executeSql(completionStorageBackend.pruneCompletions(offset(1))(_, loggingContext))
_ <- executeSql(backend.updatePrunedUptoInclusive(offset(1))) _ <- executeSql(parameterStorageBackend.updatePrunedUptoInclusive(offset(1)))
// Make sure the completion is not visible anymore // Make sure the completion is not visible anymore
after <- executeSql( after <- executeSql(
backend.commandCompletions(offset(0), offset(1), applicationId, Set(someParty)) completionStorageBackend.commandCompletions(
offset(0),
offset(1),
applicationId,
Set(someParty),
)
) )
} yield { } yield {
before should not be empty before should not be empty

View File

@ -11,18 +11,29 @@ import scala.concurrent.Future
private[backend] trait StorageBackendTestsReset extends Matchers with StorageBackendSpec { private[backend] trait StorageBackendTestsReset extends Matchers with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val configurationStorageBackend: ConfigurationStorageBackend =
backendFactory.createConfigurationStorageBackend
private val partyStorageBackend: PartyStorageBackend = backendFactory.createPartyStorageBackend
private val packageStorageBackend: PackageStorageBackend =
backendFactory.createPackageStorageBackend
private val contractStorageBackend: ContractStorageBackend =
backendFactory.createContractStorageBackend
private val resetStorageBackend: ResetStorageBackend = backendFactory.createResetStorageBackend
behavior of "StorageBackend (reset)" behavior of "StorageBackend (reset)"
import StorageBackendTestValues._ import StorageBackendTestValues._
it should "start with an empty index" in { it should "start with an empty index" in {
for { for {
identity <- executeSql(backend.ledgerIdentity) identity <- executeSql(parameterStorageBackend.ledgerIdentity)
end <- executeSql(backend.ledgerEnd) end <- executeSql(parameterStorageBackend.ledgerEnd)
parties <- executeSql(backend.knownParties) parties <- executeSql(partyStorageBackend.knownParties)
config <- executeSql(backend.ledgerConfiguration) config <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages <- executeSql(backend.lfPackages) packages <- executeSql(packageStorageBackend.lfPackages)
events <- executeSql(backend.contractStateEvents(0, Long.MaxValue)) events <- executeSql(contractStorageBackend.contractStateEvents(0, Long.MaxValue))
} yield { } yield {
identity shouldBe None identity shouldBe None
end shouldBe None end shouldBe None
@ -36,9 +47,9 @@ private[backend] trait StorageBackendTestsReset extends Matchers with StorageBac
it should "not see any data after advancing the ledger end" in { it should "not see any data after advancing the ledger end" in {
for { for {
_ <- advanceLedgerEndToMakeOldDataVisible() _ <- advanceLedgerEndToMakeOldDataVisible()
parties <- executeSql(backend.knownParties) parties <- executeSql(partyStorageBackend.knownParties)
config <- executeSql(backend.ledgerConfiguration) config <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages <- executeSql(backend.lfPackages) packages <- executeSql(packageStorageBackend.lfPackages)
} yield { } yield {
parties shouldBe empty parties shouldBe empty
packages shouldBe empty packages shouldBe empty
@ -66,23 +77,23 @@ private[backend] trait StorageBackendTestsReset extends Matchers with StorageBac
for { for {
// Initialize and insert some data // Initialize and insert some data
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ledgerEnd(5, 3L))) _ <- executeSql(parameterStorageBackend.updateLedgerEnd(ledgerEnd(5, 3L)))
// Reset // Reset
_ <- executeSql(backend.reset) _ <- executeSql(resetStorageBackend.reset)
// Check the contents // Check the contents
identity <- executeSql(backend.ledgerIdentity) identity <- executeSql(parameterStorageBackend.ledgerIdentity)
end <- executeSql(backend.ledgerEnd) end <- executeSql(parameterStorageBackend.ledgerEnd)
events <- executeSql(backend.contractStateEvents(0, Long.MaxValue)) events <- executeSql(contractStorageBackend.contractStateEvents(0, Long.MaxValue))
// Check the contents (queries that don't read beyond ledger end) // Check the contents (queries that don't read beyond ledger end)
_ <- advanceLedgerEndToMakeOldDataVisible() _ <- advanceLedgerEndToMakeOldDataVisible()
parties <- executeSql(backend.knownParties) parties <- executeSql(partyStorageBackend.knownParties)
config <- executeSql(backend.ledgerConfiguration) config <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages <- executeSql(backend.lfPackages) packages <- executeSql(packageStorageBackend.lfPackages)
} yield { } yield {
identity shouldBe None identity shouldBe None
end shouldBe None end shouldBe None
@ -113,23 +124,23 @@ private[backend] trait StorageBackendTestsReset extends Matchers with StorageBac
for { for {
// Initialize and insert some data // Initialize and insert some data
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(dtos, _)) _ <- executeSql(ingest(dtos, _))
_ <- executeSql(backend.updateLedgerEnd(ledgerEnd(5, 3L))) _ <- executeSql(parameterStorageBackend.updateLedgerEnd(ledgerEnd(5, 3L)))
// Reset // Reset
_ <- executeSql(backend.resetAll) _ <- executeSql(resetStorageBackend.resetAll)
// Check the contents (queries that do not depend on ledger end) // Check the contents (queries that do not depend on ledger end)
identity <- executeSql(backend.ledgerIdentity) identity <- executeSql(parameterStorageBackend.ledgerIdentity)
end <- executeSql(backend.ledgerEnd) end <- executeSql(parameterStorageBackend.ledgerEnd)
events <- executeSql(backend.contractStateEvents(0, Long.MaxValue)) events <- executeSql(contractStorageBackend.contractStateEvents(0, Long.MaxValue))
// Check the contents (queries that don't read beyond ledger end) // Check the contents (queries that don't read beyond ledger end)
_ <- advanceLedgerEndToMakeOldDataVisible() _ <- advanceLedgerEndToMakeOldDataVisible()
parties <- executeSql(backend.knownParties) parties <- executeSql(partyStorageBackend.knownParties)
config <- executeSql(backend.ledgerConfiguration) config <- executeSql(configurationStorageBackend.ledgerConfiguration)
packages <- executeSql(backend.lfPackages) packages <- executeSql(packageStorageBackend.lfPackages)
} yield { } yield {
identity shouldBe None identity shouldBe None
end shouldBe None end shouldBe None
@ -145,8 +156,8 @@ private[backend] trait StorageBackendTestsReset extends Matchers with StorageBac
// queries now find any left-over data not cleaned by reset. // queries now find any left-over data not cleaned by reset.
private def advanceLedgerEndToMakeOldDataVisible(): Future[Unit] = { private def advanceLedgerEndToMakeOldDataVisible(): Future[Unit] = {
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(backend.updateLedgerEnd(ledgerEnd(10000, 10000))) _ <- executeSql(parameterStorageBackend.updateLedgerEnd(ledgerEnd(10000, 10000)))
} yield () } yield ()
} }
} }

View File

@ -15,6 +15,12 @@ import scala.util.Success
private[backend] trait StorageBackendTestsTimestamps extends Matchers with StorageBackendSpec { private[backend] trait StorageBackendTestsTimestamps extends Matchers with StorageBackendSpec {
this: AsyncFlatSpec => this: AsyncFlatSpec =>
private val parameterStorageBackend: ParameterStorageBackend =
backendFactory.createParameterStorageBackend
private val eventStorageBackend: EventStorageBackend = backendFactory.createEventStorageBackend
private val contractStorageBackend: ContractStorageBackend =
backendFactory.createContractStorageBackend
behavior of "StorageBackend (timestamps)" behavior of "StorageBackend (timestamps)"
import StorageBackendTestValues._ import StorageBackendTestValues._
@ -29,14 +35,20 @@ private[backend] trait StorageBackendTestsTimestamps extends Matchers with Stora
ledgerEffectiveTime = Some(let), ledgerEffectiveTime = Some(let),
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(Vector(create), _)) _ <- executeSql(ingest(Vector(create), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))
)
let1 <- executeSql(backend.maximumLedgerTime(Set(cid))) let1 <- executeSql(contractStorageBackend.maximumLedgerTime(Set(cid)))
let2 <- executeSql(withDefaultTimeZone("GMT-1")(backend.maximumLedgerTime(Set(cid)))) let2 <- executeSql(
let3 <- executeSql(withDefaultTimeZone("GMT+1")(backend.maximumLedgerTime(Set(cid)))) withDefaultTimeZone("GMT-1")(contractStorageBackend.maximumLedgerTime(Set(cid)))
)
let3 <- executeSql(
withDefaultTimeZone("GMT+1")(contractStorageBackend.maximumLedgerTime(Set(cid)))
)
} yield { } yield {
withClue("UTC") { let1 shouldBe Success(Some(let)) } withClue("UTC") { let1 shouldBe Success(Some(let)) }
withClue("GMT-1") { let2 shouldBe Success(Some(let)) } withClue("GMT-1") { let2 shouldBe Success(Some(let)) }
@ -54,14 +66,16 @@ private[backend] trait StorageBackendTestsTimestamps extends Matchers with Stora
ledgerEffectiveTime = Some(let), ledgerEffectiveTime = Some(let),
) )
for { for {
_ <- executeSql(backend.initializeParameters(someIdentityParams)) _ <- executeSql(parameterStorageBackend.initializeParameters(someIdentityParams))
_ <- executeSql(ingest(Vector(create), _)) _ <- executeSql(ingest(Vector(create), _))
_ <- executeSql(backend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))) _ <- executeSql(
parameterStorageBackend.updateLedgerEnd(ParameterStorageBackend.LedgerEnd(offset(1), 1L))
)
events1 <- executeSql(backend.rawEvents(0L, 1L)) events1 <- executeSql(eventStorageBackend.rawEvents(0L, 1L))
events2 <- executeSql(withDefaultTimeZone("GMT-1")(backend.rawEvents(0L, 1L))) events2 <- executeSql(withDefaultTimeZone("GMT-1")(eventStorageBackend.rawEvents(0L, 1L)))
events3 <- executeSql(withDefaultTimeZone("GMT+1")(backend.rawEvents(0L, 1L))) events3 <- executeSql(withDefaultTimeZone("GMT+1")(eventStorageBackend.rawEvents(0L, 1L)))
} yield { } yield {
withClue("UTC") { events1.head.ledgerEffectiveTime shouldBe Some(let) } withClue("UTC") { events1.head.ledgerEffectiveTime shouldBe Some(let) }
withClue("GMT-1") { events2.head.ledgerEffectiveTime shouldBe Some(let) } withClue("GMT-1") { events2.head.ledgerEffectiveTime shouldBe Some(let) }

View File

@ -26,7 +26,8 @@ class SequentialWriteDaoSpec extends AnyFlatSpec with Matchers {
val storageBackendCaptor = new StorageBackendCaptor(Some(LedgerEnd(Offset.beforeBegin, 5))) val storageBackendCaptor = new StorageBackendCaptor(Some(LedgerEnd(Offset.beforeBegin, 5)))
val ledgerEndCache = MutableLedgerEndCache() val ledgerEndCache = MutableLedgerEndCache()
val testee = SequentialWriteDaoImpl( val testee = SequentialWriteDaoImpl(
storageBackend = storageBackendCaptor, parameterStorageBackend = storageBackendCaptor,
ingestionStorageBackend = storageBackendCaptor,
updateToDbDtos = updateToDbDtoFixture, updateToDbDtos = updateToDbDtoFixture,
ledgerEndCache = ledgerEndCache, ledgerEndCache = ledgerEndCache,
) )
@ -62,7 +63,8 @@ class SequentialWriteDaoSpec extends AnyFlatSpec with Matchers {
val storageBackendCaptor = new StorageBackendCaptor(None) val storageBackendCaptor = new StorageBackendCaptor(None)
val ledgerEndCache = MutableLedgerEndCache() val ledgerEndCache = MutableLedgerEndCache()
val testee = SequentialWriteDaoImpl( val testee = SequentialWriteDaoImpl(
storageBackend = storageBackendCaptor, parameterStorageBackend = storageBackendCaptor,
ingestionStorageBackend = storageBackendCaptor,
updateToDbDtos = updateToDbDtoFixture, updateToDbDtos = updateToDbDtoFixture,
ledgerEndCache = ledgerEndCache, ledgerEndCache = ledgerEndCache,
) )

View File

@ -27,8 +27,10 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
"PostCommitValidation" when { "PostCommitValidation" when {
"run without prior history" should { "run without prior history" should {
val fixture = noCommittedContract(parties = List.empty)
val store = new PostCommitValidation.BackedBy( val store = new PostCommitValidation.BackedBy(
noCommittedContract(parties = List.empty), fixture,
fixture,
validatePartyAllocation = false, validatePartyAllocation = false,
) )
@ -257,17 +259,19 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
val committedContractLedgerEffectiveTime = val committedContractLedgerEffectiveTime =
Timestamp.assertFromInstant(Instant.ofEpochMilli(1000)) Timestamp.assertFromInstant(Instant.ofEpochMilli(1000))
val store = new PostCommitValidation.BackedBy( val fixture = committedContracts(
committedContracts( parties = List.empty,
parties = List.empty, contractFixture = committed(
contractFixture = committed( id = committedContract.coid.coid,
id = committedContract.coid.coid, ledgerEffectiveTime = committedContractLedgerEffectiveTime,
ledgerEffectiveTime = committedContractLedgerEffectiveTime, key = committedContract.key.map(x =>
key = committedContract.key.map(x => GlobalKey.assertBuild(committedContract.templateId, x.key)
GlobalKey.assertBuild(committedContract.templateId, x.key)
),
), ),
), ),
)
val store = new PostCommitValidation.BackedBy(
fixture,
fixture,
validatePartyAllocation = false, validatePartyAllocation = false,
) )
@ -422,11 +426,13 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
val divulgedContract = genTestCreate() val divulgedContract = genTestCreate()
val exerciseOnDivulgedContract = genTestExercise(divulgedContract) val exerciseOnDivulgedContract = genTestExercise(divulgedContract)
val fixture = committedContracts(
parties = List.empty,
contractFixture = divulged(divulgedContract.coid.coid),
)
val store = new PostCommitValidation.BackedBy( val store = new PostCommitValidation.BackedBy(
committedContracts( fixture,
parties = List.empty, fixture,
contractFixture = divulged(divulgedContract.coid.coid),
),
validatePartyAllocation = false, validatePartyAllocation = false,
) )
@ -453,6 +459,7 @@ final class PostCommitValidationSpec extends AnyWordSpec with Matchers {
"run with unallocated parties" should { "run with unallocated parties" should {
val store = new PostCommitValidation.BackedBy( val store = new PostCommitValidation.BackedBy(
noCommittedContract(List.empty),
noCommittedContract(List.empty), noCommittedContract(List.empty),
validatePartyAllocation = true, validatePartyAllocation = true,
) )

View File

@ -3,7 +3,7 @@
package com.daml.platform.store.backend package com.daml.platform.store.backend
import com.daml.platform.store.backend.postgresql.PostgresStorageBackend import com.daml.platform.store.backend.postgresql.PostgresDataSourceStorageBackend
import org.scalatest.Inside import org.scalatest.Inside
import org.scalatest.flatspec.AsyncFlatSpec import org.scalatest.flatspec.AsyncFlatSpec
@ -18,7 +18,7 @@ final class StorageBackendPostgresSpec
it should "find the Postgres version" in { it should "find the Postgres version" in {
for { for {
version <- executeSql(PostgresStorageBackend.getPostgresVersion) version <- executeSql(PostgresDataSourceStorageBackend.getPostgresVersion)
} yield { } yield {
inside(version) { case Some(versionNumbers) => inside(version) { case Some(versionNumbers) =>
// Minimum Postgres version used in tests // Minimum Postgres version used in tests
@ -29,9 +29,9 @@ final class StorageBackendPostgresSpec
} }
it should "correctly parse a Postgres version" in { it should "correctly parse a Postgres version" in {
PostgresStorageBackend.parsePostgresVersion("1.2") shouldBe Some((1, 2)) PostgresDataSourceStorageBackend.parsePostgresVersion("1.2") shouldBe Some((1, 2))
PostgresStorageBackend.parsePostgresVersion("1.2.3") shouldBe Some((1, 2)) PostgresDataSourceStorageBackend.parsePostgresVersion("1.2.3") shouldBe Some((1, 2))
PostgresStorageBackend.parsePostgresVersion("1.2.3-alpha.4.5") shouldBe Some((1, 2)) PostgresDataSourceStorageBackend.parsePostgresVersion("1.2.3-alpha.4.5") shouldBe Some((1, 2))
PostgresStorageBackend.parsePostgresVersion("10.11") shouldBe Some((10, 11)) PostgresDataSourceStorageBackend.parsePostgresVersion("10.11") shouldBe Some((10, 11))
} }
} }

View File

@ -6,43 +6,43 @@ package com.daml.platform.store.backend.h2
import org.scalatest.matchers.should.Matchers import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec import org.scalatest.wordspec.AnyWordSpec
class H2StorageBackendSpec extends AnyWordSpec with Matchers { class H2DataSourceStorageBackendSpec extends AnyWordSpec with Matchers {
"H2StorageBackend" should { "H2StorageBackend" should {
"extractUserPasswordAndRemoveFromUrl" should { "extractUserPasswordAndRemoveFromUrl" should {
"strip user from url with user" in { "strip user from url with user" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;user=harry" "url;user=harry"
) shouldBe (("url", Some("harry"), None)) ) shouldBe (("url", Some("harry"), None))
} }
"strip user from url with user in the middle" in { "strip user from url with user in the middle" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;user=harry;password=weak" "url;user=harry;password=weak"
) shouldBe (("url", Some("harry"), Some("weak"))) ) shouldBe (("url", Some("harry"), Some("weak")))
} }
"only strip password if user absent" in { "only strip password if user absent" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;password=weak" "url;password=weak"
) shouldBe (("url", None, Some("weak"))) ) shouldBe (("url", None, Some("weak")))
} }
"not touch other properties" in { "not touch other properties" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;alpha=1;beta=2;gamma=3" "url;alpha=1;beta=2;gamma=3"
) shouldBe (("url;alpha=1;beta=2;gamma=3", None, None)) ) shouldBe (("url;alpha=1;beta=2;gamma=3", None, None))
} }
"match upper-case user and password keys" in { "match upper-case user and password keys" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;USER=sally;PASSWORD=supersafe" "url;USER=sally;PASSWORD=supersafe"
) shouldBe (("url", Some("sally"), Some("supersafe"))) ) shouldBe (("url", Some("sally"), Some("supersafe")))
} }
"match mixed-case user and password keys" in { "match mixed-case user and password keys" in {
H2StorageBackend.extractUserPasswordAndRemoveFromUrl( H2DataSourceStorageBackend.extractUserPasswordAndRemoveFromUrl(
"url;User=sally;Password=supersafe" "url;User=sally;Password=supersafe"
) shouldBe (("url", Some("sally"), Some("supersafe"))) ) shouldBe (("url", Some("sally"), Some("supersafe")))
} }