Skip to content

Commit

Permalink
feat: add cl0 transaction tracking
Browse files Browse the repository at this point in the history
  • Loading branch information
effects-ai committed Dec 11, 2024
1 parent d6aa0fa commit 46fb287
Show file tree
Hide file tree
Showing 17 changed files with 245 additions and 26 deletions.
6 changes: 3 additions & 3 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -356,7 +356,7 @@ lazy val rosetta = (project in file("modules/rosetta"))
lazy val dagL1 = (project in file("modules/dag-l1"))
.enablePlugins(AshScriptPlugin)
.enablePlugins(JavaAppPackaging)
.dependsOn(kernel, shared % "compile->compile;test->test", nodeShared, testShared % Test)
.dependsOn(kernel, shared % "compile->compile;test->test", nodeShared % "compile->compile;test->test", testShared % Test)
.configs(IntegrationTest)
.settings(
name := "tessellation-dag-l1",
Expand Down Expand Up @@ -445,7 +445,7 @@ lazy val tools = (project in file("modules/tools"))
lazy val dagL0 = (project in file("modules/dag-l0"))
.enablePlugins(AshScriptPlugin)
.enablePlugins(JavaAppPackaging)
.dependsOn(keytool, kernel, shared % "compile->compile;test->test", testShared % Test, nodeShared)
.dependsOn(keytool, kernel, shared % "compile->compile;test->test", testShared % Test, nodeShared % "compile->compile;test->test")
.settings(
name := "tessellation-dag-l0",
Defaults.itSettings,
Expand Down Expand Up @@ -506,7 +506,7 @@ lazy val dagL0 = (project in file("modules/dag-l0"))
)

lazy val currencyL1 = (project in file("modules/currency-l1"))
.dependsOn(dagL1, shared % "compile->compile;test->test", testShared % Test, nodeShared)
.dependsOn(dagL1, shared % "compile->compile;test->test", testShared % Test, nodeShared % "compile->compile;test->test")
.settings(
name := "tessellation-currency-l1",
Defaults.itSettings,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,8 @@ abstract class CurrencyL0App(
tessellationVersion,
cfg.http,
services.dataApplication,
metagraphVersion.some
metagraphVersion.some,
sharedServices.currencySnapshotValidationErrorStorage
)
_ <- MkHttpServer[IO].newEmber(ServerName("public"), cfg.http.publicHttp, api.publicApp)
_ <- MkHttpServer[IO].newEmber(ServerName("p2p"), cfg.http.p2pHttp, api.p2pApp)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
package io.constellationnetwork.currency.l0.http.routes

import java.time.Instant

import cats.effect.Async
import cats.syntax.flatMap._

import io.constellationnetwork.node.shared.domain.block.processing.BlockRejectionReason
import io.constellationnetwork.node.shared.infrastructure.consensus.ValidationErrorStorage
import io.constellationnetwork.node.shared.snapshot.currency
import io.constellationnetwork.node.shared.snapshot.currency.CurrencySnapshotEvent
import io.constellationnetwork.routes.internal._

import eu.timepit.refined.auto._
import io.circe.syntax.EncoderOps
import io.circe.{Encoder, Json}
import org.http4s.HttpRoutes
import org.http4s.circe.CirceEntityEncoder._
import org.http4s.dsl.Http4sDsl

final case class ValidationErrorRoutes[F[_]: Async](
currencySnapshotValidationErrorStorage: ValidationErrorStorage[F, CurrencySnapshotEvent, Option[BlockRejectionReason]]
) extends Http4sDsl[F]
with PublicRoutes[F] {

implicit val encoder: Encoder[(CurrencySnapshotEvent, Option[BlockRejectionReason], Instant)] =
new Encoder[(CurrencySnapshotEvent, Option[BlockRejectionReason], Instant)] {
override def apply(a: (CurrencySnapshotEvent, Option[BlockRejectionReason], Instant)): Json = Json.obj(
(
"block",
a._1 match {
case currency.BlockEvent(value) => value.asJson
case currency.DataApplicationBlockEvent(_) => Json.Null
case currency.CurrencyMessageEvent(_) => Json.Null
}
),
("timestamp", a._3.asJson)
)
}

override protected val prefixPath: InternalUrlPrefix = "/currency"

override protected val public: HttpRoutes[F] = HttpRoutes.of[F] {
case GET -> Root / "validation-errors" =>
currencySnapshotValidationErrorStorage.allValidationErrors.flatMap(Ok(_))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -9,15 +9,17 @@ import io.constellationnetwork.currency.dataApplication.dataApplication.DataAppl
import io.constellationnetwork.currency.dataApplication.{BaseDataApplicationL0Service, L0NodeContext}
import io.constellationnetwork.currency.l0.cell.{L0Cell, L0CellInput}
import io.constellationnetwork.currency.l0.cli.method.Run
import io.constellationnetwork.currency.l0.http.routes.{CurrencyBlockRoutes, CurrencyMessageRoutes, DataBlockRoutes}
import io.constellationnetwork.currency.l0.http.routes._
import io.constellationnetwork.currency.l0.snapshot.CurrencySnapshotKey
import io.constellationnetwork.currency.l0.snapshot.schema.CurrencyConsensusOutcome
import io.constellationnetwork.currency.schema.currency._
import io.constellationnetwork.env.AppEnvironment
import io.constellationnetwork.env.AppEnvironment.{Dev, Integrationnet, Testnet}
import io.constellationnetwork.node.shared.config.types.HttpConfig
import io.constellationnetwork.node.shared.domain.block.processing.BlockRejectionReason
import io.constellationnetwork.node.shared.http.p2p.middlewares.{PeerAuthMiddleware, `X-Id-Middleware`}
import io.constellationnetwork.node.shared.http.routes._
import io.constellationnetwork.node.shared.infrastructure.consensus.ValidationErrorStorage
import io.constellationnetwork.node.shared.infrastructure.metrics.Metrics
import io.constellationnetwork.node.shared.snapshot.currency.CurrencySnapshotEvent
import io.constellationnetwork.schema.peer.PeerId
Expand All @@ -43,7 +45,8 @@ object HttpApi {
nodeVersion: TessellationVersion,
httpCfg: HttpConfig,
maybeDataApplication: Option[BaseDataApplicationL0Service[F]],
maybeMetagraphVersion: Option[MetagraphVersion]
maybeMetagraphVersion: Option[MetagraphVersion],
currencySnapshotValidationErrorStorage: ValidationErrorStorage[F, CurrencySnapshotEvent, Option[BlockRejectionReason]]
): HttpApi[F] =
new HttpApi[F](
validators,
Expand All @@ -57,7 +60,8 @@ object HttpApi {
nodeVersion,
httpCfg,
maybeDataApplication,
maybeMetagraphVersion
maybeMetagraphVersion,
currencySnapshotValidationErrorStorage
) {}
}

Expand All @@ -73,7 +77,8 @@ sealed abstract class HttpApi[F[_]: Async: SecurityProvider: HasherSelector: Met
nodeVersion: TessellationVersion,
httpCfg: HttpConfig,
maybeDataApplication: Option[BaseDataApplicationL0Service[F]],
maybeMetagraphVersion: Option[MetagraphVersion]
maybeMetagraphVersion: Option[MetagraphVersion],
currencySnapshotValidationErrorStorage: ValidationErrorStorage[F, CurrencySnapshotEvent, Option[BlockRejectionReason]]
) {

private val mkCell = (event: CurrencySnapshotEvent) =>
Expand All @@ -99,6 +104,7 @@ sealed abstract class HttpApi[F[_]: Async: SecurityProvider: HasherSelector: Met
implicit val (d, e) = (da.dataDecoder, da.calculatedStateEncoder)
DataBlockRoutes[F](mkCell, da)
}
private val validationErrorRoutes = ValidationErrorRoutes(currencySnapshotValidationErrorStorage)
private val metagraphNodeRoutes = maybeMetagraphVersion.map { metagraphVersion =>
MetagraphRoutes[F](storages.node, storages.session, storages.cluster, httpCfg, selfId, nodeVersion, metagraphVersion)
}
Expand Down Expand Up @@ -143,6 +149,7 @@ sealed abstract class HttpApi[F[_]: Async: SecurityProvider: HasherSelector: Met
clusterRoutes.publicRoutes <+>
currencyBlockRoutes.publicRoutes <+>
dataBlockRoutes.map(_.publicRoutes).getOrElse(HttpRoutes.empty) <+>
validationErrorRoutes.publicRoutes <+>
walletRoutes.publicRoutes <+>
nodeRoutes.publicRoutes <+>
consensusInfoRoutes.publicRoutes <+>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,8 @@ object Services {
sharedServices.currencySnapshotAcceptanceManager,
dataApplicationAcceptanceManager,
cfg.snapshotSize,
sharedServices.currencyEventsCutter
sharedServices.currencyEventsCutter,
sharedServices.currencySnapshotValidationErrorStorage
)

validator = CurrencySnapshotValidator.make[F](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,13 @@ import io.constellationnetwork.ext.cats.effect.ResourceIO
import io.constellationnetwork.json.{JsonBrotliBinarySerializer, JsonSerializer}
import io.constellationnetwork.kryo.KryoSerializer
import io.constellationnetwork.node.shared.config.types.{AddressesConfig, SnapshotSizeConfig}
import io.constellationnetwork.node.shared.domain.block.processing.BlockRejectionReason
import io.constellationnetwork.node.shared.domain.statechannel._
import io.constellationnetwork.node.shared.infrastructure.block.processing.BlockAcceptanceManager
import io.constellationnetwork.node.shared.infrastructure.consensus.MockValidationErrorStorage
import io.constellationnetwork.node.shared.infrastructure.snapshot._
import io.constellationnetwork.node.shared.modules.SharedValidators
import io.constellationnetwork.node.shared.snapshot.currency.CurrencySnapshotEvent
import io.constellationnetwork.schema.address.Address
import io.constellationnetwork.schema.balance.Amount
import io.constellationnetwork.schema.peer.PeerId
Expand Down Expand Up @@ -75,8 +78,15 @@ object GlobalSnapshotStateChannelEventsProcessorSuite extends MutableIOSuite {
validators.currencyMessageValidator
)
currencyEventsCutter = CurrencyEventsCutter.make[IO](None)
validationErrorStorage = MockValidationErrorStorage.make[CurrencySnapshotEvent, Option[BlockRejectionReason]]()
creator = CurrencySnapshotCreator
.make[IO](currencySnapshotAcceptanceManager, None, SnapshotSizeConfig(Long.MaxValue, Long.MaxValue), currencyEventsCutter)
.make[IO](
currencySnapshotAcceptanceManager,
None,
SnapshotSizeConfig(Long.MaxValue, Long.MaxValue),
currencyEventsCutter,
validationErrorStorage
)
currencySnapshotValidator = CurrencySnapshotValidator.make[IO](creator, validators.signedValidator, None, None)
currencySnapshotContextFns = CurrencySnapshotContextFunctions.make(currencySnapshotValidator)
manager = new GlobalSnapshotStateChannelAcceptanceManager[IO] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,18 +13,20 @@ import cats.syntax.traverse._

import scala.collection.immutable.{SortedMap, SortedSet}

import io.constellationnetwork.dag.l0.infrastructure.snapshot._
import io.constellationnetwork.ext.cats.effect.ResourceIO
import io.constellationnetwork.ext.cats.syntax.next.catsSyntaxNext
import io.constellationnetwork.json.{JsonBrotliBinarySerializer, JsonSerializer}
import io.constellationnetwork.kryo.KryoSerializer
import io.constellationnetwork.node.shared.config.types.{AddressesConfig, SnapshotSizeConfig}
import io.constellationnetwork.node.shared.domain.block.processing.BlockRejectionReason
import io.constellationnetwork.node.shared.domain.statechannel.FeeCalculator
import io.constellationnetwork.node.shared.domain.transaction.{TransactionChainValidator, TransactionValidator}
import io.constellationnetwork.node.shared.infrastructure.block.processing.{BlockAcceptanceLogic, BlockAcceptanceManager, BlockValidator}
import io.constellationnetwork.node.shared.infrastructure.consensus.MockValidationErrorStorage
import io.constellationnetwork.node.shared.infrastructure.metrics.Metrics
import io.constellationnetwork.node.shared.infrastructure.snapshot._
import io.constellationnetwork.node.shared.modules.SharedValidators
import io.constellationnetwork.node.shared.snapshot.currency.CurrencySnapshotEvent
import io.constellationnetwork.schema._
import io.constellationnetwork.schema.address.Address
import io.constellationnetwork.schema.balance.{Amount, Balance}
Expand Down Expand Up @@ -244,11 +246,19 @@ object GlobalSnapshotTraverseSuite extends MutableIOSuite with Checkers {
)
val currencyEventsCutter = CurrencyEventsCutter.make[IO](None)

val validationErrorStorage = MockValidationErrorStorage.make[CurrencySnapshotEvent, Option[BlockRejectionReason]]()

implicit val hs = HasherSelector.forSyncAlwaysCurrent(H)

val currencySnapshotCreator =
CurrencySnapshotCreator
.make[IO](currencySnapshotAcceptanceManager, None, SnapshotSizeConfig(Long.MaxValue, Long.MaxValue), currencyEventsCutter)
.make[IO](
currencySnapshotAcceptanceManager,
None,
SnapshotSizeConfig(Long.MaxValue, Long.MaxValue),
currencyEventsCutter,
validationErrorStorage
)
val currencySnapshotValidator = CurrencySnapshotValidator.make[IO](currencySnapshotCreator, validators.signedValidator, None, None)

val currencySnapshotContextFns = CurrencySnapshotContextFunctions.make(currencySnapshotValidator)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,12 +21,15 @@ import io.constellationnetwork.ext.collection.MapRefUtils._
import io.constellationnetwork.json.{JsonBrotliBinarySerializer, JsonSerializer}
import io.constellationnetwork.kryo.KryoSerializer
import io.constellationnetwork.node.shared.config.types.{AddressesConfig, SnapshotSizeConfig}
import io.constellationnetwork.node.shared.domain.block.processing.BlockRejectionReason
import io.constellationnetwork.node.shared.domain.statechannel.FeeCalculator
import io.constellationnetwork.node.shared.infrastructure.block.processing.BlockAcceptanceManager
import io.constellationnetwork.node.shared.infrastructure.consensus.MockValidationErrorStorage
import io.constellationnetwork.node.shared.infrastructure.snapshot._
import io.constellationnetwork.node.shared.infrastructure.snapshot.storage.LastSnapshotStorage
import io.constellationnetwork.node.shared.modules.SharedValidators
import io.constellationnetwork.node.shared.nodeSharedKryoRegistrar
import io.constellationnetwork.node.shared.snapshot.currency.CurrencySnapshotEvent
import io.constellationnetwork.schema._
import io.constellationnetwork.schema.address.Address
import io.constellationnetwork.schema.balance.{Amount, Balance}
Expand Down Expand Up @@ -99,8 +102,15 @@ object SnapshotProcessorSuite extends SimpleIOSuite with TransactionGenerator {
)
implicit0(hs: HasherSelector[IO]) = HasherSelector.forSyncAlwaysCurrent(h)
currencyEventsCutter = CurrencyEventsCutter.make[IO](None)
validationErrorStorage = MockValidationErrorStorage.make[CurrencySnapshotEvent, Option[BlockRejectionReason]]()
currencySnapshotCreator = CurrencySnapshotCreator
.make[IO](currencySnapshotAcceptanceManager, None, SnapshotSizeConfig(Long.MaxValue, Long.MaxValue), currencyEventsCutter)
.make[IO](
currencySnapshotAcceptanceManager,
None,
SnapshotSizeConfig(Long.MaxValue, Long.MaxValue),
currencyEventsCutter,
validationErrorStorage
)
currencySnapshotValidator = CurrencySnapshotValidator
.make[IO](currencySnapshotCreator, validators.signedValidator, None, None)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,26 +9,27 @@ import io.constellationnetwork.schema.{BlockReference, SnapshotOrdinal}
import io.constellationnetwork.security.hash.Hash

import derevo.cats.{eqv, show}
import derevo.circe.magnolia.encoder
import derevo.derive

// NOTE: @derive(eqv, show) should not be required for each individual case class, but otherwise it throws in runtime

@derive(eqv, show)
sealed trait BlockNotAcceptedReason

@derive(eqv, show)
@derive(eqv, show, encoder)
sealed trait BlockRejectionReason extends BlockNotAcceptedReason

@derive(eqv, show)
@derive(eqv, show, encoder)
case class ValidationFailed(reasons: NonEmptyList[BlockValidationError]) extends BlockRejectionReason

@derive(eqv, show)
@derive(eqv, show, encoder)
case class ParentNotFound(parent: BlockReference) extends BlockRejectionReason

@derive(eqv, show)
@derive(eqv, show, encoder)
case object SnapshotOrdinalUnavailable extends BlockRejectionReason

@derive(eqv, show)
@derive(eqv, show, encoder)
case class RejectedTransaction(tx: TransactionReference, reason: TransactionRejectionReason) extends BlockRejectionReason

@derive(eqv, show)
Expand All @@ -53,7 +54,7 @@ sealed trait TransactionAwaitReason
case class ParentOrdinalAboveLastTxOrdinal(parentOrdinal: TransactionOrdinal, lastTxOrdinal: TransactionOrdinal)
extends TransactionAwaitReason

@derive(eqv, show)
@derive(eqv, show, encoder)
sealed trait TransactionRejectionReason

@derive(eqv, show)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import io.constellationnetwork.security.signature.Signed
import io.constellationnetwork.security.signature.SignedValidator.SignedValidationError

import derevo.cats.{eqv, show}
import derevo.circe.magnolia.encoder
import derevo.derive
import eu.timepit.refined.auto._
import eu.timepit.refined.types.numeric.PosInt
Expand Down Expand Up @@ -49,7 +50,7 @@ object BlockValidationParams {
val default: BlockValidationParams = BlockValidationParams(minSignatureCount = 3, minParentCount = 2)
}

@derive(eqv, show)
@derive(eqv, show, encoder)
sealed trait BlockValidationError

case class InvalidTransactionChain(error: TransactionChainBroken) extends BlockValidationError
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import io.constellationnetwork.security.Hasher
import io.constellationnetwork.security.signature.Signed

import derevo.cats.{eqv, show}
import derevo.circe.magnolia.encoder
import derevo.derive

trait TransactionChainValidator[F[_]] {
Expand Down Expand Up @@ -64,7 +65,7 @@ object TransactionChainValidator {
}
}

@derive(eqv, show)
@derive(eqv, show, encoder)
case class TransactionChainBroken(address: Address, referenceNotFound: TransactionReference)

type TransactionNel = NonEmptyList[Signed[Transaction]]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import io.constellationnetwork.security.signature.SignedValidator.SignedValidati
import io.constellationnetwork.security.signature.{Signed, SignedValidator}

import derevo.cats.{eqv, show}
import derevo.circe.magnolia.encoder
import derevo.derive
import eu.timepit.refined.auto._

Expand Down Expand Up @@ -71,7 +72,7 @@ object TransactionValidator {

}

@derive(eqv, show)
@derive(eqv, show, encoder)
sealed trait TransactionValidationError
case class InvalidSigned(error: SignedValidationError) extends TransactionValidationError
case object NotSignedBySourceAddressOwner extends TransactionValidationError
Expand Down
Loading

0 comments on commit 46fb287

Please sign in to comment.