Extract the log_entry from daml_kvutils.proto [KVL-1090] (#11193)

CHANGELOG_BEGIN
CHANGELOG_END
This commit is contained in:
nicu-da 2021-10-12 04:23:11 -07:00 committed by GitHub
parent 9c107a7aa8
commit b6f3f2087a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
52 changed files with 241 additions and 172 deletions

View File

@ -5,7 +5,7 @@ package com.daml.ledger.on.sql
import java.util.UUID
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.daml.ledger.validator.LogEntryIdAllocator
import com.google.protobuf.ByteString

View File

@ -17,70 +17,11 @@ option java_package = "com.daml.ledger.participant.state.kvutils";
option csharp_namespace = "Com.Daml.Ledger.Participant.State.KVUtils";
import "google/protobuf/duration.proto";
import "google/protobuf/empty.proto";
import "google/protobuf/timestamp.proto";
import "com/daml/daml_lf_dev/daml_lf.proto";
import "com/daml/lf/transaction.proto";
import "com/daml/ledger/configuration/ledger_configuration.proto";
import "com/daml/ledger/participant/state/kvutils/store/state.proto";
import "com/daml/ledger/participant/state/kvutils/store/events/configuration.proto";
// A log entry for a committed Daml submission.
// Produced by [[KeyValueCommitting]] from the `DamlSubmission` message.
// Each entry can be converted into a participant state `Update` event
// with [[KeyValueConsumption]], except for a time update entry.
//
// Please read comments in [[com.daml.ledger.participant.state.v2.package]]
// and [[com.daml.ledger.participant.state.kvutils.package]] for background
// information.
message DamlLogEntry {
// The record time at which this entry was committed.
google.protobuf.Timestamp record_time = 1;
oneof payload {
// A committed Daml transaction.
// Can be converted into `Update.TransactionAccepted`.
DamlTransactionEntry transaction_entry = 2;
// A rejected transaction.
DamlTransactionRejectionEntry transaction_rejection_entry = 3;
// A collection of Daml-LF archives uploaded together by a participant.
DamlPackageUploadEntry package_upload_entry = 4;
// A rejection of package upload.
DamlPackageUploadRejectionEntry package_upload_rejection_entry = 5;
// A Daml ledger configuration change.
com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry configuration_entry = 6;
// A rejected configuration change.
DamlConfigurationRejectionEntry configuration_rejection_entry = 7;
// Allocation of a new Daml party and its assignment to a participant.
DamlPartyAllocationEntry party_allocation_entry = 8;
// A rejection of party allocation request.
DamlPartyAllocationRejectionEntry party_allocation_rejection_entry = 9;
// A rejection of a pre-executed submission because of out-of-time-bounds.
DamlOutOfTimeBoundsEntry out_of_time_bounds_entry = 10;
// A log entry whose purpose is to transmit a current record time for pre-executed submissions.
google.protobuf.Empty time_update_entry = 101;
}
}
message DamlLogEntryId {
// Entry identifiers are opaque bytestrings chosen by the implementation that provide a unique
// reference to the entry. They do not need to be comparable, as long as the implementation can
// provide an ordered sequence of log entries.
// The entry identifier is used:
// * to lookup the log entry when producing `Update`.
// * as the Daml transaction identifier and rendered as hexadecimal.
// * as part of contract identifiers.
bytes entry_id = 1;
}
message DamlSubmitterInfo {
repeated string submitters = 1;
@ -301,17 +242,6 @@ message DamlPartyAllocationRejectionEntry {
}
}
// Indicates that a submission has been rejected after pre-execution.
// [[KeyValueConsumption.logEntryToUpdate]] will pick the right rejection reason based on current
// record time.
message DamlOutOfTimeBoundsEntry {
// We don't expect entry.recordTime to be present.
DamlLogEntry entry = 1;
google.protobuf.Timestamp duplicate_until = 2;
google.protobuf.Timestamp too_early_until = 3;
google.protobuf.Timestamp too_late_from = 4;
}
// Errors
//

View File

@ -0,0 +1,84 @@
// Copyright (c) 2021 Digital Asset (Switzerland) GmbH and/or its affiliates. All rights reserved.
// SPDX-License-Identifier: Apache-2.0
syntax = "proto3";
package com.daml.ledger.participant.state.kvutils.store;
option java_package = "com.daml.ledger.participant.state.kvutils.store";
option java_multiple_files = true;
option csharp_namespace = "Com.Daml.Ledger.Participant.State.KVUtils.Store";
import "google/protobuf/empty.proto";
import "google/protobuf/timestamp.proto";
import "com/daml/ledger/participant/state/kvutils/daml_kvutils.proto";
import "com/daml/ledger/participant/state/kvutils/store/events/configuration.proto";
// A log entry for a committed Daml submission.
// Produced by [[KeyValueCommitting]] from the `DamlSubmission` message.
// Each entry can be converted into a participant state `Update` event
// with [[KeyValueConsumption]], except for a time update entry.
//
// Please read comments in [[com.daml.ledger.participant.state.v2.package]]
// and [[com.daml.ledger.participant.state.kvutils.package]] for background
// information.
message DamlLogEntry {
// The record time at which this entry was committed.
google.protobuf.Timestamp record_time = 1;
oneof payload {
// A committed Daml transaction.
// Can be converted into `Update.TransactionAccepted`.
DamlTransactionEntry transaction_entry = 2;
// A rejected transaction.
DamlTransactionRejectionEntry transaction_rejection_entry = 3;
// A collection of Daml-LF archives uploaded together by a participant.
DamlPackageUploadEntry package_upload_entry = 4;
// A rejection of package upload.
DamlPackageUploadRejectionEntry package_upload_rejection_entry = 5;
// A Daml ledger configuration change.
com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry configuration_entry = 6;
// A rejected configuration change.
DamlConfigurationRejectionEntry configuration_rejection_entry = 7;
// Allocation of a new Daml party and its assignment to a participant.
DamlPartyAllocationEntry party_allocation_entry = 8;
// A rejection of party allocation request.
DamlPartyAllocationRejectionEntry party_allocation_rejection_entry = 9;
// A rejection of a pre-executed submission because of out-of-time-bounds.
DamlOutOfTimeBoundsEntry out_of_time_bounds_entry = 10;
// A log entry whose purpose is to transmit a current record time for pre-executed submissions.
google.protobuf.Empty time_update_entry = 101;
}
}
message DamlLogEntryId {
// Entry identifiers are opaque bytestrings chosen by the implementation that provide a unique
// reference to the entry. They do not need to be comparable, as long as the implementation can
// provide an ordered sequence of log entries.
// The entry identifier is used:
// * to lookup the log entry when producing `Update`.
// * as the Daml transaction identifier and rendered as hexadecimal.
// * as part of contract identifiers.
bytes entry_id = 1;
}
// There's a circular dependency between DamlLogEntry and DamlOutOfTimeBoundsEntry so we must define them in the same file
// Indicates that a submission has been rejected after pre-execution.
// [[KeyValueConsumption.logEntryToUpdate]] will pick the right rejection reason based on current
// record time.
message DamlOutOfTimeBoundsEntry {
// We don't expect entry.recordTime to be present.
DamlLogEntry entry = 1;
google.protobuf.Timestamp duplicate_until = 2;
google.protobuf.Timestamp too_early_until = 3;
google.protobuf.Timestamp too_late_from = 4;
}

View File

@ -5,8 +5,8 @@ package com.daml.ledger.participant.state.kvutils
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
import com.daml.ledger.participant.state.kvutils.store.DamlStateValue
import com.daml.ledger.participant.state.kvutils.{DamlKvutils => Proto, envelope => proto}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.{envelope => proto}
import com.google.protobuf.ByteString
import scala.util.Try
@ -21,7 +21,7 @@ object Envelope {
final case class SubmissionMessage(submission: wire.DamlSubmission) extends Message
final case class LogEntryMessage(logEntry: Proto.DamlLogEntry) extends Message
final case class LogEntryMessage(logEntry: DamlLogEntry) extends Message
final case class StateValueMessage(value: DamlStateValue) extends Message
@ -54,10 +54,10 @@ object Envelope {
def enclose(sub: wire.DamlSubmission, compression: Boolean): Raw.Envelope =
enclose(proto.Envelope.MessageKind.SUBMISSION, sub.toByteString, compression)
def enclose(logEntry: Proto.DamlLogEntry): Raw.Envelope =
def enclose(logEntry: DamlLogEntry): Raw.Envelope =
enclose(logEntry, compression = DefaultCompression)
def enclose(logEntry: Proto.DamlLogEntry, compression: Boolean): Raw.Envelope =
def enclose(logEntry: DamlLogEntry, compression: Boolean): Raw.Envelope =
enclose(proto.Envelope.MessageKind.LOG_ENTRY, logEntry.toByteString, compression)
def enclose(stateValue: DamlStateValue): Raw.Envelope =
@ -93,7 +93,7 @@ object Envelope {
}
message <- parsedEnvelope.getKind match {
case proto.Envelope.MessageKind.LOG_ENTRY =>
parseMessageSafe(() => Proto.DamlLogEntry.parseFrom(uncompressedMessage))
parseMessageSafe(() => DamlLogEntry.parseFrom(uncompressedMessage))
.map(LogEntryMessage)
case proto.Envelope.MessageKind.SUBMISSION =>
parseMessageSafe(() => wire.DamlSubmission.parseFrom(uncompressedMessage))
@ -109,7 +109,7 @@ object Envelope {
}
} yield message
def openLogEntry(envelopeBytes: Raw.Envelope): Either[String, Proto.DamlLogEntry] =
def openLogEntry(envelopeBytes: Raw.Envelope): Either[String, DamlLogEntry] =
open(envelopeBytes).flatMap {
case LogEntryMessage(entry) => Right(entry)
case msg => Left(s"Expected log entry, got ${msg.getClass}")

View File

@ -17,6 +17,8 @@ import com.daml.ledger.participant.state.kvutils.committer.{
}
import com.daml.ledger.participant.state.kvutils.store.{
DamlContractKey,
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}

View File

@ -7,7 +7,12 @@ import com.daml.ledger.configuration.Configuration
import com.daml.ledger.grpc.GrpcStatuses
import com.daml.ledger.participant.state.kvutils.Conversions._
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.store.DamlStateKey
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlOutOfTimeBoundsEntry,
DamlStateKey,
}
import com.daml.ledger.participant.state.v2.Update.CommandRejected.FinalReason
import com.daml.ledger.participant.state.v2.{DivulgedContract, TransactionMeta, Update}
import com.daml.lf.data.Ref

View File

@ -3,7 +3,7 @@
package com.daml.ledger.participant.state.kvutils
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.google.common.io.BaseEncoding
object Pretty {

View File

@ -3,8 +3,7 @@
package com.daml.ledger.participant.state.kvutils
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlStateKey
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntryId, DamlStateKey}
import com.google.protobuf.ByteString
object Raw {

View File

@ -5,7 +5,7 @@ package com.daml.ledger.participant.state.kvutils
import java.util.concurrent.atomic.AtomicLong
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.google.protobuf.ByteString
class SequentialLogEntryId(prefix: String) {

View File

@ -8,7 +8,7 @@ import akka.stream.scaladsl.Source
import com.daml.ledger.api.health.HealthStatus
import com.daml.ledger.configuration.LedgerInitialConditions
import com.daml.ledger.offset.Offset
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.{Envelope, KeyValueConsumption, OffsetBuilder}
import com.daml.ledger.participant.state.v2._
import com.daml.ledger.validator.preexecution.TimeUpdatesProvider

View File

@ -5,8 +5,7 @@ package com.daml.ledger.participant.state.kvutils.committer
import java.time.Instant
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.{DamlStateMap, Err}
import com.daml.lf.data.Ref
import com.daml.lf.data.Time.Timestamp

View File

@ -6,13 +6,14 @@ package com.daml.ledger.participant.state.kvutils.committer
import com.codahale.metrics.Timer
import com.daml.ledger.configuration.Configuration
import com.daml.ledger.participant.state.kvutils.Conversions.buildTimestamp
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlOutOfTimeBoundsEntry,
}
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils._
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlOutOfTimeBoundsEntry,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.lf.data.Time.Timestamp

View File

@ -11,7 +11,11 @@ import com.daml.ledger.participant.state.kvutils.Conversions.{
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.committer.Committer._
import com.daml.ledger.participant.state.kvutils.store.{DamlStateValue, DamlSubmissionDedupValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlStateValue,
DamlSubmissionDedupValue,
}
import com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry
import com.daml.ledger.participant.state.kvutils.wire.{DamlConfigurationSubmission, DamlSubmission}
import com.daml.lf.data.Time.Timestamp

View File

@ -11,6 +11,7 @@ import com.daml.ledger.participant.state.kvutils.DamlKvutils
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.committer.Committer.buildLogEntryWithOptionalRecordTime
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlStateKey,
DamlStateValue,
DamlSubmissionDedupValue,

View File

@ -7,6 +7,7 @@ import com.daml.ledger.participant.state.kvutils.Conversions.partyAllocationDedu
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.committer.Committer.buildLogEntryWithOptionalRecordTime
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlPartyAllocation,
DamlStateKey,
DamlStateValue,

View File

@ -3,7 +3,7 @@
package com.daml.ledger.participant.state.kvutils.committer
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry
private[kvutils] sealed trait StepResult[+PartialResult] {
def map[NewPartialResult](f: PartialResult => NewPartialResult): StepResult[NewPartialResult]

View File

@ -3,10 +3,9 @@
package com.daml.ledger.participant.state.kvutils.committer
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.DamlStateMap
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.lf.data.{Ref, Time}
import com.daml.logging.LoggingContext

View File

@ -22,6 +22,7 @@ import com.daml.ledger.participant.state.kvutils.store.{
DamlCommandDedupValue,
DamlContractKeyState,
DamlContractState,
DamlLogEntry,
DamlStateKey,
DamlStateValue,
}

View File

@ -7,7 +7,6 @@ import java.time.Instant
import com.daml.ledger.configuration.{Configuration, LedgerTimeModel}
import com.daml.ledger.participant.state.kvutils.Conversions.{commandDedupKey, parseTimestamp}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.committer.Committer.getCurrentConfiguration
import com.daml.ledger.participant.state.kvutils.committer.transaction.{
DamlTransactionEntrySummary,
@ -16,6 +15,7 @@ import com.daml.ledger.participant.state.kvutils.committer.transaction.{
Step,
}
import com.daml.ledger.participant.state.kvutils.committer.{CommitContext, StepContinue, StepResult}
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry
import com.daml.lf.data.Time.Timestamp
import com.daml.logging.LoggingContext

View File

@ -3,9 +3,13 @@
package com.daml.ledger.validator
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.export.SubmissionAggregator
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.lf.data.Ref
import com.daml.logging.LoggingContext

View File

@ -3,9 +3,13 @@
package com.daml.ledger.validator
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.export.SubmissionAggregator
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.{Envelope, Raw}
import com.daml.lf.data.Ref
import com.daml.logging.LoggingContext

View File

@ -3,7 +3,7 @@
package com.daml.ledger.validator
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
trait LogEntryIdAllocator {
def allocate(): DamlLogEntryId

View File

@ -5,8 +5,8 @@ package com.daml.ledger.validator
import java.security.MessageDigest
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.Raw
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.google.protobuf.ByteString
/** Computes log entry IDs from raw submission envelopes. */

View File

@ -6,10 +6,14 @@ package com.daml.ledger.validator
import java.util.concurrent.atomic.AtomicBoolean
import com.codahale.metrics.Timer
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.api.LedgerReader
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.{DamlStateMap, Envelope, KeyValueCommitting, Raw}
import com.daml.ledger.validator.SubmissionValidator._
import com.daml.ledger.validator.ValidationFailed.{MissingInputState, ValidationError}

View File

@ -8,15 +8,18 @@ import java.time.Instant
import akka.NotUsed
import akka.stream.Materializer
import akka.stream.scaladsl.{Sink, Source}
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.api.LedgerReader
import com.daml.ledger.participant.state.kvutils.export.{
LedgerDataExporter,
SubmissionAggregator,
SubmissionInfo,
}
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.{CorrelationId, Envelope, KeyValueCommitting, Raw}
import com.daml.ledger.validator
import com.daml.ledger.validator.SubmissionValidator.LogEntryAndState

View File

@ -3,9 +3,8 @@
package com.daml.ledger.validator.batch
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry.PayloadCase._
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry.PayloadCase._
import com.daml.lf.value.ValueCoder
import com.daml.logging.{ContextualizedLogger, LoggingContext}
import com.daml.metrics.Metrics

View File

@ -4,9 +4,13 @@
package com.daml.ledger.validator.caching
import com.daml.caching.Cache
import com.daml.ledger.participant.state.kvutils.DamlKvutils
import com.daml.ledger.participant.state.kvutils.export.SubmissionAggregator
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.validator.{
CommitStrategy,
LedgerStateOperations,
@ -27,8 +31,8 @@ final class CachingCommitStrategy[Result](
override def commit(
participantId: Ref.ParticipantId,
correlationId: String,
entryId: DamlKvutils.DamlLogEntryId,
entry: DamlKvutils.DamlLogEntry,
entryId: DamlLogEntryId,
entry: DamlLogEntry,
inputState: Map[DamlStateKey, Option[DamlStateValue]],
outputState: Map[DamlStateKey, DamlStateValue],
exporterWriteSet: Option[SubmissionAggregator.WriteSetBuilder],

View File

@ -3,8 +3,8 @@
package com.daml.ledger.validator.preexecution
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.daml.lf.data.Ref
import scala.concurrent.{ExecutionContext, Future}

View File

@ -3,8 +3,12 @@
package com.daml.ledger.validator.preexecution
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.{Envelope, KeyValueCommitting, Raw}
import com.daml.ledger.validator.preexecution.RawPreExecutingCommitStrategy.{InputState, ReadSet}
import com.daml.ledger.validator.{

View File

@ -9,9 +9,13 @@ import com.codahale.metrics.MetricRegistry
import com.daml.daml_lf_dev.DamlLf
import com.daml.ledger.api.DeduplicationPeriod
import com.daml.ledger.configuration.{Configuration, LedgerTimeModel}
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.ledger.participant.state.v2.{SubmitterInfo, TransactionMeta}
import com.daml.ledger.test.SimplePackagePartyTestDar

View File

@ -9,13 +9,13 @@ import java.util.UUID
import com.daml.daml_lf_dev.DamlLf
import com.daml.ledger.configuration.{Configuration, LedgerTimeModel}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntryId,
DamlSubmitterInfo,
DamlTransactionEntry,
DamlTransactionRejectionEntry,
}
import com.daml.ledger.participant.state.kvutils.committer.transaction.DamlTransactionEntrySummary
import com.daml.ledger.participant.state.kvutils.committer.{CommitContext, StepResult, StepStop}
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.daml.lf.data.Ref
import com.daml.lf.data.Time.Timestamp
import com.daml.lf.transaction.SubmittedTransaction

View File

@ -12,6 +12,8 @@ import com.daml.ledger.participant.state.kvutils.store.{
DamlContractKey,
DamlContractKeyState,
DamlContractState,
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
DamlSubmissionDedupKey,

View File

@ -3,9 +3,8 @@
package com.daml.ledger.participant.state.kvutils
import com.daml.ledger.participant.state.kvutils.store.DamlStateValue
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.{DamlKvutils => Proto}
import com.google.protobuf.ByteString
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec
@ -19,7 +18,7 @@ class EnvelopeSpec extends AnyWordSpec with Matchers {
Envelope.open(Envelope.enclose(submission)) shouldEqual
Right(Envelope.SubmissionMessage(submission))
val logEntry = Proto.DamlLogEntry.getDefaultInstance
val logEntry = DamlLogEntry.getDefaultInstance
Envelope.open(Envelope.enclose(logEntry)) shouldEqual
Right(Envelope.LogEntryMessage(logEntry))

View File

@ -8,6 +8,7 @@ import java.time.Duration
import com.codahale.metrics.MetricRegistry
import com.daml.ledger.configuration.Configuration
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry
import com.daml.lf.data.Ref
import com.daml.logging.LoggingContext
import com.daml.metrics.Metrics

View File

@ -4,10 +4,8 @@
package com.daml.ledger.participant.state.kvutils
import com.daml.bazeltools.BazelRunfiles
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlPackageUploadRejectionEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPackageUploadRejectionEntry
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry
import com.daml.ledger.test.{ModelTestDar, SimplePackagePartyTestDar}
import com.daml.logging.LoggingContext
import com.daml.platform.testing.TestDarReader

View File

@ -3,10 +3,8 @@
package com.daml.ledger.participant.state.kvutils
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlPartyAllocationRejectionEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPartyAllocationRejectionEntry
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry
import com.daml.logging.LoggingContext
import org.scalatest.matchers.should.Matchers
import org.scalatest.wordspec.AnyWordSpec

View File

@ -5,12 +5,9 @@ package com.daml.ledger.participant.state.kvutils
import java.time.Duration
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlTransactionRejectionEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlTransactionRejectionEntry
import com.daml.ledger.participant.state.kvutils.TestHelpers._
import com.daml.ledger.participant.state.kvutils.store.DamlStateValue
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.ledger.participant.state.v2.Update
import com.daml.ledger.test._

View File

@ -8,7 +8,6 @@ import java.time.Instant
import com.daml.ledger.configuration.Configuration
import com.daml.ledger.grpc.GrpcStatuses
import com.daml.ledger.participant.state.kvutils.Conversions.{buildTimestamp, parseInstant}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntry.PayloadCase._
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.KeyValueConsumption.{
TimeBounds,
@ -16,7 +15,13 @@ import com.daml.ledger.participant.state.kvutils.KeyValueConsumption.{
outOfTimeBoundsEntryToUpdate,
}
import com.daml.ledger.participant.state.kvutils.api.LedgerReader
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntry.PayloadCase._
import com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlOutOfTimeBoundsEntry,
}
import com.daml.ledger.participant.state.v2.Update
import com.daml.ledger.participant.state.v2.Update.CommandRejected
import com.daml.ledger.participant.state.v2.Update.CommandRejected.FinalReason

View File

@ -11,7 +11,12 @@ import com.daml.ledger.offset.Offset
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.api.KeyValueParticipantStateReader.offsetForUpdate
import com.daml.ledger.participant.state.kvutils.api.KeyValueParticipantStateReaderSpec._
import com.daml.ledger.participant.state.kvutils.store.{DamlPartyAllocation, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlPartyAllocation,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.{Envelope, OffsetBuilder, Raw}
import com.daml.ledger.participant.state.v2.Update
import com.daml.lf.data.Ref

View File

@ -15,7 +15,7 @@ import com.daml.ledger.participant.state.kvutils.Err
import com.daml.ledger.participant.state.kvutils.TestHelpers.{createCommitContext, theDefaultConfig}
import com.daml.ledger.participant.state.kvutils.committer.CommitterSpec._
import com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.lf.data.Ref
import com.daml.lf.data.Time.Timestamp

View File

@ -10,7 +10,7 @@ import com.daml.daml_lf_dev.DamlLf
import com.daml.ledger.participant.state.kvutils.Conversions.buildTimestamp
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.TestHelpers._
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmission
import com.daml.lf.archive.Decode
import com.daml.lf.archive.testing.Encode

View File

@ -16,6 +16,7 @@ import com.daml.ledger.participant.state.kvutils.committer.transaction.{
import com.daml.ledger.participant.state.kvutils.committer.{StepContinue, StepStop}
import com.daml.ledger.participant.state.kvutils.store.{
DamlContractState,
DamlLogEntry,
DamlStateKey,
DamlStateValue,
}

View File

@ -7,12 +7,16 @@ import java.time.Clock
import com.codahale.metrics.MetricRegistry
import com.daml.ledger.api.testing.utils.AkkaBeforeAndAfterAll
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.export.{
NoOpLedgerDataExporter,
SubmissionAggregator,
}
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmissionBatch.CorrelatedSubmission
import com.daml.ledger.participant.state.kvutils.wire._
import com.daml.ledger.participant.state.kvutils.{Envelope, KeyValueCommitting, Raw}

View File

@ -7,6 +7,7 @@ import com.codahale.metrics.MetricRegistry
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.store.{
DamlContractKey,
DamlLogEntry,
DamlPartyAllocation,
DamlStateKey,
DamlStateValue,

View File

@ -4,10 +4,14 @@
package com.daml.ledger.validator.caching
import com.daml.caching.{Cache, WeightedCache}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.caching.`Message Weight`
import com.daml.ledger.participant.state.kvutils.export.SubmissionAggregator
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.validator.CommitStrategy
import com.daml.ledger.validator.TestHelper._
import com.daml.ledger.validator.caching.CachingCommitStrategySpec._

View File

@ -7,9 +7,12 @@ import java.time.Instant
import com.codahale.metrics.MetricRegistry
import com.daml.ledger.configuration.Configuration
import com.daml.ledger.participant.state.kvutils.DamlKvutils._
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.wire.DamlSubmissionBatch.CorrelatedSubmission
import com.daml.ledger.participant.state.kvutils.wire.{
DamlConfigurationSubmission,

View File

@ -3,13 +3,11 @@
package com.daml.ledger.validator.preexecution
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlPartyAllocationRejectionEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPartyAllocationRejectionEntry
import com.daml.ledger.participant.state.kvutils.KeyValueCommitting.PreExecutionResult
import com.daml.ledger.participant.state.kvutils.store.{
DamlContractState,
DamlLogEntry,
DamlStateKey,
DamlStateValue,
}

View File

@ -5,9 +5,14 @@ package com.daml.ledger.participant.state.kvutils.tools.integritycheck
import com.daml.ledger.participant.state.kvutils
import com.daml.ledger.participant.state.kvutils.export.WriteSet
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlStateValue}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlStateKey,
DamlStateValue,
}
import com.daml.ledger.participant.state.kvutils.tools.integritycheck.WriteSetComparison._
import com.daml.ledger.participant.state.kvutils.{DamlKvutils, Envelope, Raw}
import com.daml.ledger.participant.state.kvutils.{Envelope, Raw}
import com.daml.ledger.validator.StateKeySerializationStrategy
import scala.PartialFunction.condOpt
@ -129,15 +134,15 @@ final class RawWriteSetComparison(stateKeySerializationStrategy: StateKeySeriali
rawKey: Raw.Key,
rawEnvelope: Raw.Envelope,
): Either[String, Either[
(DamlKvutils.DamlLogEntryId, DamlKvutils.DamlLogEntry),
(DamlLogEntryId, DamlLogEntry),
(DamlStateKey, DamlStateValue),
]] =
Envelope.open(rawEnvelope) match {
case Left(errorMessage) =>
Left(s"Invalid value envelope: $errorMessage")
case Right(Envelope.LogEntryMessage(logEntry)) =>
val logEntryId = DamlKvutils.DamlLogEntryId.parseFrom(rawKey.bytes)
if (logEntry.getPayloadCase == DamlKvutils.DamlLogEntry.PayloadCase.PAYLOAD_NOT_SET)
val logEntryId = DamlLogEntryId.parseFrom(rawKey.bytes)
if (logEntry.getPayloadCase == DamlLogEntry.PayloadCase.PAYLOAD_NOT_SET)
Left("Log entry payload not set.")
else
Right(Left(logEntryId -> logEntry))

View File

@ -9,12 +9,9 @@ import java.util.concurrent.TimeUnit
import akka.stream.scaladsl.Sink
import com.codahale.metrics.MetricRegistry
import com.daml.ledger.api.testing.utils.AkkaBeforeAndAfterAll
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlLogEntryId,
DamlPartyAllocationEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPartyAllocationEntry
import com.daml.ledger.participant.state.kvutils.export.SubmissionInfo
import com.daml.ledger.participant.state.kvutils.store.{DamlLogEntry, DamlLogEntryId}
import com.daml.ledger.participant.state.kvutils.tools.integritycheck.LogAppendingReadServiceFactorySpec._
import com.daml.ledger.participant.state.kvutils.{Envelope, Raw}
import com.daml.ledger.participant.state.v2.Update

View File

@ -8,12 +8,13 @@ import java.time.{Duration, Instant, ZoneOffset, ZonedDateTime}
import com.codahale.metrics.MetricRegistry
import com.daml.ledger.api.testing.utils.AkkaBeforeAndAfterAll
import com.daml.ledger.configuration.{Configuration, LedgerTimeModel}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlPartyAllocationEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPartyAllocationEntry
import com.daml.ledger.participant.state.kvutils.export.{SubmissionInfo, WriteSet}
import com.daml.ledger.participant.state.kvutils.store.{DamlStateKey, DamlSubmissionDedupKey}
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlStateKey,
DamlSubmissionDedupKey,
}
import com.daml.ledger.participant.state.kvutils.tools.integritycheck.RawPreExecutingCommitStrategySupportSpec._
import com.daml.ledger.participant.state.kvutils.wire.{DamlConfigurationSubmission, DamlSubmission}
import com.daml.ledger.participant.state.kvutils.{Envelope, Raw}

View File

@ -4,13 +4,11 @@
package com.daml.ledger.participant.state.kvutils.tools.integritycheck
import com.daml.ledger.configuration.protobuf.LedgerConfiguration
import com.daml.ledger.participant.state.kvutils.DamlKvutils.{
DamlLogEntry,
DamlLogEntryId,
DamlPartyAllocationEntry,
}
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlPartyAllocationEntry
import com.daml.ledger.participant.state.kvutils.store.events.DamlConfigurationEntry
import com.daml.ledger.participant.state.kvutils.store.{
DamlLogEntry,
DamlLogEntryId,
DamlPartyAllocation,
DamlStateKey,
DamlStateValue,

View File

@ -5,7 +5,7 @@ package com.daml.platform.sandboxnext
import java.util.UUID
import com.daml.ledger.participant.state.kvutils.DamlKvutils.DamlLogEntryId
import com.daml.ledger.participant.state.kvutils.store.DamlLogEntryId
import com.daml.ledger.validator.LogEntryIdAllocator
import com.daml.platform.apiserver.SeedService
import com.google.protobuf.ByteString