From 13f991ab291700eacf534806a300d093858eed30 Mon Sep 17 00:00:00 2001 From: Denis Mikhaylov Date: Mon, 7 Aug 2017 16:49:28 +0300 Subject: [PATCH] Runtime overhaul and modularization (#17) --- .travis.yml | 4 +- README.md | 126 +++++------ .../akkageneric/GenericAkkaRuntime.scala | 58 +++++ .../akkageneric/GenericAkkaRuntimeActor.scala | 84 +++++++ .../GenericAkkaRuntimeSettings.scala | 16 +- .../AkkaPersistenceRuntime.scala | 101 +++++++++ .../AkkaPersistenceRuntimeActor.scala | 185 ++++++++++------ .../AkkaPersistenceRuntimeSettings.scala | 34 +++ .../CassandraEventJournalQuery.scala | 72 ++++++ .../CassandraOffsetStore.scala | 56 +++-- .../CommittableEventJournalQuery.scala | 52 +++++ .../akkapersistence/EventJournalQuery.scala | 21 ++ .../serialization/Codec.scala | 2 +- .../serialization/PersistentDecoder.scala | 36 +++ .../serialization/PersistentEncoder.scala | 11 +- .../serialization/PersistentRepr.scala | 4 +- .../CassandraSessionInitSerialization.scala | 0 .../aecor/effect/fs2/Fs2TaskInstances.scala | 36 +++ .../main/scala/aecor/effect/fs2/package.scala | 3 + .../effect/monix/MonixTaskInstances.scala | 35 +++ .../scala/aecor/effect/monix/package.scala | 3 + .../main/scala/aecor/testkit/E2eSupport.scala | 101 +++++++++ .../scala/aecor/testkit/EventJournal.scala | 10 + .../scala/aecor/testkit/Eventsourced.scala | 108 +++++++++ .../main/scala/aecor/testkit/StateClock.scala | 30 +++ .../aecor/testkit/StateEventJournal.scala | 112 ++++++++++ .../aecor/testkit/StateKeyValueStore.scala | 29 +++ .../scala/aecor/testkit/StateRuntime.scala | 62 ++++++ build.sbt | 177 ++++++++++++--- core/src/main/resources/reference.conf | 6 +- .../scala/aecor/aggregate/AkkaRuntime.scala | 66 ------ .../main/scala/aecor/aggregate/Folder.scala | 18 -- .../scala/aecor/aggregate/StateRuntime.scala | 51 ----- .../main/scala/aecor/aggregate/Tagging.scala | 44 ---- .../main/scala/aecor/aggregate/package.scala | 20 -- .../serialization/PersistentDecoder.scala | 35 --- core/src/main/scala/aecor/data/Behavior.scala | 43 ++++ .../main/scala/aecor/data/Committable.scala | 55 +++++ .../main/scala/aecor/data/ConsumerId.scala | 5 + .../main/scala/aecor/data/Correlation.scala | 16 ++ core/src/main/scala/aecor/data/EventTag.scala | 2 +- .../aecor/data/EventsourcedBehavior.scala | 8 + core/src/main/scala/aecor/data/Folded.scala | 4 + core/src/main/scala/aecor/data/Folder.scala | 29 +++ core/src/main/scala/aecor/data/Handler.scala | 19 +- core/src/main/scala/aecor/data/Reducer.scala | 6 + core/src/main/scala/aecor/data/Tagging.scala | 39 ++++ core/src/main/scala/aecor/data/package.scala | 11 + core/src/main/scala/aecor/effect/Async.scala | 70 ++++++ .../src/main/scala/aecor/effect/Capture.scala | 23 ++ .../scala/aecor/effect/CaptureFuture.scala | 28 +++ .../aecor/streaming/AggregateJournal.scala | 63 ------ .../streaming/CassandraAggregateJournal.scala | 81 ------- .../scala/aecor/streaming/Committable.scala | 41 ---- .../scala/aecor/streaming/OffsetStore.scala | 10 - .../aecor/streaming/StreamSupervisor.scala | 52 ----- .../streaming/StreamSupervisorActor.scala | 51 ----- core/src/main/scala/aecor/util/Clock.scala | 16 ++ .../main/scala/aecor/util/JavaTimeClock.scala | 15 ++ .../main/scala/aecor/util/KeyValueStore.scala | 22 ++ .../scala/aecor/util/NoopKeyValueStore.scala | 16 ++ .../main/protobuf/DistributedProcessing.proto | 11 + .../src/main/resources/reference.conf | 13 ++ .../AkkaStreamProcess.scala | 28 +++ .../DistributedProcessing.scala | 82 +++++++ .../DistributedProcessingSupervisor.scala | 60 +++++ .../DistributedProcessingWorker.scala | 54 +++++ .../serialization/Message.scala | 6 + .../serialization/MessageSerializer.scala | 27 +++ example/src/main/protobuf/Account.proto | 36 --- example/src/main/resources/application.conf | 21 +- example/src/main/resources/logback.xml | 2 +- .../resources/requests/AuthorizePayment.json | 9 - .../resources/requests/CreditAccount.json | 7 - .../main/resources/requests/OpenAccount.json | 5 - .../main/scala/aecor/example/AccountAPI.scala | 74 ------- .../scala/aecor/example/AccountEndpoint.scala | 44 ++++ .../aecor/example/AnyValCirceEncoding.scala | 26 +++ .../src/main/scala/aecor/example/App.scala | 129 ++++++++++- .../main/scala/aecor/example/AppActor.scala | 121 ---------- .../aecor/example/AuthorizePaymentAPI.scala | 118 ---------- .../scala/aecor/example/EventStream.scala | 109 --------- .../scala/aecor/example/MonixSupport.scala | 22 ++ .../scala/aecor/example/ScheduleApp.scala | 119 ++++++---- .../aecor/example/TransactionEndpoint.scala | 126 +++++++++++ .../example/domain/AccountAggregate.scala | 138 ------------ .../domain/AccountAggregateEvent.scala | 34 --- .../example/domain/AccountAggregateOp.scala | 49 ----- .../scala/aecor/example/domain/Amount.scala | 7 +- .../example/domain/AuthorizationProcess.scala | 72 ------ .../domain/CardAuthorizationAggregate.scala | 138 ------------ .../CardAuthorizationAggregateEvent.scala | 27 --- .../domain/CardAuthorizationAggregateOp.scala | 36 --- .../example/domain/TransactionProcess.scala | 74 +++++++ .../domain/account/AccountAggregate.scala | 52 +++++ .../example/domain/account/AccountEvent.scala | 34 +++ .../example/domain/account/AccountId.scala | 3 + .../EventsourcedAccountAggregate.scala | 107 +++++++++ .../EventsourcedTransactionAggregate.scala | 153 +++++++++++++ .../transaction/TransactionAggregate.scala | 30 +++ .../domain/transaction/TransactionEvent.scala | 36 +++ .../aecor/example/persistentEncoderUtil.scala | 3 +- project/build.properties | 2 +- project/plugins.sbt | 7 +- .../CassandraScheduleEntryRepository.scala | 115 ++++++---- .../aecor/schedule/DefaultSchedule.scala | 64 +++--- .../schedule/DefaultScheduleAggregate.scala | 150 +++++++++++++ .../main/scala/aecor/schedule/Schedule.scala | 161 +++++++------- .../aecor/schedule/ScheduleAggregate.scala | 162 +------------- .../schedule/ScheduleEntryRepository.scala | 14 +- .../aecor/schedule/ScheduleProcess.scala | 141 ------------ .../process/DefaultScheduleEventJournal.scala | 43 ++++ .../process/PeriodicProcessRuntime.scala | 43 ++++ .../process/ScheduleEventJournal.scala | 7 + .../schedule/process/ScheduleProcess.scala | 64 ++++++ .../protobuf/ScheduleEventCodec.scala | 5 +- .../protobuf/ScheduleEventSerializer.scala | 2 +- .../tests/AkkaPersistenceRuntimeSpec.scala | 66 ++++++ .../tests/CompositeCorrelationIdSpec.scala | 2 +- .../tests/DistributedSourceWorkerSpec.scala | 37 ++++ .../test/scala/aecor/tests/EndToEndTest.scala | 207 ++++++++++++++++++ .../aecor/tests/PersistentEncoderCirce.scala | 30 +++ .../aecor/tests/ScheduleAggregateSpec.scala | 5 +- .../aecor/tests/ScheduleEventCodecSpec.scala | 4 +- .../aecor/tests/ShardedRuntimeSpec.scala | 67 ++++++ .../scala/aecor/tests/StateRuntimeSpec.scala | 55 ++--- .../scala/aecor/tests/e2e/CounterOp.scala | 69 ++++++ .../aecor/tests/e2e/CounterViewProcess.scala | 21 ++ .../tests/e2e/CounterViewRepository.scala | 37 ++++ .../aecor/tests/e2e/NotificationProcess.scala | 39 ++++ .../e2e/TestScheduleEntryRepository.scala | 78 +++++++ .../scala/aecor/tests/e2e/notification.scala | 58 +++++ 132 files changed, 4254 insertions(+), 2276 deletions(-) create mode 100644 aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntime.scala create mode 100644 aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeActor.scala rename core/src/main/scala/aecor/aggregate/AkkaRuntimeSettings.scala => aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeSettings.scala (60%) create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntime.scala rename core/src/main/scala/aecor/aggregate/AggregateActor.scala => aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeActor.scala (52%) create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeSettings.scala create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraEventJournalQuery.scala rename {core/src/main/scala/aecor/streaming => aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence}/CassandraOffsetStore.scala (52%) create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CommittableEventJournalQuery.scala create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/EventJournalQuery.scala rename {core/src/main/scala/aecor/aggregate => aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence}/serialization/Codec.scala (85%) create mode 100644 aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentDecoder.scala rename {core/src/main/scala/aecor/aggregate => aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence}/serialization/PersistentEncoder.scala (55%) rename {core/src/main/scala/aecor/aggregate => aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence}/serialization/PersistentRepr.scala (73%) rename {core => aecor-akka-persistence}/src/main/scala/akka/persistence/cassandra/CassandraSessionInitSerialization.scala (100%) create mode 100644 aecor-fs2/src/main/scala/aecor/effect/fs2/Fs2TaskInstances.scala create mode 100644 aecor-fs2/src/main/scala/aecor/effect/fs2/package.scala create mode 100644 aecor-monix/src/main/scala/aecor/effect/monix/MonixTaskInstances.scala create mode 100644 aecor-monix/src/main/scala/aecor/effect/monix/package.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/E2eSupport.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/EventJournal.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/Eventsourced.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/StateClock.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/StateEventJournal.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/StateKeyValueStore.scala create mode 100644 aecor-test-kit/src/main/scala/aecor/testkit/StateRuntime.scala delete mode 100644 core/src/main/scala/aecor/aggregate/AkkaRuntime.scala delete mode 100644 core/src/main/scala/aecor/aggregate/Folder.scala delete mode 100644 core/src/main/scala/aecor/aggregate/StateRuntime.scala delete mode 100644 core/src/main/scala/aecor/aggregate/Tagging.scala delete mode 100644 core/src/main/scala/aecor/aggregate/package.scala delete mode 100644 core/src/main/scala/aecor/aggregate/serialization/PersistentDecoder.scala create mode 100644 core/src/main/scala/aecor/data/Behavior.scala create mode 100644 core/src/main/scala/aecor/data/Committable.scala create mode 100644 core/src/main/scala/aecor/data/ConsumerId.scala create mode 100644 core/src/main/scala/aecor/data/Correlation.scala create mode 100644 core/src/main/scala/aecor/data/EventsourcedBehavior.scala create mode 100644 core/src/main/scala/aecor/data/Folder.scala create mode 100644 core/src/main/scala/aecor/data/Reducer.scala create mode 100644 core/src/main/scala/aecor/data/Tagging.scala create mode 100644 core/src/main/scala/aecor/data/package.scala create mode 100644 core/src/main/scala/aecor/effect/Async.scala create mode 100644 core/src/main/scala/aecor/effect/Capture.scala create mode 100644 core/src/main/scala/aecor/effect/CaptureFuture.scala delete mode 100644 core/src/main/scala/aecor/streaming/AggregateJournal.scala delete mode 100644 core/src/main/scala/aecor/streaming/CassandraAggregateJournal.scala delete mode 100644 core/src/main/scala/aecor/streaming/Committable.scala delete mode 100644 core/src/main/scala/aecor/streaming/OffsetStore.scala delete mode 100644 core/src/main/scala/aecor/streaming/StreamSupervisor.scala delete mode 100644 core/src/main/scala/aecor/streaming/StreamSupervisorActor.scala create mode 100644 core/src/main/scala/aecor/util/Clock.scala create mode 100644 core/src/main/scala/aecor/util/JavaTimeClock.scala create mode 100644 core/src/main/scala/aecor/util/KeyValueStore.scala create mode 100644 core/src/main/scala/aecor/util/NoopKeyValueStore.scala create mode 100644 distributed-processing/src/main/protobuf/DistributedProcessing.proto create mode 100644 distributed-processing/src/main/resources/reference.conf create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/AkkaStreamProcess.scala create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessing.scala create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingSupervisor.scala create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingWorker.scala create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/Message.scala create mode 100644 distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/MessageSerializer.scala delete mode 100644 example/src/main/protobuf/Account.proto delete mode 100644 example/src/main/resources/requests/AuthorizePayment.json delete mode 100644 example/src/main/resources/requests/CreditAccount.json delete mode 100644 example/src/main/resources/requests/OpenAccount.json delete mode 100644 example/src/main/scala/aecor/example/AccountAPI.scala create mode 100644 example/src/main/scala/aecor/example/AccountEndpoint.scala create mode 100644 example/src/main/scala/aecor/example/AnyValCirceEncoding.scala delete mode 100644 example/src/main/scala/aecor/example/AppActor.scala delete mode 100644 example/src/main/scala/aecor/example/AuthorizePaymentAPI.scala delete mode 100644 example/src/main/scala/aecor/example/EventStream.scala create mode 100644 example/src/main/scala/aecor/example/MonixSupport.scala create mode 100644 example/src/main/scala/aecor/example/TransactionEndpoint.scala delete mode 100644 example/src/main/scala/aecor/example/domain/AccountAggregate.scala delete mode 100644 example/src/main/scala/aecor/example/domain/AccountAggregateEvent.scala delete mode 100644 example/src/main/scala/aecor/example/domain/AccountAggregateOp.scala delete mode 100644 example/src/main/scala/aecor/example/domain/AuthorizationProcess.scala delete mode 100644 example/src/main/scala/aecor/example/domain/CardAuthorizationAggregate.scala delete mode 100644 example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateEvent.scala delete mode 100644 example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateOp.scala create mode 100644 example/src/main/scala/aecor/example/domain/TransactionProcess.scala create mode 100644 example/src/main/scala/aecor/example/domain/account/AccountAggregate.scala create mode 100644 example/src/main/scala/aecor/example/domain/account/AccountEvent.scala create mode 100644 example/src/main/scala/aecor/example/domain/account/AccountId.scala create mode 100644 example/src/main/scala/aecor/example/domain/account/EventsourcedAccountAggregate.scala create mode 100644 example/src/main/scala/aecor/example/domain/transaction/EventsourcedTransactionAggregate.scala create mode 100644 example/src/main/scala/aecor/example/domain/transaction/TransactionAggregate.scala create mode 100644 example/src/main/scala/aecor/example/domain/transaction/TransactionEvent.scala create mode 100644 schedule/src/main/scala/aecor/schedule/DefaultScheduleAggregate.scala delete mode 100644 schedule/src/main/scala/aecor/schedule/ScheduleProcess.scala create mode 100644 schedule/src/main/scala/aecor/schedule/process/DefaultScheduleEventJournal.scala create mode 100644 schedule/src/main/scala/aecor/schedule/process/PeriodicProcessRuntime.scala create mode 100644 schedule/src/main/scala/aecor/schedule/process/ScheduleEventJournal.scala create mode 100644 schedule/src/main/scala/aecor/schedule/process/ScheduleProcess.scala create mode 100644 tests/src/test/scala/aecor/tests/AkkaPersistenceRuntimeSpec.scala create mode 100644 tests/src/test/scala/aecor/tests/DistributedSourceWorkerSpec.scala create mode 100644 tests/src/test/scala/aecor/tests/EndToEndTest.scala create mode 100644 tests/src/test/scala/aecor/tests/PersistentEncoderCirce.scala create mode 100644 tests/src/test/scala/aecor/tests/ShardedRuntimeSpec.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/CounterOp.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/CounterViewProcess.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/CounterViewRepository.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/NotificationProcess.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/TestScheduleEntryRepository.scala create mode 100644 tests/src/test/scala/aecor/tests/e2e/notification.scala diff --git a/.travis.yml b/.travis.yml index c0ea35ba..83fbca4d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,7 +1,7 @@ language: scala scala: - - 2.11.8 - - 2.12.0 + - 2.11.11-bin-typelevel-4 + - 2.12.2-bin-typelevel-4 jdk: - oraclejdk8 diff --git a/README.md b/README.md index dcff8375..44a129d5 100644 --- a/README.md +++ b/README.md @@ -26,26 +26,9 @@ scalacOptions += "-Ypartial-unification" ### Defining and running behavior -Let's start with entity operations: +Let's start with defining domain events: ```scala -sealed trait SubscriptionOp[A] { - def subscriptionId: String -} -object SubscriptionOp { - case class CreateSubscription(subscriptionId: String, userId: String, productId: String, planId: String) extends SubscriptionOp[Unit] - case class PauseSubscription(subscriptionId: String) extends SubscriptionOp[Unit] - case class ResumeSubscription(subscriptionId: String) extends SubscriptionOp[Unit] - case class CancelSubscription(subscriptionId: String) extends SubscriptionOp[Unit] -} -``` - -Entity events with persistent encoder and decoder: - -```scala -import aecor.data.Folded.syntax._ -import cats.syntax.option._ - sealed trait SubscriptionEvent object SubscriptionEvent { case class SubscriptionCreated(subscriptionId: String, userId: String, productId: String, planId: String) extends SubscriptionEvent @@ -58,10 +41,12 @@ object SubscriptionEvent { } ``` +Then we define how this events affect aggregate state: + `Folder[F, E, S]` instance represents the ability to fold `E`s into `S`, with effect `F` on each step. Aecor runtime uses `Folded[A]` data type, with two possible states: -`Next(a)` - says that a should be used as a state for next folding step -`Impossible` - says that folding should be aborted (underlying runtime actor throws `IllegalStateException`) +`Next(a)` - says that `a` should be used as a state for next folding step +`Impossible` - says that folding should be aborted (e.g. underlying runtime actor throws `IllegalStateException`) ```scala sealed trait SubscriptionStatus @@ -71,89 +56,104 @@ object SubscriptionStatus { case object Cancelled extends SubscriptionStatus } -case class Subscription(status: SubscriptionStatus) +case class Subscription(status: SubscriptionStatus) { + def applyEvent(e: SubscriptionEvent): Folded[Subscription] = e match { + case e: SubscriptionCreated => + impossible + case e: SubscriptionPaused => + subscription.copy(status = Paused).next + case e: SubscriptionResumed => + subscription.copy(status = Active).next + case e: SubscriptionCancelled => + subscription.copy(status = Cancelled).next + } +} object Subscription { import SubscriptionStatus._ - - implicit def folder: Folder[Folded, SubscriptionEvent, Option[Subscription]] = - Folder.instance(Option.empty[Subscription]) { - case Some(subscription) => { - case e: SubscriptionCreated => - impossible - case e: SubscriptionPaused => - subscription.copy(status = Paused).some.next - case e: SubscriptionResumed => - subscription.copy(status = Active).some.next - case e: SubscriptionCancelled => - subscription.copy(status = Cancelled).some.next - } - case None => { - case SubscriptionCreated(subscriptionId, userId, productId, planId) => - Subscription(Active).some.next - case _ => - impossible - } - } + def init(e: SubscriptionEvent): Folded[Subscription] = e match { + case SubscriptionCreated(subscriptionId, userId, productId, planId) => + Subscription(Active).next + case _ => impossible + } + def folder: Folder[Folded, SubscriptionEvent, Option[Subscription]] = + Folder.optionInstance(init)(_.applyEvent) } - ``` + Now let's define a behavior that converts operation to its handler. -A `Handler[State, Event, Reply]` is just a wrapper around `State => (Seq[Event], Reply)`, +A `Handler[F, State, Event, Reply]` is just a wrapper around `State => (Seq[Event], Reply)`, you can think of it as `Kleisli[(Seq[Event], ?), State, Reply]` i.e. a side-effecting function with a side effect being a sequence of events representing state change caused by operation. ```scala -val behavior = Lambda[SubscriptionOp ~> Handler[Option[Subscription], SubscriptionEvent, ?]] { + +sealed trait SubscriptionOp[A] { + def subscriptionId: String +} +object SubscriptionOp { + case class CreateSubscription(subscriptionId: String, userId: String, productId: String, planId: String) extends SubscriptionOp[Unit] + case class PauseSubscription(subscriptionId: String) extends SubscriptionOp[Unit] + case class ResumeSubscription(subscriptionId: String) extends SubscriptionOp[Unit] + case class CancelSubscription(subscriptionId: String) extends SubscriptionOp[Unit] +} + +def handler[F[_]](implicit F: Applicative[F]) = Lambda[SubscriptionOp ~> Handler[F, Option[Subscription], SubscriptionEvent, ?]] { case CreateSubscription(subscriptionId, userId, productId, planId) => { case Some(subscription) => // Do nothing reply with () - Seq.empty -> () + F.pure(Seq.empty -> ()) case None => // Produce event and reply with () - Seq(SubscriptionCreated(subscriptionId, userId, productId, planId)) -> () + F.pure(Seq(SubscriptionCreated(subscriptionId, userId, productId, planId)) -> ()) } case PauseSubscription(subscriptionId) => { case Some(subscription) if subscription.status == Active => - Seq(SubscriptionPaused(subscriptionId)) -> () + F.pure(Seq(SubscriptionPaused(subscriptionId)) -> ()) case _ => - Seq.empty -> () + F.pure(Seq.empty -> ()) } case ResumeSubscription(subscriptionId) => { case Some(subscription) if subscription.status == Paused => - Seq(SubscriptionResumed(subscriptionId)) -> () + F.pure(Seq(SubscriptionResumed(subscriptionId)) -> ()) case _ => - Seq.empty -> () + F.pure(Seq.empty -> ()) } case CancelSubscription(subscriptionId) => { case Some(subscription) => - Seq(SubscriptionCancelled(subscriptionId)) -> () + F.pure(Seq(SubscriptionCancelled(subscriptionId)) -> ()) case _ => - Seq.empty -> () + F.pure(Seq.empty -> ()) } } ``` Then you define a correlation function, entity name and a value provided by correlation function form unique primary key for aggregate. -It should not be changed in the future, at least without prior event migration. +It should not be changed between releases, at least without prior event migration. ```scala -def correlation: Correlation[SubscriptionOp] = { - def mk[A](op: SubscriptionOp[A]): CorrelationF[A] = op.subscriptionId - FunctionK.lift(mk _) -} +def correlation: Correlation[SubscriptionOp] = _.subscriptionId ``` After that we are ready to launch. ```scala -implicit val system = ActorSystem("foo") +import monix.cats._ +import monix.eval.Task +import aecor.effect.monix._ +import aecor.runtime.akkapersistence.AkkaPersistenceRuntime + +val system = ActorSystem("foo") -val subscriptions: SubscriptionOp ~> Future = - AkkaRuntime(system).start( - entityName = "Subscription", - behavior, +val subscriptions: SubscriptionOp ~> Task = + AkkaPersistenceRuntime( + system, + "Subscription", correlation, - Tagging(EventTag("Payment") + EventsourcedBehavior( + handler, + Subscription.folder + ), + Tagging.const[SubscriptionEvent](EventTag("Payment")) ) ``` diff --git a/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntime.scala b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntime.scala new file mode 100644 index 00000000..aff20336 --- /dev/null +++ b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntime.scala @@ -0,0 +1,58 @@ +package aecor.runtime.akkageneric + +import aecor.data.{ Behavior, Correlation } +import aecor.effect.{ Async, Capture, CaptureFuture } +import aecor.runtime.akkageneric.GenericAkkaRuntime.CorrelatedCommand +import akka.actor.ActorSystem +import akka.cluster.sharding.{ ClusterSharding, ShardRegion } +import akka.pattern._ +import akka.util.Timeout +import cats.~> + +import scala.concurrent.Future + +object GenericAkkaRuntime { + def apply[F[_]: Async: CaptureFuture: Capture](system: ActorSystem): GenericAkkaRuntime[F] = + new GenericAkkaRuntime(system) + private final case class CorrelatedCommand[A](entityId: String, command: A) +} + +class GenericAkkaRuntime[F[_]: Async: CaptureFuture: Capture](system: ActorSystem) { + def start[Op[_]](typeName: String, + correlation: Correlation[Op], + behavior: Behavior[F, Op], + settings: GenericAkkaRuntimeSettings = + GenericAkkaRuntimeSettings.default(system)): F[Op ~> F] = + Capture[F] + .capture { + val numberOfShards = settings.numberOfShards + + val extractEntityId: ShardRegion.ExtractEntityId = { + case CorrelatedCommand(entityId, c) => + (entityId, GenericAkkaRuntimeActor.PerformOp(c.asInstanceOf[Op[_]])) + } + + val extractShardId: ShardRegion.ExtractShardId = { + case CorrelatedCommand(entityId, _) => + (scala.math.abs(entityId.hashCode) % numberOfShards).toString + case other => throw new IllegalArgumentException(s"Unexpected message [$other]") + } + + val props = GenericAkkaRuntimeActor.props(behavior, settings.idleTimeout) + + val shardRegionRef = ClusterSharding(system).start( + typeName = typeName, + entityProps = props, + settings = settings.clusterShardingSettings, + extractEntityId = extractEntityId, + extractShardId = extractShardId + ) + + implicit val timeout = Timeout(settings.askTimeout) + new (Op ~> F) { + override def apply[A](fa: Op[A]): F[A] = CaptureFuture[F].captureFuture { + (shardRegionRef ? CorrelatedCommand(correlation(fa), fa)).asInstanceOf[Future[A]] + } + } + } +} diff --git a/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeActor.scala b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeActor.scala new file mode 100644 index 00000000..bb282e1c --- /dev/null +++ b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeActor.scala @@ -0,0 +1,84 @@ +package aecor.runtime.akkageneric + +import java.util.UUID + +import aecor.data.Behavior +import aecor.effect.Async +import aecor.effect.Async.ops._ +import aecor.runtime.akkageneric.GenericAkkaRuntimeActor.PerformOp +import akka.actor.{ Actor, ActorLogging, Props, ReceiveTimeout, Stash, Status } +import akka.cluster.sharding.ShardRegion +import akka.pattern.pipe + +import scala.concurrent.duration.FiniteDuration +import scala.util.control.NonFatal +import scala.util.{ Failure, Success, Try } + +object GenericAkkaRuntimeActor { + def props[F[_]: Async, Op[_]](behavior: Behavior[F, Op], idleTimeout: FiniteDuration): Props = + Props(new GenericAkkaRuntimeActor(behavior, idleTimeout)) + + final case class PerformOp[Op[_], A](op: Op[A]) + case object Stop +} + +private[aecor] final class GenericAkkaRuntimeActor[F[_]: Async, Op[_]](behavior: Behavior[F, Op], + idleTimeout: FiniteDuration) + extends Actor + with Stash + with ActorLogging { + + import context._ + + private case class Result(id: UUID, value: Try[(Behavior[F, Op], Any)]) + + setIdleTimeout() + + override def receive: Receive = withBehavior(behavior) + + private def withBehavior(behavior: Behavior[F, Op]): Receive = { + case PerformOp(op) => + val opId = UUID.randomUUID() + behavior + .run(op.asInstanceOf[Op[Any]]) + .unsafeRun + .map(x => Result(opId, Success(x))) + .recover { + case NonFatal(e) => Result(opId, Failure(e)) + } + .pipeTo(self)(sender) + + become { + case Result(`opId`, value) => + value match { + case Success((newBehavior, reply)) => + sender() ! reply + become(withBehavior(newBehavior)) + case Failure(cause) => + sender() ! Status.Failure(cause) + throw cause + } + unstashAll() + case _ => + stash() + } + case ReceiveTimeout => + passivate() + case GenericAkkaRuntimeActor.Stop => + context.stop(self) + case Result(_, _) => + log.debug( + "Ignoring result of another operation. Probably targeted previous instance of actor." + ) + } + + private def passivate(): Unit = { + log.debug("Passivating...") + context.parent ! ShardRegion.Passivate(GenericAkkaRuntimeActor.Stop) + } + + private def setIdleTimeout(): Unit = { + log.debug("Setting idle timeout to [{}]", idleTimeout) + context.setReceiveTimeout(idleTimeout) + } +} diff --git a/core/src/main/scala/aecor/aggregate/AkkaRuntimeSettings.scala b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeSettings.scala similarity index 60% rename from core/src/main/scala/aecor/aggregate/AkkaRuntimeSettings.scala rename to aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeSettings.scala index 8c63541c..2040c808 100644 --- a/core/src/main/scala/aecor/aggregate/AkkaRuntimeSettings.scala +++ b/aecor-akka-generic/src/main/scala/aecor/runtime/akkageneric/GenericAkkaRuntimeSettings.scala @@ -1,4 +1,4 @@ -package aecor.aggregate +package aecor.runtime.akkageneric import java.util.concurrent.TimeUnit @@ -7,24 +7,24 @@ import akka.cluster.sharding.ClusterShardingSettings import scala.concurrent.duration._ -final case class AkkaRuntimeSettings(numberOfShards: Int, - idleTimeout: FiniteDuration, - askTimeout: FiniteDuration, - clusterShardingSettings: ClusterShardingSettings) +final case class GenericAkkaRuntimeSettings(numberOfShards: Int, + idleTimeout: FiniteDuration, + askTimeout: FiniteDuration, + clusterShardingSettings: ClusterShardingSettings) -object AkkaRuntimeSettings { +object GenericAkkaRuntimeSettings { /** * Reads config from `aecor.akka-runtime`, see reference.conf for details * @param system Actor system to get config from * @return default settings */ - def default(system: ActorSystem): AkkaRuntimeSettings = { + def default(system: ActorSystem): GenericAkkaRuntimeSettings = { val config = system.settings.config.getConfig("aecor.akka-runtime") def getMillisDuration(path: String): FiniteDuration = Duration(config.getDuration(path, TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS) - AkkaRuntimeSettings( + GenericAkkaRuntimeSettings( config.getInt("number-of-shards"), getMillisDuration("idle-timeout"), getMillisDuration("ask-timeout"), diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntime.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntime.scala new file mode 100644 index 00000000..719c51b0 --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntime.scala @@ -0,0 +1,101 @@ +package aecor.runtime.akkapersistence + +import java.util.UUID + +import aecor.data._ +import aecor.effect.{ Async, Capture, CaptureFuture } +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentEncoder } +import akka.actor.ActorSystem +import akka.cluster.sharding.{ ClusterSharding, ShardRegion } +import akka.pattern.ask +import akka.util.Timeout +import cats.{ Monad, ~> } + +import scala.concurrent.Future + +object AkkaPersistenceRuntime { + def apply[F[_]: Async: CaptureFuture: Capture: Monad, Op[_], State, Event: PersistentEncoder: PersistentDecoder]( + system: ActorSystem, + entityName: String, + correlation: Correlation[Op], + behavior: EventsourcedBehavior[F, Op, State, Event], + tagging: Tagging[Event], + onPersisted: Option[Event => F[Unit]] = None, + snapshotPolicy: SnapshotPolicy[State] = SnapshotPolicy.never, + settings: Option[AkkaPersistenceRuntimeSettings] = None + ): AkkaPersistenceRuntime[F, Op, State, Event] = { + val pureUnit = Monad[F].pure(()) + new AkkaPersistenceRuntime[F, Op, State, Event]( + system, + entityName, + correlation, + behavior, + tagging, + onPersisted.getOrElse(_ => pureUnit), + snapshotPolicy, + settings + ) + } + + private final case class CorrelatedCommand[C[_], A](entityId: String, command: C[A]) +} + +class AkkaPersistenceRuntime[F[_]: Async: CaptureFuture: Capture: Monad, Op[_], State, Event: PersistentEncoder: PersistentDecoder]( + system: ActorSystem, + entityName: String, + correlation: Correlation[Op], + behavior: EventsourcedBehavior[F, Op, State, Event], + tagging: Tagging[Event], + onPersisted: Event => F[Unit], + snapshotPolicy: SnapshotPolicy[State] = SnapshotPolicy.never, + customSettings: Option[AkkaPersistenceRuntimeSettings] = None +) { + + import AkkaPersistenceRuntime._ + def start: F[Op ~> F] = { + val settings = customSettings.getOrElse(AkkaPersistenceRuntimeSettings.default(system)) + val props = + AkkaPersistenceRuntimeActor.props( + entityName, + behavior, + snapshotPolicy, + tagging, + onPersisted, + settings.idleTimeout + ) + + def extractEntityId: ShardRegion.ExtractEntityId = { + case CorrelatedCommand(entityId, c) => + (entityId, AkkaPersistenceRuntimeActor.HandleCommand(c)) + } + + val numberOfShards = settings.numberOfShards + + def extractShardId: ShardRegion.ExtractShardId = { + case CorrelatedCommand(entityId, _) => + (scala.math.abs(entityId.hashCode) % numberOfShards).toString + case other => throw new IllegalArgumentException(s"Unexpected message [$other]") + } + + def startShardRegion = ClusterSharding(system).start( + typeName = entityName, + entityProps = props, + settings = settings.clusterShardingSettings, + extractEntityId = extractEntityId, + extractShardId = extractShardId + ) + + Capture[F].capture { + val regionRef = startShardRegion + new (Op ~> F) { + implicit private val timeout = Timeout(settings.askTimeout) + override def apply[A](fa: Op[A]): F[A] = + CaptureFuture[F].captureFuture { + (regionRef ? CorrelatedCommand(correlation(fa), fa)).asInstanceOf[Future[A]] + } + } + } + } + + def journal: EventJournalQuery[UUID, Event] = CassandraEventJournalQuery[Event](system) +} diff --git a/core/src/main/scala/aecor/aggregate/AggregateActor.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeActor.scala similarity index 52% rename from core/src/main/scala/aecor/aggregate/AggregateActor.scala rename to aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeActor.scala index 6ed1a3ff..45fae826 100644 --- a/core/src/main/scala/aecor/aggregate/AggregateActor.scala +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeActor.scala @@ -1,20 +1,28 @@ -package aecor.aggregate +package aecor.runtime.akkapersistence import java.net.URLDecoder import java.nio.charset.StandardCharsets import java.time.{ Duration, Instant } - -import aecor.aggregate.AggregateActor.HandleCommand -import aecor.aggregate.SnapshotPolicy.{ EachNumberOfEvents, Never } -import aecor.aggregate.serialization.PersistentDecoder.Result -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentEncoder, PersistentRepr } -import aecor.data.{ Folded, Handler } -import akka.actor.{ ActorLogging, Props, ReceiveTimeout } +import java.util.UUID + +import aecor.data._ +import aecor.effect.Async +import aecor.effect.Async.ops._ +import aecor.runtime.akkapersistence.AkkaPersistenceRuntimeActor.HandleCommand +import aecor.runtime.akkapersistence.SnapshotPolicy.{ EachNumberOfEvents, Never } +import aecor.runtime.akkapersistence.serialization.PersistentDecoder.DecodingResult +import aecor.runtime.akkapersistence.serialization.{ + PersistentDecoder, + PersistentEncoder, + PersistentRepr +} +import akka.actor.{ ActorLogging, Props, ReceiveTimeout, Stash, Status } import akka.cluster.sharding.ShardRegion +import akka.pattern.pipe import akka.persistence.journal.Tagged import akka.persistence.{ PersistentActor, RecoveryCompleted, SnapshotOffer } -import cats.~> +import scala.collection.immutable.Seq import scala.concurrent.duration.FiniteDuration import scala.util.{ Left, Right } @@ -27,35 +35,38 @@ object SnapshotPolicy { numberOfEvents: Int ): SnapshotPolicy[E] = EachNumberOfEvents(numberOfEvents) - private[aggregate] case object Never extends SnapshotPolicy[Nothing] + private[akkapersistence] case object Never extends SnapshotPolicy[Nothing] - private[aggregate] final case class EachNumberOfEvents[ + private[akkapersistence] final case class EachNumberOfEvents[ State: PersistentEncoder: PersistentDecoder ](numberOfEvents: Int) extends SnapshotPolicy[State] { def encode(state: State): PersistentRepr = PersistentEncoder[State].encode(state) - def decode(repr: PersistentRepr): Result[State] = PersistentDecoder[State].decode(repr) + def decode(repr: PersistentRepr): DecodingResult[State] = PersistentDecoder[State].decode(repr) } } -sealed abstract class Identity extends Product with Serializable -object Identity { - final case class Provided(value: String) extends Identity - case object FromPathName extends Identity -} - -object AggregateActor { +object AkkaPersistenceRuntimeActor { - def props[Command[_], State, Event: PersistentEncoder: PersistentDecoder]( + def props[F[_]: Async, Op[_], State, Event: PersistentEncoder: PersistentDecoder]( entityName: String, - behavior: Command ~> Handler[State, Event, ?], - identity: Identity, + behavior: EventsourcedBehavior[F, Op, State, Event], snapshotPolicy: SnapshotPolicy[State], tagging: Tagging[Event], + onPersisted: Event => F[Unit], idleTimeout: FiniteDuration - )(implicit folder: Folder[Folded, Event, State]): Props = - Props(new AggregateActor(entityName, behavior, identity, snapshotPolicy, tagging, idleTimeout)) + ): Props = + Props( + new AkkaPersistenceRuntimeActor( + entityName, + behavior, + snapshotPolicy, + tagging, + onPersisted, + idleTimeout + ) + ) final case class HandleCommand[C[_], A](command: C[A]) case object Stop @@ -67,50 +78,55 @@ object AggregateActor { * * @param entityName entity name used as persistence prefix and as a tag for all events * @param behavior entity behavior - * @param identity describes how to extract entity identifier * @param snapshotPolicy snapshot policy to use * @param idleTimeout - time with no commands after which graceful actor shutdown is initiated */ -class AggregateActor[Command[_], State, Event: PersistentEncoder: PersistentDecoder] private[aecor] ( +final class AkkaPersistenceRuntimeActor[F[_]: Async, Op[_], State, Event: PersistentEncoder: PersistentDecoder] private[aecor] ( entityName: String, - behavior: Command ~> Handler[State, Event, ?], - identity: Identity, + behavior: EventsourcedBehavior[F, Op, State, Event], snapshotPolicy: SnapshotPolicy[State], tagger: Tagging[Event], + onPersist: Event => F[Unit], idleTimeout: FiniteDuration -)(implicit folder: Folder[Folded, Event, State]) - extends PersistentActor - with ActorLogging { - - final private val entityId: String = identity match { - case Identity.Provided(value) => value - case Identity.FromPathName => - URLDecoder.decode(self.path.name, StandardCharsets.UTF_8.name()) - } +) extends PersistentActor + with ActorLogging + with Stash { + + import context.dispatcher + + case class CommandResult[A](opId: UUID, events: Seq[Event], reply: A) + + private val entityId: String = + URLDecoder.decode(self.path.name, StandardCharsets.UTF_8.name()) private val eventEncoder = PersistentEncoder[Event] private val eventDecoder = PersistentDecoder[Event] - final override val persistenceId: String = s"$entityName-$entityId" + override val persistenceId: String = s"$entityName-$entityId" private val recoveryStartTimestamp: Instant = Instant.now() log.info("[{}] Starting...", persistenceId) - protected var state: State = folder.zero + protected var state: State = behavior.folder.zero private var eventCount = 0L private var snapshotPending = false - final override def receiveRecover: Receive = { + private def recover(repr: PersistentRepr): Unit = + eventDecoder.decode(repr) match { + case Left(cause) => + onRecoveryFailure(cause, Some(repr)) + case Right(event) => + applyEvent(event) + eventCount += 1 + } + override def receiveRecover: Receive = { case repr: PersistentRepr => - eventDecoder.decode(repr) match { - case Left(cause) => - onRecoveryFailure(cause, Some(repr)) - case Right(event) => - applyEvent(event) - eventCount += 1 - } + recover(repr) + + case Tagged(repr: PersistentRepr, _) => + recover(repr) case SnapshotOffer(_, snapshotRepr: PersistentRepr) => snapshotPolicy match { @@ -132,31 +148,66 @@ class AggregateActor[Command[_], State, Event: PersistentEncoder: PersistentDeco Duration.between(recoveryStartTimestamp, Instant.now()).toMillis ) setIdleTimeout() + + case other => + throw new IllegalStateException(s"Unexpected message during recovery [$other]") } - final override def receiveCommand: Receive = { + override def receiveCommand: Receive = { case HandleCommand(command) => - handleCommand(command.asInstanceOf[Command[_]]) + handleCommand(command.asInstanceOf[Op[_]]) case ReceiveTimeout => passivate() - case AggregateActor.Stop => + case AkkaPersistenceRuntimeActor.Stop => context.stop(self) + case CommandResult(opId, events, reply) => + log.debug( + "[{}] Received result of unknown command invocation [{}], ignoring reply [{}] and events [{}]", + persistenceId, + opId, + reply, + events + ) } - private def handleCommand(command: Command[_]): Unit = { - val (events, reply) = behavior(command).run(state) - log.debug( - "[{}] Command [{}] produced reply [{}] and events [{}]", - persistenceId, - command, - reply, - events - ) + private def handleCommand(command: Op[_]): Unit = { + val opId = UUID.randomUUID() + behavior + .handler(command) + .run(state) + .unsafeRun + .map { + case (events, reply) => + CommandResult(opId, events, reply) + } + .pipeTo(self)(sender) + context.become { + case CommandResult(`opId`, events, reply) => + log.debug( + "[{}] Command [{}] produced reply [{}] and events [{}]", + persistenceId, + command, + reply, + events + ) + handleCommandResult(events, reply) + unstashAll() + context.become(receiveCommand) + case Status.Failure(e) => + sender() ! Status.Failure(e) + unstashAll() + context.become(receiveCommand) + case _ => + stash() + } + } + + private def handleCommandResult[A](events: Seq[Event], reply: A): Unit = if (events.isEmpty) { sender() ! reply } else { val envelopes = - events.map(e => Tagged(eventEncoder.encode(e), tagger(e))) + events.map(e => Tagged(eventEncoder.encode(e), tagger(e).map(_.value))) events.foreach(applyEvent) @@ -167,6 +218,7 @@ class AggregateActor[Command[_], State, Event: PersistentEncoder: PersistentDeco eventCount += 1 markSnapshotAsPendingIfNeeded() snapshotIfPending() + events.map(onPersist).foreach(_.unsafeRun) } } else { persistAll(envelopes) { _ => @@ -177,22 +229,19 @@ class AggregateActor[Command[_], State, Event: PersistentEncoder: PersistentDeco sender() ! reply snapshotIfPending() } + events.map(onPersist).foreach(_.unsafeRun) } } } - } - private def applyEvent(event: Event): Unit = { - state = folder - .fold(state, event) + private def applyEvent(event: Event): Unit = + state = behavior.folder + .reduce(state, event) .getOrElse { val error = new IllegalStateException(s"Illegal state after applying [$event] to [$state]") log.error(error, error.getMessage) throw error } - if (recoveryFinished) - log.debug("[{}] Current state [{}]", persistenceId, state) - } private def markSnapshotAsPendingIfNeeded(): Unit = snapshotPolicy match { @@ -211,7 +260,7 @@ class AggregateActor[Command[_], State, Event: PersistentEncoder: PersistentDeco private def passivate(): Unit = { log.debug("[{}] Passivating...", persistenceId) - context.parent ! ShardRegion.Passivate(AggregateActor.Stop) + context.parent ! ShardRegion.Passivate(AkkaPersistenceRuntimeActor.Stop) } private def setIdleTimeout(): Unit = { diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeSettings.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeSettings.scala new file mode 100644 index 00000000..715e68cc --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/AkkaPersistenceRuntimeSettings.scala @@ -0,0 +1,34 @@ +package aecor.runtime.akkapersistence + +import java.util.concurrent.TimeUnit + +import akka.actor.ActorSystem +import akka.cluster.sharding.ClusterShardingSettings + +import scala.concurrent.duration._ + +final case class AkkaPersistenceRuntimeSettings(numberOfShards: Int, + idleTimeout: FiniteDuration, + askTimeout: FiniteDuration, + clusterShardingSettings: ClusterShardingSettings) + +object AkkaPersistenceRuntimeSettings { + + /** + * Reads config from `aecor.akka-runtime`, see reference.conf for details + * @param system Actor system to get config from + * @return default settings + */ + def default(system: ActorSystem): AkkaPersistenceRuntimeSettings = { + val config = system.settings.config.getConfig("aecor.akka-runtime") + def getMillisDuration(path: String): FiniteDuration = + Duration(config.getDuration(path, TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS) + + AkkaPersistenceRuntimeSettings( + config.getInt("number-of-shards"), + getMillisDuration("idle-timeout"), + getMillisDuration("ask-timeout"), + ClusterShardingSettings(system) + ) + } +} diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraEventJournalQuery.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraEventJournalQuery.scala new file mode 100644 index 00000000..21d59699 --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraEventJournalQuery.scala @@ -0,0 +1,72 @@ +package aecor.runtime.akkapersistence + +import java.util.UUID + +import aecor.data.EventTag +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentRepr } +import akka.NotUsed +import akka.actor.ActorSystem +import akka.persistence.cassandra.query.scaladsl.CassandraReadJournal +import akka.persistence.query._ +import akka.stream.scaladsl.Source + +import scala.concurrent.Future + +class CassandraEventJournalQuery[E: PersistentDecoder](system: ActorSystem, parallelism: Int) + extends EventJournalQuery[UUID, E] { + + private val readJournal = + PersistenceQuery(system).readJournalFor[CassandraReadJournal](CassandraReadJournal.Identifier) + + private def createSource( + inner: Source[EventEnvelope, NotUsed] + ): Source[JournalEntry[UUID, E], NotUsed] = + inner.mapAsync(parallelism) { + case EventEnvelope(offset, persistenceId, sequenceNr, event) => + offset match { + case TimeBasedUUID(offsetValue) => + event match { + case repr: PersistentRepr => + PersistentDecoder[E] + .decode(repr) + .right + .map { event => + JournalEntry(offsetValue, persistenceId, sequenceNr, event) + } + .fold(Future.failed, Future.successful) + case other => + Future.failed( + new IllegalArgumentException( + s"Unexpected persistent representation [$other] at sequenceNr = [$sequenceNr], persistenceId = [$persistenceId]" + ) + ) + } + case other => + Future.failed( + new IllegalArgumentException( + s"Unexpected offset of type [${other.getClass}] at sequenceNr = [$sequenceNr], persistenceId = [$persistenceId]" + ) + ) + } + } + + def eventsByTag(tag: EventTag, offset: Option[UUID]): Source[JournalEntry[UUID, E], NotUsed] = + createSource( + readJournal + .eventsByTag(tag.value, offset.map(TimeBasedUUID).getOrElse(NoOffset)) + ) + + override def currentEventsByTag(tag: EventTag, + offset: Option[UUID]): Source[JournalEntry[UUID, E], NotUsed] = + createSource( + readJournal + .currentEventsByTag(tag.value, offset.map(TimeBasedUUID).getOrElse(NoOffset)) + ) + +} + +object CassandraEventJournalQuery { + def apply[E: PersistentDecoder](system: ActorSystem, + decodingParallelism: Int = 8): EventJournalQuery[UUID, E] = + new CassandraEventJournalQuery(system, decodingParallelism) +} diff --git a/core/src/main/scala/aecor/streaming/CassandraOffsetStore.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraOffsetStore.scala similarity index 52% rename from core/src/main/scala/aecor/streaming/CassandraOffsetStore.scala rename to aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraOffsetStore.scala index beadf060..1e0e1e98 100644 --- a/core/src/main/scala/aecor/streaming/CassandraOffsetStore.scala +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CassandraOffsetStore.scala @@ -1,10 +1,14 @@ -package aecor.streaming +package aecor.runtime.akkapersistence import java.util.UUID +import aecor.data.TagConsumer +import aecor.effect.{ Async, CaptureFuture } +import aecor.util.KeyValueStore +import akka.persistence.cassandra._ import akka.persistence.cassandra.session.scaladsl.CassandraSession import com.datastax.driver.core.Session -import akka.persistence.cassandra._ + import scala.concurrent.{ ExecutionContext, Future } object CassandraOffsetStore { @@ -21,36 +25,42 @@ object CassandraOffsetStore { implicit executionContext: ExecutionContext ): Session => Future[Unit] = _.executeAsync(config.createTableQuery).asScala.map(_ => ()) - def apply(session: CassandraSession, config: CassandraOffsetStore.Config)( - implicit executionContext: ExecutionContext - ): CassandraOffsetStore = + def apply[F[_]: Async: CaptureFuture]( + session: CassandraSession, + config: CassandraOffsetStore.Config + )(implicit executionContext: ExecutionContext): CassandraOffsetStore[F] = new CassandraOffsetStore(session, config) + } -class CassandraOffsetStore(session: CassandraSession, config: CassandraOffsetStore.Config)( - implicit executionContext: ExecutionContext -) extends OffsetStore[UUID] { +class CassandraOffsetStore[F[_]: Async: CaptureFuture]( + session: CassandraSession, + config: CassandraOffsetStore.Config +)(implicit executionContext: ExecutionContext) + extends KeyValueStore[F, TagConsumer, UUID] { private val selectOffsetStatement = session.prepare(config.selectOffsetQuery) private val updateOffsetStatement = session.prepare(config.updateOffsetQuery) - override def getOffset(tag: String, consumerId: ConsumerId): Future[Option[UUID]] = + override def setValue(key: TagConsumer, value: UUID): F[Unit] = + CaptureFuture[F].captureFuture { + updateOffsetStatement + .map { stmt => + stmt + .bind() + .setUUID("offset", value) + .setString("tag", key.tag.value) + .setString("consumer_id", key.consumerId.value) + } + .flatMap(session.executeWrite) + .map(_ => ()) + } + + override def getValue(key: TagConsumer): F[Option[UUID]] = CaptureFuture[F].captureFuture { selectOffsetStatement - .map(_.bind(consumerId.value, tag)) + .map(_.bind(key.consumerId.value, key.tag.value)) .flatMap(session.selectOne) .map(_.map(_.getUUID("offset"))) - - override def setOffset(tag: String, consumerId: ConsumerId, offset: UUID): Future[Unit] = - updateOffsetStatement - .map { stmt => - stmt - .bind() - .setUUID("offset", offset) - .setString("tag", tag) - .setString("consumer_id", consumerId.value) - } - .flatMap(session.executeWrite) - .map(_ => ()) - + } } diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CommittableEventJournalQuery.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CommittableEventJournalQuery.scala new file mode 100644 index 00000000..d1923365 --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/CommittableEventJournalQuery.scala @@ -0,0 +1,52 @@ +package aecor.runtime.akkapersistence + +import aecor.data.{ Committable, ConsumerId, EventTag, TagConsumer } +import aecor.effect.Async +import aecor.util.KeyValueStore +import akka.NotUsed +import akka.stream.scaladsl.Source +import aecor.effect.Async.ops._ + +final class CommittableEventJournalQuery[F[_]: Async, O, E]( + underlying: EventJournalQuery[O, E], + offsetStore: KeyValueStore[F, TagConsumer, O] +) { + + def eventsByTag(tag: EventTag, + consumerId: ConsumerId): Source[Committable[F, JournalEntry[O, E]], NotUsed] = { + val tagConsumerId = TagConsumer(tag, consumerId) + Source + .single(NotUsed) + .mapAsync(1) { _ => + offsetStore.getValue(tagConsumerId).unsafeRun + } + .flatMapConcat { storedOffset => + underlying.eventsByTag(tag, storedOffset) + } + .map(x => Committable(offsetStore.setValue(tagConsumerId, x.offset), x)) + } + + def currentEventsByTag( + tag: EventTag, + consumerId: ConsumerId + ): Source[Committable[F, JournalEntry[O, E]], NotUsed] = { + val tagConsumerId = TagConsumer(tag, consumerId) + Source + .single(NotUsed) + .mapAsync(1) { _ => + offsetStore.getValue(tagConsumerId).unsafeRun + } + .flatMapConcat { storedOffset => + underlying.currentEventsByTag(tag, storedOffset) + } + .map(x => Committable(offsetStore.setValue(tagConsumerId, x.offset), x)) + } +} + +object CommittableEventJournalQuery { + def apply[F[_]: Async, Offset, E]( + underlying: EventJournalQuery[Offset, E], + offsetStore: KeyValueStore[F, TagConsumer, Offset] + ): CommittableEventJournalQuery[F, Offset, E] = + new CommittableEventJournalQuery(underlying, offsetStore) +} diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/EventJournalQuery.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/EventJournalQuery.scala new file mode 100644 index 00000000..b1a726fd --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/EventJournalQuery.scala @@ -0,0 +1,21 @@ +package aecor.runtime.akkapersistence + +import aecor.data.{ EventTag, TagConsumer } +import aecor.effect.Async +import aecor.util.KeyValueStore +import akka.NotUsed +import akka.stream.scaladsl.Source + +final case class JournalEntry[+O, +A](offset: O, persistenceId: String, sequenceNr: Long, event: A) + +trait EventJournalQuery[Offset, E] { + def eventsByTag(tag: EventTag, offset: Option[Offset]): Source[JournalEntry[Offset, E], NotUsed] + + def currentEventsByTag(tag: EventTag, + offset: Option[Offset]): Source[JournalEntry[Offset, E], NotUsed] + + def committable[F[_]: Async]( + offsetStore: KeyValueStore[F, TagConsumer, Offset] + ): CommittableEventJournalQuery[F, Offset, E] = + new CommittableEventJournalQuery(this, offsetStore) +} diff --git a/core/src/main/scala/aecor/aggregate/serialization/Codec.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/Codec.scala similarity index 85% rename from core/src/main/scala/aecor/aggregate/serialization/Codec.scala rename to aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/Codec.scala index 5277f0bb..2a45ae05 100644 --- a/core/src/main/scala/aecor/aggregate/serialization/Codec.scala +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/Codec.scala @@ -1,4 +1,4 @@ -package aecor.aggregate.serialization +package aecor.runtime.akkapersistence.serialization import scala.util.Try diff --git a/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentDecoder.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentDecoder.scala new file mode 100644 index 00000000..56c1e117 --- /dev/null +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentDecoder.scala @@ -0,0 +1,36 @@ +package aecor.runtime.akkapersistence.serialization + +import scala.util.{ Failure, Success } + +trait PersistentDecoder[A] { + def decode(repr: PersistentRepr): PersistentDecoder.DecodingResult[A] +} + +object PersistentDecoder { + def apply[A](implicit instance: PersistentDecoder[A]): PersistentDecoder[A] = instance + + def instance[A](f: PersistentRepr => DecodingResult[A]): PersistentDecoder[A] = + new PersistentDecoder[A] { + override def decode(repr: PersistentRepr): DecodingResult[A] = f(repr) + } + + implicit def fromCodec[A](implicit codec: Codec[A]): PersistentDecoder[A] = + new PersistentDecoder[A] { + override def decode(repr: PersistentRepr): DecodingResult[A] = + codec.decode(repr.payload, repr.manifest) match { + case Failure(exception) => Left(DecodingFailure(exception.getMessage, Some(exception))) + case Success(value) => Right(value) + } + } + + type DecodingResult[A] = Either[DecodingFailure, A] + +} + +final case class DecodingFailure(message: String, underlyingException: Option[Throwable] = None) + extends RuntimeException(message, underlyingException.orNull) + +object DecodingFailure { + def fromThrowable(e: Throwable): DecodingFailure = + DecodingFailure(e.getMessage, Some(e)) +} diff --git a/core/src/main/scala/aecor/aggregate/serialization/PersistentEncoder.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentEncoder.scala similarity index 55% rename from core/src/main/scala/aecor/aggregate/serialization/PersistentEncoder.scala rename to aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentEncoder.scala index e3933038..d757983d 100644 --- a/core/src/main/scala/aecor/aggregate/serialization/PersistentEncoder.scala +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentEncoder.scala @@ -1,4 +1,4 @@ -package aecor.aggregate.serialization +package aecor.runtime.akkapersistence.serialization trait PersistentEncoder[A] { def encode(a: A): PersistentRepr @@ -12,8 +12,9 @@ object PersistentEncoder { override def encode(a: A) = f(a) } - def fromCodec[A](codec: Codec[A]): PersistentEncoder[A] = new PersistentEncoder[A] { - override def encode(a: A) = - PersistentRepr(codec.manifest(a), codec.encode(a)) - } + implicit def fromCodec[A](implicit codec: Codec[A]): PersistentEncoder[A] = + new PersistentEncoder[A] { + override def encode(a: A) = + PersistentRepr(codec.manifest(a), codec.encode(a)) + } } diff --git a/core/src/main/scala/aecor/aggregate/serialization/PersistentRepr.scala b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentRepr.scala similarity index 73% rename from core/src/main/scala/aecor/aggregate/serialization/PersistentRepr.scala rename to aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentRepr.scala index 2e8e5d18..5df2df81 100644 --- a/core/src/main/scala/aecor/aggregate/serialization/PersistentRepr.scala +++ b/aecor-akka-persistence/src/main/scala/aecor/runtime/akkapersistence/serialization/PersistentRepr.scala @@ -1,4 +1,4 @@ -package aecor.aggregate.serialization +package aecor.runtime.akkapersistence.serialization import akka.actor.ExtendedActorSystem import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } @@ -11,10 +11,12 @@ class PersistentReprSerializer(val system: ExtendedActorSystem) override def toBinary(o: AnyRef): Array[Byte] = o match { case pr: PersistentRepr => pr.payload + case x => throw new IllegalArgumentException(s"Serialization of [$x] is not supported") } override def manifest(o: AnyRef): String = o match { case pr: PersistentRepr => pr.manifest + case x => throw new IllegalArgumentException(s"Serialization of [$x] is not supported") } override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = diff --git a/core/src/main/scala/akka/persistence/cassandra/CassandraSessionInitSerialization.scala b/aecor-akka-persistence/src/main/scala/akka/persistence/cassandra/CassandraSessionInitSerialization.scala similarity index 100% rename from core/src/main/scala/akka/persistence/cassandra/CassandraSessionInitSerialization.scala rename to aecor-akka-persistence/src/main/scala/akka/persistence/cassandra/CassandraSessionInitSerialization.scala diff --git a/aecor-fs2/src/main/scala/aecor/effect/fs2/Fs2TaskInstances.scala b/aecor-fs2/src/main/scala/aecor/effect/fs2/Fs2TaskInstances.scala new file mode 100644 index 00000000..d49de194 --- /dev/null +++ b/aecor-fs2/src/main/scala/aecor/effect/fs2/Fs2TaskInstances.scala @@ -0,0 +1,36 @@ +package aecor.effect.fs2 + +import aecor.effect.{ Async, Capture, CaptureFuture } + +import scala.concurrent.Future +import scala.util.{ Failure, Success, Try } + +trait Fs2TaskInstances { + implicit def aecorEffectFs2AsyncInstanceTask: Async[_root_.fs2.Task] = + new Async[_root_.fs2.Task] { + + override def unsafeRunCallback[A](fa: _root_.fs2.Task[A])(f: (Try[A]) => Unit): Unit = + fa.unsafeRunAsync { + case Right(a) => f(Success(a)) + case Left(e) => f(Failure(e)) + } + + override def unsafeRun[A](fa: _root_.fs2.Task[A]): Future[A] = + fa.unsafeRunAsyncFuture() + } + + implicit def aecorEffectFs2CaptureFutureInstanceForTask( + implicit S: fs2.Strategy, + E: scala.concurrent.ExecutionContext + ): CaptureFuture[_root_.fs2.Task] = + new CaptureFuture[_root_.fs2.Task] { + override def captureFuture[A](future: => Future[A]): _root_.fs2.Task[A] = + _root_.fs2.Task.fromFuture(future) + } + + implicit def aecorEffectFs2CaptureInstanceForTask: Capture[_root_.fs2.Task] = + new Capture[_root_.fs2.Task] { + override def capture[A](a: => A): _root_.fs2.Task[A] = + _root_.fs2.Task.delay(a) + } +} diff --git a/aecor-fs2/src/main/scala/aecor/effect/fs2/package.scala b/aecor-fs2/src/main/scala/aecor/effect/fs2/package.scala new file mode 100644 index 00000000..7a19279d --- /dev/null +++ b/aecor-fs2/src/main/scala/aecor/effect/fs2/package.scala @@ -0,0 +1,3 @@ +package aecor.effect + +package object fs2 extends Fs2TaskInstances diff --git a/aecor-monix/src/main/scala/aecor/effect/monix/MonixTaskInstances.scala b/aecor-monix/src/main/scala/aecor/effect/monix/MonixTaskInstances.scala new file mode 100644 index 00000000..a7ea263b --- /dev/null +++ b/aecor-monix/src/main/scala/aecor/effect/monix/MonixTaskInstances.scala @@ -0,0 +1,35 @@ +package aecor.effect.monix + +import aecor.effect.{ Async, Capture, CaptureFuture } +import monix.eval.Task + +import scala.concurrent.Future +import scala.util.Try + +trait MonixTaskInstances { + implicit def aecorEffectMonixAsyncInstanceForTask( + implicit scheduler: _root_.monix.execution.Scheduler + ): Async[_root_.monix.eval.Task] = + new Async[_root_.monix.eval.Task] { + + override def unsafeRunCallback[A](fa: Task[A])(f: (Try[A]) => Unit): Unit = { + fa.runOnComplete(f) + () + } + + override def unsafeRun[A](fa: _root_.monix.eval.Task[A]): Future[A] = fa.runAsync + } + + implicit def aecorEffectMonixCaptureInstanceForTask: Capture[_root_.monix.eval.Task] = + new Capture[_root_.monix.eval.Task] { + override def capture[A](a: => A): _root_.monix.eval.Task[A] = _root_.monix.eval.Task(a) + } + + implicit def aecorEffectMonixCaptureFutureInstanceForTask + : CaptureFuture[_root_.monix.eval.Task] = + new CaptureFuture[_root_.monix.eval.Task] { + override def captureFuture[A](future: => Future[A]): _root_.monix.eval.Task[A] = + _root_.monix.eval.Task.deferFuture(future) + + } +} diff --git a/aecor-monix/src/main/scala/aecor/effect/monix/package.scala b/aecor-monix/src/main/scala/aecor/effect/monix/package.scala new file mode 100644 index 00000000..9dcfce00 --- /dev/null +++ b/aecor-monix/src/main/scala/aecor/effect/monix/package.scala @@ -0,0 +1,3 @@ +package aecor.effect + +package object monix extends MonixTaskInstances diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/E2eSupport.scala b/aecor-test-kit/src/main/scala/aecor/testkit/E2eSupport.scala new file mode 100644 index 00000000..95582b5c --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/E2eSupport.scala @@ -0,0 +1,101 @@ +package aecor.testkit + +import aecor.data._ +import aecor.testkit.Eventsourced.{ BehaviorFailure, RunningState } +import aecor.util.NoopKeyValueStore +import cats.data.{ EitherT, StateT } +import cats.implicits._ +import cats.{ Eval, Monad, ~> } + +import scala.collection.immutable._ + +trait E2eSupport { + final type SpecF[A] = EitherT[Eval, BehaviorFailure, A] + type SpecState + final def mkJournal[E]( + extract: SpecState => StateEventJournal.State[E], + update: (SpecState, StateEventJournal.State[E]) => SpecState + ): StateEventJournal[SpecF, SpecState, E] = + StateEventJournal[SpecF, SpecState, E](extract, update) + + final def mkBehavior[Op[_], S, E]( + behavior: EventsourcedBehavior[StateT[SpecF, SpecState, ?], Op, S, E], + correlation: Correlation[Op], + tagging: Tagging[E], + journal: StateEventJournal[SpecF, SpecState, E] + ): Op ~> StateT[SpecF, SpecState, ?] = + new (Op ~> StateT[SpecF, SpecState, ?]) { + override def apply[A](fa: Op[A]): StateT[SpecF, SpecState, A] = + Eventsourced[StateT[SpecF, SpecState, ?], Op, S, E]( + correlation, + behavior, + tagging, + journal, + Option.empty, + NoopKeyValueStore[StateT[SpecF, SpecState, ?], String, RunningState[S]] + ).run(fa) + .map(_._2) + } + + final def wireProcess[F[_], S, In](process: In => F[Unit], + sources: Processable[F, In]*): WiredProcess[F] = { + val process0 = process + val sources0 = sources + type In0 = In + new WiredProcess[F] { + type In = In0 + override val process: (In) => F[Unit] = process0 + override val sources: Vector[Processable[F, In]] = + sources0.toVector + } + } + + sealed abstract class WiredProcess[F[_]] { + protected type In + protected val process: In => F[Unit] + protected val sources: Vector[Processable[F, In]] + final def run(implicit F: Monad[F]): F[Unit] = + sources + .fold(Processable.empty[F, In])(_ merge _) + .process(process) + .void + + } + + def processes: Vector[WiredProcess[StateT[SpecF, SpecState, ?]]] + + def otherStuff: Vector[StateT[SpecF, SpecState, Unit]] = Vector.empty + + private def runProcesses: StateT[SpecF, SpecState, Unit] = + for { + stateBefore <- StateT.get[SpecF, SpecState] + _ <- (processes.map(_.run) ++ otherStuff).sequence + stateAfter <- StateT.get[SpecF, SpecState] + _ <- if (stateAfter == stateBefore) { + ().pure[StateT[SpecF, SpecState, ?]] + } else { + runProcesses + } + } yield () + + final def wiredK[Op[_]]( + behavior: Op ~> StateT[SpecF, SpecState, ?] + ): Op ~> StateT[SpecF, SpecState, ?] = + new (Op ~> StateT[SpecF, SpecState, ?]) { + override def apply[A](fa: Op[A]): StateT[SpecF, SpecState, A] = + for { + x <- behavior(fa) + _ <- runProcesses + } yield x + } + + final def wired[A, B](f: A => StateT[SpecF, SpecState, B]): A => StateT[SpecF, SpecState, B] = + new (A => StateT[SpecF, SpecState, B]) { + override def apply(a: A): StateT[SpecF, SpecState, B] = + for { + x <- f(a) + _ <- runProcesses + } yield x + } + +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/EventJournal.scala b/aecor-test-kit/src/main/scala/aecor/testkit/EventJournal.scala new file mode 100644 index 00000000..c593a2bd --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/EventJournal.scala @@ -0,0 +1,10 @@ +package aecor.testkit + +import aecor.data.Folder +import cats.Monad +import cats.data.NonEmptyVector + +trait EventJournal[F[_], I, E] { + def append(entityId: I, events: NonEmptyVector[E]): F[Unit] + def foldById[G[_]: Monad, S](entityId: I, offset: Long, folder: Folder[G, E, S]): F[G[S]] +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/Eventsourced.scala b/aecor-test-kit/src/main/scala/aecor/testkit/Eventsourced.scala new file mode 100644 index 00000000..f2caefcc --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/Eventsourced.scala @@ -0,0 +1,108 @@ +package aecor.testkit + +import aecor.data.Folded.{ Impossible, Next } +import aecor.data._ +import aecor.util.KeyValueStore +import akka.cluster.sharding.ShardRegion.EntityId +import cats.data.{ NonEmptyVector, StateT } +import cats.implicits._ +import cats.{ MonadError, ~> } + +import scala.collection.immutable.{ Seq, Set } + +object Eventsourced { + final case class RunningState[S](entityState: S, version: Long) + + object RunningState { + def folder[S, E]( + folder: Folder[Folded, E, S] + ): Folder[Folded, EventEnvelope[E], RunningState[S]] = + Folder( + RunningState(folder.zero, 0), + (s, e) => folder.reduce(s.entityState, e.event).map(RunningState(_, s.version + 1)) + ) + } + + sealed abstract class BehaviorFailure extends Exception + object BehaviorFailure { + def illegalFold(entityId: EntityId): BehaviorFailure = IllegalFold(entityId) + final case class IllegalFold(entityId: EntityId) extends BehaviorFailure + } + + final case class EventEnvelope[E](sequenceNr: Long, event: E, tags: Set[EventTag]) + + def apply[F[_]: MonadError[?[_], BehaviorFailure], Op[_], S, E]( + correlation: Correlation[Op], + entityBehavior: EventsourcedBehavior[F, Op, S, E], + tagging: Tagging[E], + journal: EventJournal[F, String, EventEnvelope[E]], + snapshotEach: Option[Long], + snapshotStore: KeyValueStore[F, String, RunningState[S]] + ): Behavior[F, Op] = + Behavior.correlated[F, Op] { i => + val entityId = correlation(i) + val internalFolder = RunningState.folder(entityBehavior.folder) + def loadState: F[RunningState[S]] = + for { + snapshot <- snapshotStore.getValue(entityId) + effectiveFolder = snapshot.map(internalFolder.withZero).getOrElse(internalFolder) + zero <- journal + .foldById(entityId, effectiveFolder.zero.version, effectiveFolder) + .flatMap { + case Next(x) => x.pure[F] + case Impossible => + BehaviorFailure + .illegalFold(entityId.toString) + .raiseError[F, RunningState[S]] + } + } yield zero + + def updateState(state: RunningState[S], events: Seq[E]) = + if (events.isEmpty) { + state.pure[F] + } else { + val folded = + events.toVector.foldM((false, state, Vector.empty[EventEnvelope[E]])) { + case ((snapshotPending, s, es), e) => + val eventEnvelope = EventEnvelope(s.version + 1, e, tagging(e)) + internalFolder.reduce(s, eventEnvelope).map { next => + def shouldSnapshotNow = + snapshotEach + .exists(x => next.version % x == 0) + (snapshotPending || shouldSnapshotNow, next, es :+ eventEnvelope) + } + } + folded match { + case Next((snapshotNeeded, nextState, envelopes)) => + for { + _ <- journal + .append(entityId, NonEmptyVector.of(envelopes.head, envelopes.tail: _*)) + _ <- if (snapshotNeeded) { + snapshotStore.setValue(entityId, nextState).map(_ => nextState) + } else { + nextState.pure[F] + } + } yield nextState + case Impossible => + BehaviorFailure + .illegalFold(entityId.toString) + .raiseError[F, RunningState[S]] + } + } + Behavior.roll { + loadState.map { s => + Behavior.fromState(s, new (Op ~> StateT[F, RunningState[S], ?]) { + override def apply[A](op: Op[A]): StateT[F, RunningState[S], A] = + StateT { state => + for { + x <- entityBehavior.handler(op).run(state.entityState) + (events, reply) = x + nextState <- updateState(state, events) + } yield (nextState, reply) + } + + }) + } + } + } +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/StateClock.scala b/aecor-test-kit/src/main/scala/aecor/testkit/StateClock.scala new file mode 100644 index 00000000..c0ab2bd2 --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/StateClock.scala @@ -0,0 +1,30 @@ +package aecor.testkit + +import java.time.temporal.TemporalAmount +import java.time.{ Instant, ZoneId } + +import aecor.util.Clock +import cats.Applicative +import cats.data.StateT + +class StateClock[F[_]: Applicative, S](zoneId: ZoneId, + extract: S => Instant, + update: (S, Instant) => S) + extends Clock[StateT[F, S, ?]] { + + override def zone: StateT[F, S, ZoneId] = StateT.pure(zoneId) + + override def instant: StateT[F, S, Instant] = StateT.get[F, Instant].transformS(extract, update) + + def tick(temporalAmount: TemporalAmount): StateT[F, S, Unit] = + StateT + .modify[F, Instant](_.plus(temporalAmount)) + .transformS(extract, update) +} + +object StateClock { + def apply[F[_]: Applicative, S](zoneId: ZoneId, + extract: S => Instant, + update: (S, Instant) => S): StateClock[F, S] = + new StateClock[F, S](zoneId, extract, update) +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/StateEventJournal.scala b/aecor-test-kit/src/main/scala/aecor/testkit/StateEventJournal.scala new file mode 100644 index 00000000..574d1ed7 --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/StateEventJournal.scala @@ -0,0 +1,112 @@ +package aecor.testkit + +import aecor.data.{ ConsumerId, EventTag, Folder } +import aecor.testkit.Eventsourced.EventEnvelope +import aecor.testkit.StateEventJournal.State +import cats.data.{ NonEmptyVector, StateT } +import cats.implicits._ +import cats.{ Applicative, Monad } + +object StateEventJournal { + case class State[E](eventsById: Map[String, Vector[EventEnvelope[E]]], + eventsByTag: Map[EventTag, Vector[EventEnvelope[E]]], + consumerOffsets: Map[(EventTag, ConsumerId), Int]) { + def getConsumerOffset(tag: EventTag, consumerId: ConsumerId): Int = + consumerOffsets.getOrElse(tag -> consumerId, 0) + + def setConsumerOffset(tag: EventTag, consumerId: ConsumerId, offset: Int): State[E] = + copy(consumerOffsets = consumerOffsets.updated(tag -> consumerId, offset)) + + def appendEvents(id: String, events: NonEmptyVector[EventEnvelope[E]]): State[E] = + copy( + eventsById = eventsById + .updated(id, eventsById.getOrElse(id, Vector.empty) ++ events.toVector), + eventsByTag = + eventsByTag |+| events.toVector + .flatMap { env => + env.tags.toVector.map(t => (t, env)) + } + .groupBy(_._1) + .mapValues(_.map(_._2)) + ) + } + + object State { + def init[E]: State[E] = State(Map.empty, Map.empty, Map.empty) + } + + def apply[F[_]: Monad, A, E](extract: A => State[E], + update: (A, State[E]) => A): StateEventJournal[F, A, E] = + new StateEventJournal(extract, update) +} + +class StateEventJournal[F[_]: Monad, A, E](extract: A => State[E], update: (A, State[E]) => A) + extends EventJournal[StateT[F, A, ?], String, EventEnvelope[E]] { + override def append(id: String, events: NonEmptyVector[EventEnvelope[E]]): StateT[F, A, Unit] = + StateT + .modify[F, State[E]](_.appendEvents(id, events)) + .transformS(extract, update) + + override def foldById[G[_]: Monad, S]( + id: String, + offset: Long, + folder: Folder[G, EventEnvelope[E], S] + ): StateT[F, A, G[S]] = + StateT + .inspect[F, State[E], Vector[EventEnvelope[E]]]( + _.eventsById + .get(id) + .map(_.drop(offset.toInt)) + .getOrElse(Vector.empty) + ) + .map(folder.consume(_)) + .transformS(extract, update) + + def eventsByTag(tag: EventTag, consumerId: ConsumerId): Processable[StateT[F, A, ?], E] = + new Processable[StateT[F, A, ?], E] { + override def process(f: (E) => StateT[F, A, Unit]): StateT[F, A, Unit] = + for { + offset0 <- StateT + .inspect[F, State[E], Int](_.getConsumerOffset(tag, consumerId)) + .transformS(extract, update) + result <- StateT + .inspect[F, State[E], Vector[(EventEnvelope[E], Int)]] { + _.eventsByTag + .getOrElse(tag, Vector.empty) + .zipWithIndex + .drop(offset0) + } + .transformS(extract, update) + + _ <- result.traverse { + case (envelope, offset) => + for { + _ <- f(envelope.event) + _ <- StateT + .modify[F, State[E]](_.setConsumerOffset(tag, consumerId, offset + 1)) + .transformS(extract, update) + } yield () + } + } yield () + } + +} + +trait Processable[F[_], A] { outer => + def process(f: A => F[Unit]): F[Unit] + def map[B](f: A => B): Processable[F, B] = + new Processable[F, B] { + override def process(f0: (B) => F[Unit]): F[Unit] = outer.process(a => f0(f(a))) + } + def merge(that: Processable[F, A])(implicit F: Applicative[F]): Processable[F, A] = + new Processable[F, A] { + override def process(f: (A) => F[Unit]): F[Unit] = + F.map2(outer.process(f), that.process(f))((_, _) => ()) + } +} + +object Processable { + def empty[F[_]: Applicative, A]: Processable[F, A] = new Processable[F, A] { + override def process(f: (A) => F[Unit]): F[Unit] = ().pure[F] + } +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/StateKeyValueStore.scala b/aecor-test-kit/src/main/scala/aecor/testkit/StateKeyValueStore.scala new file mode 100644 index 00000000..fdcbca05 --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/StateKeyValueStore.scala @@ -0,0 +1,29 @@ +package aecor.testkit + +import aecor.util.KeyValueStore +import cats.Applicative +import cats.data.StateT + +object StateKeyValueStore { + def apply[F[_]: Applicative, S, K, A]( + extract: S => Map[K, A], + update: (S, Map[K, A]) => S + ): KeyValueStore[StateT[F, S, ?], K, A] = + new StateKeyValueStore(extract, update) +} + +class StateKeyValueStore[F[_]: Applicative, S, K, A](extract: S => Map[K, A], + update: (S, Map[K, A]) => S) + extends KeyValueStore[StateT[F, S, ?], K, A] { + + override def setValue(key: K, value: A): StateT[F, S, Unit] = + StateT + .modify[F, Map[K, A]](_.updated(key, value)) + .transformS(extract, update) + + override def getValue(key: K): StateT[F, S, Option[A]] = + StateT + .inspect[F, Map[K, A], Option[A]](_.get(key)) + .transformS(extract, update) + +} diff --git a/aecor-test-kit/src/main/scala/aecor/testkit/StateRuntime.scala b/aecor-test-kit/src/main/scala/aecor/testkit/StateRuntime.scala new file mode 100644 index 00000000..77172485 --- /dev/null +++ b/aecor-test-kit/src/main/scala/aecor/testkit/StateRuntime.scala @@ -0,0 +1,62 @@ +package aecor.testkit + +import aecor.data.Folded.{ Impossible, Next } +import aecor.data.{ Correlation, EventsourcedBehavior } +import cats.data._ +import cats.implicits._ +import cats.{ Functor, MonadError, ~> } + +object StateRuntime { + + /** + * Creates an aggregate runtime that uses StateT as a target context + * + * This runtime doesn't account for correlation, + * i.e. all operations are executed against common sequence of events + * + */ + def shared[F[_], Op[_], S, E]( + behavior: EventsourcedBehavior[F, Op, S, E] + )(implicit F: MonadError[F, Throwable]): Op ~> StateT[F, Vector[E], ?] = + new (Op ~> StateT[F, Vector[E], ?]) { + override def apply[A](op: Op[A]): StateT[F, Vector[E], A] = + for { + events <- StateT.get[F, Vector[E]] + foldedState = behavior.folder.consume(events) + result <- foldedState match { + case Next(state) => + StateT.lift(behavior.handler(op).run(state)).flatMap { + case (es, r) => + StateT + .modify[F, Vector[E]](_ ++ es) + .map(_ => r) + } + case Impossible => + StateT.lift[F, Vector[E], A]( + F.raiseError(new IllegalStateException(s"Failed to fold $events")) + ) + } + + } yield result + } + + /** + * Creates an aggregate runtime that uses StateT as a target context + * + * This runtime uses correlation function to get entity identifier + * that is used to execute commands against corresponding + * sequence of events + * + */ + def correlate[F[_]: Functor, O[_], E]( + behavior: O ~> StateT[F, Vector[E], ?], + correlation: Correlation[O] + ): O ~> StateT[F, Map[String, Vector[E]], ?] = + new (O ~> StateT[F, Map[String, Vector[E]], ?]) { + override def apply[A](fa: O[A]): StateT[F, Map[String, Vector[E]], A] = { + val entityId = correlation(fa) + behavior(fa).transformS(_.getOrElse(entityId, Vector.empty[E]), _.updated(entityId, _)) + } + } + +} diff --git a/build.sbt b/build.sbt index ece2f52b..6a1d7566 100644 --- a/build.sbt +++ b/build.sbt @@ -1,32 +1,45 @@ import ReleaseTransformations._ import sbtrelease.Version.Bump -lazy val buildSettings = Seq( - organization := "io.aecor", - scalaVersion := "2.11.8", - scalaOrganization := "org.typelevel", - crossScalaVersions := Seq("2.11.8", "2.12.0") +lazy val buildSettings = inThisBuild( + Seq( + organization := "io.aecor", + scalaVersion := "2.11.11-bin-typelevel-4", + scalaOrganization := "org.typelevel", + crossScalaVersions := Seq("2.11.11-bin-typelevel-4", "2.12.2-bin-typelevel-4") + ) ) -lazy val akkaVersion = "2.4.17" -lazy val akkaPersistenceCassandra = "0.23" +lazy val akkaVersion = "2.5.3" +lazy val akkaPersistenceCassandra = "0.54" lazy val catsVersion = "0.9.0" lazy val logbackVersion = "1.1.7" lazy val cassandraDriverExtrasVersion = "3.1.0" lazy val jsr305Version = "3.0.1" +lazy val monixVersion = "2.3.0" +lazy val fs2Version = "0.9.6" lazy val scalaCheckVersion = "1.13.4" lazy val scalaTestVersion = "3.0.1" lazy val scalaCheckShapelessVersion = "1.1.4" lazy val shapelessVersion = "2.3.2" lazy val kindProjectorVersion = "0.9.3" lazy val paradiseVersion = "2.1.0" +lazy val simulacrumVersion = "0.10.0" + +// Example dependencies + +lazy val circeVersion = "0.8.0" +lazy val akkaHttpVersion = "10.0.5" +lazy val akkaHttpJsonVersion = "1.16.0" +lazy val scalametaParadiseVersion = "3.0.0-M9" + +lazy val liberatorVersion = "0.4.3" lazy val commonSettings = Seq( scalacOptions ++= commonScalacOptions, libraryDependencies ++= Seq( - compilerPlugin("org.spire-math" %% "kind-projector" % kindProjectorVersion), - compilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.full) + compilerPlugin("org.spire-math" %% "kind-projector" % kindProjectorVersion) ), parallelExecution in Test := false, scalacOptions in (Compile, doc) := (scalacOptions in (Compile, doc)).value @@ -37,44 +50,113 @@ lazy val aecorSettings = buildSettings ++ commonSettings ++ publishSettings lazy val aecor = project .in(file(".")) - .settings(moduleName := "aecor") + .settings(moduleName := "aecor", name := "Aecor") .settings(aecorSettings) .settings(noPublishSettings) - .aggregate(core, example, schedule, tests) + .aggregate( + core, + akkaPersistence, + distributedProcessing, + example, + schedule, + effectMonix, + effectFs2, + tests + ) .dependsOn(core, example % "compile-internal", tests % "test-internal -> test") lazy val core = - project.settings(moduleName := "aecor-core").settings(aecorSettings).settings(coreSettings) + project + .settings(moduleName := "aecor-core", name := "Aecor Core") + .settings(aecorSettings) + .settings(coreSettings) -lazy val schedule = project +lazy val akkaPersistence = project + .in(file("aecor-akka-persistence")) + .settings( + moduleName := "aecor-akka-persistence", + name := "Aecor Runtime based on Akka Cluster Sharding and Persistence" + ) + .dependsOn(core) + .settings(aecorSettings) + .settings(akkaPersistenceSettings) + +lazy val akkaGeneric = project + .in(file("aecor-akka-generic")) + .settings( + moduleName := "aecor-akka-generic", + name := "Aecor Runtime based on Akka Cluster Sharding" + ) .dependsOn(core) - .settings(moduleName := "aecor-schedule") + .settings(aecorSettings) + .settings(akkaPersistenceSettings) + +lazy val distributedProcessing = + project + .in(file("distributed-processing")) + .settings(moduleName := "aecor-distributed-processing", name := "Aecor Distributed Processing") + .dependsOn(core) + .settings(aecorSettings) + .settings(distributedProcessingSettings) + +lazy val schedule = project + .dependsOn(akkaPersistence, distributedProcessing) + .settings(moduleName := "aecor-schedule", name := "Aecor Schedule") .settings(aecorSettings) .settings(scheduleSettings) +lazy val effectMonix = project + .in(file("aecor-monix")) + .settings(moduleName := "aecor-monix", name := "Aecor Monix") + .dependsOn(core) + .settings(aecorSettings) + .settings(effectMonixSettings) + +lazy val effectFs2 = project + .in(file("aecor-fs2")) + .settings(moduleName := "aecor-fs2", name := "Aecor FS2") + .dependsOn(core) + .settings(aecorSettings) + .settings(effectFs2Settings) + +lazy val testKit = project + .in(file("aecor-test-kit")) + .settings(moduleName := "aecor-test-kit", name := "Aecor Test Kit") + .dependsOn(core) + .settings(aecorSettings) + lazy val tests = project - .dependsOn(core, example, schedule) - .settings(moduleName := "aecor-tests") + .dependsOn( + core, + example, + schedule, + effectMonix, + effectFs2, + testKit, + akkaPersistence, + distributedProcessing, + akkaGeneric + ) + .settings(moduleName := "aecor-tests", name := "Aecor Tests") .settings(aecorSettings) .settings(noPublishSettings) .settings(testingSettings) lazy val example = project - .dependsOn(core, schedule) - .settings(moduleName := "aecor-example") + .dependsOn(core, schedule, effectMonix, distributedProcessing) + .settings(moduleName := "aecor-example", name := "Aecor Example Application") .settings(aecorSettings) .settings(noPublishSettings) .settings(exampleSettings) lazy val coreSettings = Seq( libraryDependencies ++= Seq( - "com.typesafe.akka" %% "akka-cluster-sharding" % akkaVersion, - "com.typesafe.akka" %% "akka-persistence" % akkaVersion, - "com.typesafe.akka" %% "akka-persistence-query-experimental" % akkaVersion, + compilerPlugin("org.scalamacros" % "paradise" % paradiseVersion cross CrossVersion.patch), "com.typesafe.akka" %% "akka-slf4j" % akkaVersion, - "com.typesafe.akka" %% "akka-persistence-cassandra" % akkaPersistenceCassandra, + "com.typesafe.akka" %% "akka-cluster-sharding" % akkaVersion, "com.chuusai" %% "shapeless" % shapelessVersion, - "org.typelevel" %% "cats" % catsVersion + "org.typelevel" %% "cats" % catsVersion, + "com.github.mpilquist" %% "simulacrum" % simulacrumVersion ) ) @@ -85,22 +167,57 @@ lazy val scheduleSettings = commonProtobufSettings ++ Seq( ) ) +lazy val distributedProcessingSettings = commonProtobufSettings ++ Seq( + libraryDependencies ++= Seq( + "com.typesafe.akka" %% "akka-slf4j" % akkaVersion, + "com.typesafe.akka" %% "akka-cluster-sharding" % akkaVersion + ) +) + +lazy val akkaPersistenceSettings = Seq( + libraryDependencies ++= Seq( + "com.typesafe.akka" %% "akka-cluster-sharding" % akkaVersion, + "com.typesafe.akka" %% "akka-persistence" % akkaVersion, + "com.typesafe.akka" %% "akka-persistence-query" % akkaVersion, + "com.typesafe.akka" %% "akka-persistence-cassandra" % akkaPersistenceCassandra + ) +) + +lazy val akkaGenericSettings = Seq( + libraryDependencies ++= Seq("com.typesafe.akka" %% "akka-cluster-sharding" % akkaVersion) +) + +lazy val effectMonixSettings = Seq( + libraryDependencies ++= Seq("io.monix" %% "monix-eval" % monixVersion) +) + +lazy val effectFs2Settings = Seq(libraryDependencies ++= Seq("co.fs2" %% "fs2-core" % fs2Version)) + lazy val exampleSettings = { - val circeVersion = "0.6.1" - val akkaHttpVersion = "10.0.3" - val akkaHttpJsonVersion = "1.11.0" - val freekVersion = "0.6.5" + val akkaKryoVersion = SettingKey[String]("akka-kryo-version", "") + Seq( - resolvers ++= Seq(Resolver.bintrayRepo("projectseptemberinc", "maven")), + akkaKryoVersion := { + if (scalaVersion.value startsWith "2.11") "0.5.0" else "0.5.1" + }, + resolvers += Resolver.sonatypeRepo("releases"), + sources in (Compile, doc) := Nil, libraryDependencies ++= Seq( + compilerPlugin( + "org.scalameta" % "paradise" % scalametaParadiseVersion cross CrossVersion.patch + ), + "com.github.romix.akka" %% "akka-kryo-serialization" % akkaKryoVersion.value, + "io.aecor" %% "liberator" % liberatorVersion, + "com.typesafe.akka" %% "akka-slf4j" % akkaVersion, + "io.monix" %% "monix-cats" % monixVersion, + "io.monix" %% "monix-reactive" % monixVersion, "com.typesafe.akka" %% "akka-http" % akkaHttpVersion, "de.heikoseeberger" %% "akka-http-circe" % akkaHttpJsonVersion, - ("com.projectseptember" %% "freek" % freekVersion) - .exclude("org.typelevel", "cats-free_2.12.0-RC2"), "io.circe" %% "circe-core" % circeVersion, "io.circe" %% "circe-generic" % circeVersion, "io.circe" %% "circe-parser" % circeVersion, + "io.circe" %% "circe-java8" % circeVersion, "ch.qos.logback" % "logback-classic" % logbackVersion ) ) diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index bf077621..c75244f7 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -28,13 +28,13 @@ akka { "akka.dispatch.sysmsg.DeathWatchNotification" = java } serialization-identifiers { - "aecor.aggregate.serialization.PersistentReprSerializer" = 100 + "aecor.runtime.akkapersistence.serialization.PersistentReprSerializer" = 100 } serializers { - persistent-repr = "aecor.aggregate.serialization.PersistentReprSerializer" + persistent-repr = "aecor.runtime.akkapersistence.serialization.PersistentReprSerializer" } serialization-bindings { - "aecor.aggregate.serialization.PersistentRepr" = persistent-repr + "aecor.runtime.akkapersistence.serialization.PersistentRepr" = persistent-repr } } persistence { diff --git a/core/src/main/scala/aecor/aggregate/AkkaRuntime.scala b/core/src/main/scala/aecor/aggregate/AkkaRuntime.scala deleted file mode 100644 index a568ce04..00000000 --- a/core/src/main/scala/aecor/aggregate/AkkaRuntime.scala +++ /dev/null @@ -1,66 +0,0 @@ -package aecor.aggregate - -import aecor.aggregate.AkkaRuntime.CorrelatedCommand -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentEncoder } -import aecor.data.{ Folded, Handler } -import akka.actor.ActorSystem -import akka.cluster.sharding.{ ClusterSharding, ShardRegion } -import akka.pattern.ask -import akka.util.Timeout -import cats.~> - -import scala.concurrent.Future - -object AkkaRuntime { - def apply(system: ActorSystem): AkkaRuntime = - new AkkaRuntime(system) - - private final case class CorrelatedCommand[C[_], A](entityId: String, command: C[A]) -} - -class AkkaRuntime(system: ActorSystem) { - def start[Command[_], State, Event: PersistentEncoder: PersistentDecoder]( - entityName: String, - behavior: Command ~> Handler[State, Event, ?], - correlation: Correlation[Command], - tagging: Tagging[Event], - snapshotPolicy: SnapshotPolicy[State] = SnapshotPolicy.never, - settings: AkkaRuntimeSettings = AkkaRuntimeSettings.default(system) - )(implicit folder: Folder[Folded, Event, State]): Command ~> Future = { - - val props = AggregateActor.props( - entityName, - behavior, - Identity.FromPathName, - snapshotPolicy, - tagging, - settings.idleTimeout - ) - - def extractEntityId: ShardRegion.ExtractEntityId = { - case CorrelatedCommand(entityId, c) => - (entityId, AggregateActor.HandleCommand(c)) - } - - val numberOfShards = settings.numberOfShards - - def extractShardId: ShardRegion.ExtractShardId = { - case CorrelatedCommand(entityId, _) => - (scala.math.abs(entityId.hashCode) % numberOfShards).toString - } - - val shardRegionRef = ClusterSharding(system).start( - typeName = entityName, - entityProps = props, - settings = settings.clusterShardingSettings, - extractEntityId = extractEntityId, - extractShardId = extractShardId - ) - - new (Command ~> Future) { - implicit private val timeout = Timeout(settings.askTimeout) - override def apply[A](fa: Command[A]): Future[A] = - (shardRegionRef ? CorrelatedCommand(correlation(fa), fa)).asInstanceOf[Future[A]] - } - } -} diff --git a/core/src/main/scala/aecor/aggregate/Folder.scala b/core/src/main/scala/aecor/aggregate/Folder.scala deleted file mode 100644 index a7f183c6..00000000 --- a/core/src/main/scala/aecor/aggregate/Folder.scala +++ /dev/null @@ -1,18 +0,0 @@ -package aecor.aggregate - -import cats.{ Foldable, Monad } -import cats.implicits._ - -trait Folder[F[_], A, B] { - def zero: B - def fold(b: B, a: A): F[B] - def consume[I[_]: Foldable](f: I[A])(implicit F: Monad[F]): F[B] = f.foldM(zero)(fold) -} - -object Folder { - def instance[F[_], A, B](b: B)(f: (B) => (A) => F[B]): Folder[F, A, B] = - new Folder[F, A, B] { - override def zero: B = b - override def fold(b: B, a: A): F[B] = f(b)(a) - } -} diff --git a/core/src/main/scala/aecor/aggregate/StateRuntime.scala b/core/src/main/scala/aecor/aggregate/StateRuntime.scala deleted file mode 100644 index 25880ccc..00000000 --- a/core/src/main/scala/aecor/aggregate/StateRuntime.scala +++ /dev/null @@ -1,51 +0,0 @@ -package aecor.aggregate - -import aecor.data.Handler -import cats.data._ -import cats.implicits._ -import cats.{ Monad, ~> } - -object StateRuntime { - - /** - * Creates an aggregate runtime that uses StateT as a target context - * - * This runtime doesn't account for correlation, - * i.e. all operations are executed against common sequence of events - * - */ - def shared[Op[_], S, E, F[_]: Monad]( - behavior: Op ~> Handler[S, E, ?] - )(implicit folder: Folder[F, E, S]): Op ~> StateT[F, Vector[E], ?] = - new (Op ~> StateT[F, Vector[E], ?]) { - override def apply[A](fa: Op[A]): StateT[F, Vector[E], A] = - for { - events <- StateT.get[F, Vector[E]] - state <- StateT.lift(folder.consume(events)) - result <- { - val (es, r) = behavior(fa).run(state) - StateT.modify[F, Vector[E]](_ ++ es).map(_ => r) - } - } yield result - } - - /** - * Creates an aggregate runtime that uses StateT as a target context - * - * This runtime uses correlation function to get entity identifier - * that is used to execute commands against corresponding - * sequence of events - * - */ - def correlated[O[_], S, E, F[_]: Monad]( - behavior: O ~> Handler[S, E, ?], - correlation: Correlation[O] - )(implicit folder: Folder[F, E, S]): O ~> StateT[F, Map[String, Vector[E]], ?] = - new (O ~> StateT[F, Map[String, Vector[E]], ?]) { - override def apply[A](fa: O[A]): StateT[F, Map[String, Vector[E]], A] = { - val inner: O ~> StateT[F, Vector[E], ?] = shared(behavior) - val entityId = correlation(fa) - inner(fa).transformS(_.getOrElse(entityId, Vector.empty[E]), _.updated(entityId, _)) - } - } -} diff --git a/core/src/main/scala/aecor/aggregate/Tagging.scala b/core/src/main/scala/aecor/aggregate/Tagging.scala deleted file mode 100644 index 59812ce8..00000000 --- a/core/src/main/scala/aecor/aggregate/Tagging.scala +++ /dev/null @@ -1,44 +0,0 @@ -package aecor.aggregate - -import aecor.data.EventTag - -sealed abstract class Tagging[A] { - def apply(e: A): Set[String] -} - -/** - * Please refer to akka-persistence-cassandra documentation and its reference.conf - * to understand how tagging works internally - */ -object Tagging { - - def apply[A](tag1: EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1.value) - } - - def apply[A](tag1: A => EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1(e).value) - } - - def apply[A](tag1: EventTag[A], tag2: EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1.value, tag2.value) - } - - def apply[A](tag1: A => EventTag[A], tag2: EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1(e).value, tag2.value) - } - - def apply[A](tag1: EventTag[A], tag2: EventTag[A], tag3: EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1.value, tag2.value, tag3.value) - } - - def apply[A](tag1: A => EventTag[A], tag2: EventTag[A], tag3: EventTag[A]): Tagging[A] = - new Tagging[A] { - override def apply(e: A): Set[String] = Set(tag1(e).value, tag2.value, tag3.value) - } -} diff --git a/core/src/main/scala/aecor/aggregate/package.scala b/core/src/main/scala/aecor/aggregate/package.scala deleted file mode 100644 index 1897b0ff..00000000 --- a/core/src/main/scala/aecor/aggregate/package.scala +++ /dev/null @@ -1,20 +0,0 @@ -package aecor - -import cats.~> - -package object aggregate { - type CorrelationId = String - type CorrelationIdF[A] = CorrelationId - type Correlation[C[_]] = (C ~> CorrelationIdF) - object CorrelationId { - def composite(sep: String, firstComponent: String, otherComponents: String*): CorrelationId = { - val replacement = s"\\$sep" - val builder = new StringBuilder(firstComponent.replace(sep, replacement)) - otherComponents.foreach { component => - builder.append(sep) - builder.append(component.replace(sep, replacement)) - } - builder.result() - } - } -} diff --git a/core/src/main/scala/aecor/aggregate/serialization/PersistentDecoder.scala b/core/src/main/scala/aecor/aggregate/serialization/PersistentDecoder.scala deleted file mode 100644 index 31deeb02..00000000 --- a/core/src/main/scala/aecor/aggregate/serialization/PersistentDecoder.scala +++ /dev/null @@ -1,35 +0,0 @@ -package aecor.aggregate.serialization - -import scala.util.{ Failure, Success } - -trait PersistentDecoder[A] { - def decode(repr: PersistentRepr): PersistentDecoder.Result[A] -} - -object PersistentDecoder { - def apply[A](implicit instance: PersistentDecoder[A]): PersistentDecoder[A] = instance - - def instance[A](f: PersistentRepr => Result[A]): PersistentDecoder[A] = - new PersistentDecoder[A] { - override def decode(repr: PersistentRepr): Result[A] = f(repr) - } - - def fromCodec[A](codec: Codec[A]): PersistentDecoder[A] = new PersistentDecoder[A] { - override def decode(repr: PersistentRepr): Result[A] = - codec.decode(repr.payload, repr.manifest) match { - case Failure(exception) => Left(DecodingFailure(exception.getMessage, Some(exception))) - case Success(value) => Right(value) - } - } - - type Result[A] = Either[DecodingFailure, A] - -} - -final case class DecodingFailure(message: String, underlyingException: Option[Throwable] = None) - extends RuntimeException(message, underlyingException.orNull) - -object DecodingFailure { - def fromThrowable(e: Throwable): DecodingFailure = - DecodingFailure(e.getMessage, Some(e)) -} diff --git a/core/src/main/scala/aecor/data/Behavior.scala b/core/src/main/scala/aecor/data/Behavior.scala new file mode 100644 index 00000000..dca1322d --- /dev/null +++ b/core/src/main/scala/aecor/data/Behavior.scala @@ -0,0 +1,43 @@ +package aecor.data + +import cats.arrow.FunctionK +import cats.data.StateT +import cats.implicits._ +import cats.{ FlatMap, Functor, ~> } + +/** + * `Behavior[Op, F]` says that each operation `Op[A]` will cause effect `F` + * producing a pair consisting of next `Behavior[Op, F]` and an `A` + */ +final case class Behavior[F[_], Op[_]](run: Op ~> PairT[F, Behavior[F, Op], ?]) { + def mapK[G[_]: Functor](f: F ~> G): Behavior[G, Op] = Behavior[G, Op] { + def mk[A](op: Op[A]): PairT[G, Behavior[G, Op], A] = + f(run(op)).map { + case (b, a) => + (b.mapK(f), a) + } + FunctionK.lift(mk _) + } +} + +object Behavior { + def roll[F[_]: FlatMap, Op[_]](f: F[Behavior[F, Op]]): Behavior[F, Op] = + Behavior[F, Op](new (Op ~> PairT[F, Behavior[F, Op], ?]) { + override def apply[A](op: Op[A]): PairT[F, Behavior[F, Op], A] = + FlatMap[F].flatMap(f)(_.run(op)) + }) + + def fromState[F[_]: FlatMap, Op[_], S](zero: S, f: Op ~> StateT[F, S, ?]): Behavior[F, Op] = + Behavior[F, Op](new (Op ~> PairT[F, Behavior[F, Op], ?]) { + override def apply[A](fa: Op[A]): PairT[F, Behavior[F, Op], A] = + f(fa).run(zero).map { + case (next, a) => + fromState(next, f) -> a + } + }) + + def correlated[F[_], Op[_]](f: Op[_] => Behavior[F, Op]): Behavior[F, Op] = + Behavior(Lambda[Op ~> PairT[F, Behavior[F, Op], ?]] { op => + f(op).run(op) + }) +} diff --git a/core/src/main/scala/aecor/data/Committable.scala b/core/src/main/scala/aecor/data/Committable.scala new file mode 100644 index 00000000..a98ad4da --- /dev/null +++ b/core/src/main/scala/aecor/data/Committable.scala @@ -0,0 +1,55 @@ +package aecor.data + +import cats.implicits._ +import cats.{ Applicative, Eval, Functor, Monad, Traverse } + +import scala.util.{ Left, Right } + +final case class Committable[F[_], +A](commit: F[Unit], value: A) { + def map[B](f: A => B): Committable[F, B] = copy(value = f(value)) + def traverse[G[_], B](f: A => G[B])(implicit G: Functor[G]): G[Committable[F, B]] = + G.map(f(value))(b => copy(value = b)) + def foldLeft[B](b: B)(f: (B, A) => B): B = f(b, value) +} + +object Committable { + implicit def catsMonadAndTraversInstance[F[_]: Applicative] + : Monad[Committable[F, ?]] with Traverse[Committable[F, ?]] = + new Monad[Committable[F, ?]] with Traverse[Committable[F, ?]] { + override def traverse[G[_], A, B]( + fa: Committable[F, A] + )(f: (A) => G[B])(implicit evidence$1: Applicative[G]): G[Committable[F, B]] = + fa.traverse(f) + + override def flatMap[A, B]( + fa: Committable[F, A] + )(f: (A) => Committable[F, B]): Committable[F, B] = + f(fa.value) + + override def tailRecM[A, B]( + a: A + )(f: (A) => Committable[F, Either[A, B]]): Committable[F, B] = { + val c = f(a) + c.value match { + case Left(aa) => tailRecM(aa)(f) + case Right(b) => c.copy(value = b) + } + } + + override def foldLeft[A, B](fa: Committable[F, A], b: B)(f: (B, A) => B): B = + fa.foldLeft(b)(f) + + override def foldRight[A, B](fa: Committable[F, A], lb: Eval[B])( + f: (A, Eval[B]) => Eval[B] + ): Eval[B] = f(fa.value, lb) + + override def pure[A](x: A): Committable[F, A] = Committable.pure(x) + } + def pure[F[_]: Applicative, A](a: A): Committable[F, A] = Committable(().pure[F], a) + def unit[F[_]: Applicative]: Committable[F, Unit] = pure(()) + def collector[F[_], A, B]( + pf: PartialFunction[A, B] + ): PartialFunction[Committable[F, A], Committable[F, B]] = { + case c if pf.isDefinedAt(c.value) => c.map(pf) + } +} diff --git a/core/src/main/scala/aecor/data/ConsumerId.scala b/core/src/main/scala/aecor/data/ConsumerId.scala new file mode 100644 index 00000000..5be1f7dd --- /dev/null +++ b/core/src/main/scala/aecor/data/ConsumerId.scala @@ -0,0 +1,5 @@ +package aecor.data + +final case class ConsumerId(value: String) extends AnyVal + +final case class TagConsumer(tag: EventTag, consumerId: ConsumerId) diff --git a/core/src/main/scala/aecor/data/Correlation.scala b/core/src/main/scala/aecor/data/Correlation.scala new file mode 100644 index 00000000..01a5e5f5 --- /dev/null +++ b/core/src/main/scala/aecor/data/Correlation.scala @@ -0,0 +1,16 @@ +package aecor.data + +object Correlation { + def apply[C[_]](f: C[_] => CorrelationId): Correlation[C] = f +} +object CorrelationId { + def composite(separator: String, + firstComponent: String, + secondComponent: String, + otherComponents: String*): CorrelationId = { + val replacement = s"\\$separator" + (firstComponent +: secondComponent +: otherComponents) + .map(_.replace(separator, replacement)) + .mkString(separator) + } +} diff --git a/core/src/main/scala/aecor/data/EventTag.scala b/core/src/main/scala/aecor/data/EventTag.scala index 9379aef0..f166776d 100644 --- a/core/src/main/scala/aecor/data/EventTag.scala +++ b/core/src/main/scala/aecor/data/EventTag.scala @@ -1,3 +1,3 @@ package aecor.data -final case class EventTag[E](value: String) extends AnyVal +final case class EventTag(value: String) extends AnyVal diff --git a/core/src/main/scala/aecor/data/EventsourcedBehavior.scala b/core/src/main/scala/aecor/data/EventsourcedBehavior.scala new file mode 100644 index 00000000..a25b9f69 --- /dev/null +++ b/core/src/main/scala/aecor/data/EventsourcedBehavior.scala @@ -0,0 +1,8 @@ +package aecor.data + +import cats.~> + +final case class EventsourcedBehavior[F[_], Op[_], State, Event]( + handler: Op ~> Handler[F, State, Event, ?], + folder: Folder[Folded, Event, State] +) diff --git a/core/src/main/scala/aecor/data/Folded.scala b/core/src/main/scala/aecor/data/Folded.scala index ffc3ccfb..189af0cc 100644 --- a/core/src/main/scala/aecor/data/Folded.scala +++ b/core/src/main/scala/aecor/data/Folded.scala @@ -1,6 +1,7 @@ package aecor.data import aecor.data.Folded.{ Impossible, Next } + import cats.kernel.Eq import cats.{ Alternative, @@ -55,6 +56,9 @@ object Folded extends FoldedInstances { final case class Next[+A](a: A) extends Folded[A] def impossible[A]: Folded[A] = Impossible def next[A](a: A): Folded[A] = Next(a) + def collectNext[A]: PartialFunction[Folded[A], Next[A]] = { + case next @ Next(_) => next + } object syntax { implicit class FoldedIdOps[A](val a: A) extends AnyVal { def next: Folded[A] = Folded.next(a) diff --git a/core/src/main/scala/aecor/data/Folder.scala b/core/src/main/scala/aecor/data/Folder.scala new file mode 100644 index 00000000..bcf965a2 --- /dev/null +++ b/core/src/main/scala/aecor/data/Folder.scala @@ -0,0 +1,29 @@ +package aecor.data + +import cats.implicits._ +import cats.{ Foldable, Functor, Monad } + +final case class Folder[F[_], A, B](zero: B, reduce: (B, A) => F[B]) { + def withZero(b: B): Folder[F, A, B] = copy(zero = b) + def consume[I[_]: Foldable](f: I[A])(implicit F: Monad[F]): F[B] = f.foldM(zero)(reduce) + def imap[C](bc: B => C, cb: C => B)(implicit F: Functor[F]): Folder[F, A, C] = + Folder[F, A, C](bc(zero), (b, a) => reduce(cb(b), a).map(bc)) + def mapK[G[_]](f: F[B] => G[B]): Folder[G, A, B] = Folder(zero, (b, a) => f(reduce(b, a))) +} + +object Folder { + def curried[F[_], A, B](b: B)(reducer: (B) => (A) => F[B]): Folder[F, A, B] = + Folder(b, (b, a) => reducer(b)(a)) + + def optionInstance[F[_]: Functor, A, B]( + init: A => F[B] + )(reduce: B => A => F[B]): Folder[F, A, Option[B]] = + curried(Option.empty[B]) { + case None => init.andThen(_.map(Some(_))) + case Some(b) => reduce(b).andThen(_.map(Some(_))) + } +} + +trait Eventsourced[A, E] { + def apply(a: Option[A], e: E): Folded[A] +} diff --git a/core/src/main/scala/aecor/data/Handler.scala b/core/src/main/scala/aecor/data/Handler.scala index 18cf8f5d..5e82cba2 100644 --- a/core/src/main/scala/aecor/data/Handler.scala +++ b/core/src/main/scala/aecor/data/Handler.scala @@ -1,9 +1,20 @@ package aecor.data -import cats.data.Kleisli +import cats.Applicative -import scala.collection.immutable.Seq +import scala.collection.immutable._ -final case class Handler[State, Event, A](run: State => (Seq[Event], A)) extends AnyVal { - def asKleisli: Kleisli[(Seq[Event], ?), State, A] = Kleisli(run) +final case class Handler[F[_], State, Event, Result](run: State => F[(Seq[Event], Result)]) + extends AnyVal + +object Handler { + final class MkLift[F[_], State] { + def apply[E, A](f: State => (Seq[E], A))(implicit F: Applicative[F]): Handler[F, State, E, A] = + Handler(s => F.pure(f(s))) + } + def lift[F[_], State]: MkLift[F, State] = new MkLift[F, State] + def readOnly[F[_]: Applicative, State, Event, Result]( + f: State => Result + ): Handler[F, State, Event, Result] = + Handler(x => Applicative[F].pure((Seq.empty[Event], f(x)))) } diff --git a/core/src/main/scala/aecor/data/Reducer.scala b/core/src/main/scala/aecor/data/Reducer.scala new file mode 100644 index 00000000..3115d82e --- /dev/null +++ b/core/src/main/scala/aecor/data/Reducer.scala @@ -0,0 +1,6 @@ +package aecor.data + +trait Reducer[A, B] { + def unit(a: A): B + def reduce(b: B, a: A): B +} diff --git a/core/src/main/scala/aecor/data/Tagging.scala b/core/src/main/scala/aecor/data/Tagging.scala new file mode 100644 index 00000000..16ae5c7a --- /dev/null +++ b/core/src/main/scala/aecor/data/Tagging.scala @@ -0,0 +1,39 @@ +package aecor.data +import scala.collection.immutable._ +sealed abstract class Tagging[A] { + def apply(e: A): Set[EventTag] +} + +object Tagging { + sealed abstract class Partitioned[A] extends Tagging[A] { + def tags: Seq[EventTag] + } + sealed abstract class Const[A] extends Tagging[A] { + def tag: EventTag + final override def apply(e: A): Set[EventTag] = Set(tag) + } + + def const[A](tag: EventTag): Const[A] = { + val tag0 = tag + new Const[A] { + override val tag: EventTag = tag0 + } + } + + def dynamic[A](f: A => EventTag): Tagging[A] = + new Tagging[A] { + override def apply(e: A): Set[EventTag] = Set(f(e)) + } + + def partitioned[A](numberOfPartitions: Int, + tag: EventTag)(partitionKey: A => String): Partitioned[A] = + new Partitioned[A] { + private def tagForPartition(partition: Int) = EventTag(s"${tag.value}$partition") + override def tags: Seq[EventTag] = (0 to numberOfPartitions).map(tagForPartition) + override def apply(a: A): Set[EventTag] = { + val partition = scala.math.abs(partitionKey(a).hashCode % numberOfPartitions) + Set(tagForPartition(partition)) + } + } + +} diff --git a/core/src/main/scala/aecor/data/package.scala b/core/src/main/scala/aecor/data/package.scala new file mode 100644 index 00000000..9e63d84d --- /dev/null +++ b/core/src/main/scala/aecor/data/package.scala @@ -0,0 +1,11 @@ +package aecor + +package object data { + type CorrelationId = String + type Correlation[C[_]] = (C[_] => CorrelationId) + + /** + * A transformer type representing a `(A, B)` wrapped in `F` + */ + type PairT[F[_], A, B] = F[(A, B)] +} diff --git a/core/src/main/scala/aecor/effect/Async.scala b/core/src/main/scala/aecor/effect/Async.scala new file mode 100644 index 00000000..7d4c4169 --- /dev/null +++ b/core/src/main/scala/aecor/effect/Async.scala @@ -0,0 +1,70 @@ +package aecor.effect + +import cats.data.{ EitherT, Kleisli } +import cats.~> +import simulacrum.typeclass + +import scala.concurrent.{ ExecutionContext, Future, Promise } +import scala.util.{ Failure, Success, Try } + +/** + * The type class for types that can be run in async manner + * F[A] should not execute any side effects before `unsafeRun` + */ +@typeclass +trait Async[F[_]] { + def unsafeRunCallback[A](fa: F[A])(f: Try[A] => Unit): Unit + def unsafeRun[A](fa: F[A]): concurrent.Future[A] = { + val p = Promise[A] + unsafeRunCallback(fa) { a => + p.complete(a) + () + } + p.future + } + final def asFunctionK: F ~> Future = Lambda[F ~> Future](unsafeRun(_)) +} + +object Async extends AsyncInstances + +sealed trait AsyncInstances { + implicit def aecorEffectAsyncInstanceForKleisliFuture( + implicit executionContext: ExecutionContext + ): Async[Kleisli[Future, Unit, ?]] = + new Async[Kleisli[Future, Unit, ?]] { + + override def unsafeRunCallback[A](fa: Kleisli[Future, Unit, A])(f: (Try[A]) => Unit): Unit = + fa.run(()).onComplete(f) + + override def unsafeRun[B](fa: Kleisli[Future, Unit, B]): Future[B] = + fa.run(()) + } + + implicit def aecorEffectAsyncInstanceForKleisliAsyncF[F[_]: Async]: Async[Kleisli[F, Unit, ?]] = + new Async[Kleisli[F, Unit, ?]] { + + override def unsafeRunCallback[A](fa: Kleisli[F, Unit, A])(f: (Try[A]) => Unit): Unit = + Async[F].unsafeRunCallback(fa.run(()))(f) + + override def unsafeRun[B](fa: Kleisli[F, Unit, B]): Future[B] = + Async[F].unsafeRun(fa.run(())) + } + + implicit def eitherTAsync[F[_]: Async, L <: Throwable]( + implicit executionContext: ExecutionContext + ): Async[EitherT[F, L, ?]] = new Async[EitherT[F, L, ?]] { + + override def unsafeRunCallback[A](fa: EitherT[F, L, A])(f: (Try[A]) => Unit): Unit = + Async[F].unsafeRunCallback(fa.value) { + case Success(Right(a)) => f(scala.util.Success(a)) + case Success(Left(e)) => f(scala.util.Failure(e)) + case Failure(a) => f(scala.util.Failure(a)) + } + + override def unsafeRun[A](fa: EitherT[F, L, A]): Future[A] = + Async[F].unsafeRun(fa.value).flatMap { + case Right(a) => Future.successful(a) + case Left(a) => Future.failed(a) + } + } +} diff --git a/core/src/main/scala/aecor/effect/Capture.scala b/core/src/main/scala/aecor/effect/Capture.scala new file mode 100644 index 00000000..bb73a7aa --- /dev/null +++ b/core/src/main/scala/aecor/effect/Capture.scala @@ -0,0 +1,23 @@ +package aecor.effect + +import cats.data.{ EitherT, Kleisli } +import cats.{ Applicative, Functor } +import simulacrum.typeclass + +@typeclass +trait Capture[F[_]] { + def capture[A](a: => A): F[A] +} + +object Capture extends CaptureInstances + +sealed trait CaptureInstances { + implicit def kleisliCapture[F[_]: Applicative, B]: Capture[Kleisli[F, B, ?]] = + new Capture[Kleisli[F, B, ?]] { + override def capture[A](a: => A): Kleisli[F, B, A] = Kleisli(_ => Applicative[F].pure(a)) + } + implicit def captureEitherT[F[_]: Capture: Functor, B]: Capture[EitherT[F, B, ?]] = + new Capture[EitherT[F, B, ?]] { + override def capture[A](a: => A): EitherT[F, B, A] = EitherT.right(Capture[F].capture(a)) + } +} diff --git a/core/src/main/scala/aecor/effect/CaptureFuture.scala b/core/src/main/scala/aecor/effect/CaptureFuture.scala new file mode 100644 index 00000000..92fd29ed --- /dev/null +++ b/core/src/main/scala/aecor/effect/CaptureFuture.scala @@ -0,0 +1,28 @@ +package aecor.effect + +import cats.Functor +import cats.data.{ EitherT, Kleisli } +import simulacrum.typeclass + +import scala.concurrent.Future +@typeclass +trait CaptureFuture[F[_]] { + def captureFuture[A](future: => Future[A]): F[A] +} + +object CaptureFuture extends CaptureFutureInstances + +sealed trait CaptureFutureInstances { + implicit def futureCaptureFutureInstance[B]: CaptureFuture[Kleisli[Future, B, ?]] = + new CaptureFuture[Kleisli[Future, B, ?]] { + override def captureFuture[A](future: => Future[A]): Kleisli[Future, B, A] = + Kleisli(_ => future) + } + + implicit def captureFutureEitherT[F[_]: CaptureFuture: Functor, B] + : CaptureFuture[EitherT[F, B, ?]] = + new CaptureFuture[EitherT[F, B, ?]] { + override def captureFuture[A](future: => Future[A]): EitherT[F, B, A] = + EitherT.right(CaptureFuture[F].captureFuture(future)) + } +} diff --git a/core/src/main/scala/aecor/streaming/AggregateJournal.scala b/core/src/main/scala/aecor/streaming/AggregateJournal.scala deleted file mode 100644 index 6b491c76..00000000 --- a/core/src/main/scala/aecor/streaming/AggregateJournal.scala +++ /dev/null @@ -1,63 +0,0 @@ -package aecor.streaming - -import aecor.aggregate.serialization.PersistentDecoder -import aecor.data.EventTag -import akka.NotUsed -import akka.stream.scaladsl.Source - -import scala.concurrent.Future - -final case class JournalEntry[+O, +A](offset: O, persistenceId: String, sequenceNr: Long, event: A) { - def mapOffset[I](f: O => I): JournalEntry[I, A] = copy(f(offset)) -} - -object JournalEntry { - implicit def committable[Offset, A]( - implicit ev: Commit[Offset] - ): Commit[JournalEntry[Offset, A]] = - new Commit[JournalEntry[Offset, A]] { - override def commit(a: JournalEntry[Offset, A]): Future[Unit] = ev.commit(a.offset) - } -} - -trait AggregateJournal[Offset] { - def eventsByTag[E: PersistentDecoder]( - tag: EventTag[E], - offset: Option[Offset] - ): Source[JournalEntry[Offset, E], NotUsed] - - def currentEventsByTag[E: PersistentDecoder]( - tag: EventTag[E], - offset: Option[Offset] - ): Source[JournalEntry[Offset, E], NotUsed] - - final def committableEventsByTag[E: PersistentDecoder]( - offsetStore: OffsetStore[Offset], - tag: EventTag[E], - consumerId: ConsumerId - ): Source[Committable[JournalEntry[Offset, E]], NotUsed] = - Source - .single(NotUsed) - .mapAsync(1) { _ => - offsetStore.getOffset(tag.value, consumerId) - } - .flatMapConcat { storedOffset => - eventsByTag[E](tag, storedOffset) - .map(x => Committable(() => offsetStore.setOffset(tag.value, consumerId, x.offset), x)) - } - - final def committableCurrentEventsByTag[E: PersistentDecoder]( - offsetStore: OffsetStore[Offset], - tag: EventTag[E], - consumerId: ConsumerId - ): Source[Committable[JournalEntry[Offset, E]], NotUsed] = - Source - .single(NotUsed) - .mapAsync(1) { _ => - offsetStore.getOffset(tag.value, consumerId) - } - .flatMapConcat { storedOffset => - currentEventsByTag(tag, storedOffset) - .map(x => Committable(() => offsetStore.setOffset(tag.value, consumerId, x.offset), x)) - } -} diff --git a/core/src/main/scala/aecor/streaming/CassandraAggregateJournal.scala b/core/src/main/scala/aecor/streaming/CassandraAggregateJournal.scala deleted file mode 100644 index 26070d89..00000000 --- a/core/src/main/scala/aecor/streaming/CassandraAggregateJournal.scala +++ /dev/null @@ -1,81 +0,0 @@ -package aecor.streaming - -import java.util.UUID - -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentRepr } -import aecor.data.EventTag -import akka.NotUsed -import akka.actor.ActorSystem -import akka.persistence.cassandra.query.scaladsl.CassandraReadJournal -import akka.persistence.query.{ EventEnvelope2, NoOffset, PersistenceQuery, TimeBasedUUID } -import akka.stream.scaladsl.Source - -import scala.concurrent.{ ExecutionContext, Future } - -class CassandraAggregateJournal(system: ActorSystem, journalIdentifier: String, parallelism: Int)( - implicit executionContext: ExecutionContext -) extends AggregateJournal[UUID] { - - private val readJournal: CassandraReadJournal = - PersistenceQuery(system).readJournalFor[CassandraReadJournal](journalIdentifier) - - private def createSource[E: PersistentDecoder]( - inner: Source[EventEnvelope2, NotUsed] - ): Source[JournalEntry[UUID, E], NotUsed] = - inner.mapAsync(parallelism) { - case EventEnvelope2(eventOffset, persistenceId, sequenceNr, event) => - Future(eventOffset).flatMap { - case TimeBasedUUID(offsetValue) => - event match { - case repr: PersistentRepr => - PersistentDecoder[E] - .decode(repr) - .right - .map { event => - JournalEntry(offsetValue, persistenceId, sequenceNr, event) - } - .fold(Future.failed, Future.successful) - case other => - Future.failed( - new RuntimeException( - s"Unexpected persistent representation $other at sequenceNr = [$sequenceNr], persistenceId = [$persistenceId]" - ) - ) - } - case other => - Future.failed( - new RuntimeException( - s"Unexpected offset of type ${other.getClass} at sequenceNr = [$sequenceNr], persistenceId = [$persistenceId]" - ) - ) - } - } - - def eventsByTag[E: PersistentDecoder]( - tag: EventTag[E], - offset: Option[UUID] - ): Source[JournalEntry[UUID, E], NotUsed] = - createSource( - readJournal - .eventsByTag(tag.value, offset.map(TimeBasedUUID).getOrElse(NoOffset)) - ) - - override def currentEventsByTag[E: PersistentDecoder]( - tag: EventTag[E], - offset: Option[UUID] - ): Source[JournalEntry[UUID, E], NotUsed] = - createSource( - readJournal - .currentEventsByTag(tag.value, offset.map(TimeBasedUUID).getOrElse(NoOffset)) - ) - -} - -object CassandraAggregateJournal { - def apply( - system: ActorSystem, - journalIdentifier: String = CassandraReadJournal.Identifier, - parallelism: Int = 8 - )(implicit executionContext: ExecutionContext): AggregateJournal[UUID] = - new CassandraAggregateJournal(system, journalIdentifier, parallelism) -} diff --git a/core/src/main/scala/aecor/streaming/Committable.scala b/core/src/main/scala/aecor/streaming/Committable.scala deleted file mode 100644 index 49070f21..00000000 --- a/core/src/main/scala/aecor/streaming/Committable.scala +++ /dev/null @@ -1,41 +0,0 @@ -package aecor.streaming - -import cats.{ Applicative, Eval, Functor, Traverse } - -import scala.concurrent.Future - -final case class Committable[+A](commit: () => Future[Unit], value: A) { - def map[B](f: A => B): Committable[B] = copy(value = f(value)) - def traverse[G[_], B](f: A => G[B])(implicit G: Functor[G]): G[Committable[B]] = - G.map(f(value))(b => copy(value = b)) - def foldLeft[B](b: B)(f: (B, A) => B): B = f(b, value) -} - -object Committable { - implicit def catsTraverseInstance: Traverse[Committable] = new Traverse[Committable] { - - override def traverse[G[_], A, B]( - fa: Committable[A] - )(f: (A) => G[B])(implicit G: Applicative[G]): G[Committable[B]] = - fa.traverse(f) - - override def foldLeft[A, B](fa: Committable[A], b: B)(f: (B, A) => B): B = - fa.foldLeft(b)(f) - - override def foldRight[A, B](fa: Committable[A], - lb: Eval[B])(f: (A, Eval[B]) => Eval[B]): Eval[B] = - f(fa.value, lb) - } - implicit def commitInstance[Offset]: Commit[Committable[Offset]] = - new Commit[Committable[Offset]] { - override def commit(a: Committable[Offset]): Future[Unit] = a.commit() - } - def pure[A](a: A): Committable[A] = Committable(() => Future.successful(()), a) - def collector[A, B](pf: PartialFunction[A, B]): PartialFunction[Committable[A], Committable[B]] = { - case c if pf.isDefinedAt(c.value) => c.map(pf) - } -} - -trait Commit[A] { - def commit(a: A): Future[Unit] -} diff --git a/core/src/main/scala/aecor/streaming/OffsetStore.scala b/core/src/main/scala/aecor/streaming/OffsetStore.scala deleted file mode 100644 index 3acfc466..00000000 --- a/core/src/main/scala/aecor/streaming/OffsetStore.scala +++ /dev/null @@ -1,10 +0,0 @@ -package aecor.streaming - -import scala.concurrent.Future - -final case class ConsumerId(value: String) extends AnyVal - -trait OffsetStore[Offset] { - def getOffset(tag: String, consumerId: ConsumerId): Future[Option[Offset]] - def setOffset(tag: String, consumerId: ConsumerId, offset: Offset): Future[Unit] -} diff --git a/core/src/main/scala/aecor/streaming/StreamSupervisor.scala b/core/src/main/scala/aecor/streaming/StreamSupervisor.scala deleted file mode 100644 index 65b41022..00000000 --- a/core/src/main/scala/aecor/streaming/StreamSupervisor.scala +++ /dev/null @@ -1,52 +0,0 @@ -package aecor.streaming - -import aecor.streaming.StreamSupervisor.StreamKillSwitch -import akka.actor.{ ActorSystem, SupervisorStrategy } -import akka.cluster.singleton.{ ClusterSingletonManager, ClusterSingletonManagerSettings } -import akka.pattern.{ BackoffSupervisor, ask } -import akka.stream.Materializer -import akka.stream.scaladsl.{ Flow, Source } -import akka.util.Timeout - -import scala.concurrent.Future -import scala.concurrent.duration.{ FiniteDuration, _ } - -class StreamSupervisor(system: ActorSystem) { - def startClusterSingleton[A, SM, FM]( - name: String, - source: Source[A, SM], - flow: Flow[A, Unit, FM], - settings: StreamSupervisorSettings = - StreamSupervisorSettings(3.seconds, 10.seconds, 0.2, ClusterSingletonManagerSettings(system)) - )(implicit mat: Materializer): StreamKillSwitch = { - import mat.executionContext - val props = ClusterSingletonManager.props( - singletonProps = BackoffSupervisor.propsWithSupervisorStrategy( - StreamSupervisorActor.props(source, flow), - "stream", - settings.minBackoff, - settings.maxBackoff, - settings.randomFactor, - SupervisorStrategy.stoppingStrategy - ), - terminationMessage = StreamSupervisorActor.Shutdown, - settings = settings.clusterSingletonManagerSettings - ) - val ref = system.actorOf(props, name) - StreamKillSwitch { implicit timeout: Timeout => - (ref ? StreamSupervisorActor.Shutdown).map(_ => ()) - } - } -} - -object StreamSupervisor { - def apply(system: ActorSystem): StreamSupervisor = new StreamSupervisor(system) - final case class StreamKillSwitch(trigger: Timeout => Future[Unit]) -} - -final case class StreamSupervisorSettings( - minBackoff: FiniteDuration, - maxBackoff: FiniteDuration, - randomFactor: Double, - clusterSingletonManagerSettings: ClusterSingletonManagerSettings -) diff --git a/core/src/main/scala/aecor/streaming/StreamSupervisorActor.scala b/core/src/main/scala/aecor/streaming/StreamSupervisorActor.scala deleted file mode 100644 index 6c229ff5..00000000 --- a/core/src/main/scala/aecor/streaming/StreamSupervisorActor.scala +++ /dev/null @@ -1,51 +0,0 @@ -package aecor.streaming - -import akka.Done -import akka.actor.{ Actor, ActorLogging, Props, Status } -import akka.stream.scaladsl.{ Flow, Keep, Sink, Source } -import akka.stream.{ KillSwitches, Materializer } - -private[aecor] object StreamSupervisorActor { - def props[A, SM, FM](source: Source[A, SM], - flow: Flow[A, Unit, FM])(implicit mat: Materializer): Props = - Props(classOf[StreamSupervisorActor[A, SM, FM]], source, flow, mat) - - case object Shutdown -} - -private[aecor] class StreamSupervisorActor[A, SM, FM]( - source: Source[A, SM], - flow: Flow[A, Unit, FM] -)(implicit mat: Materializer) - extends Actor - with ActorLogging { - - import akka.pattern.pipe - import context.dispatcher - - val (shutDown, streamUnit) = - source - .viaMat(KillSwitches.single)(Keep.right) - .via(flow) - .toMat(Sink.ignore)(Keep.both) - .run() - - streamUnit pipeTo self - - override def postStop: Unit = - shutDown.shutdown() - - def receive: Receive = { - case Status.Failure(e) => - log.error(e, "Stream failed") - throw e - - case StreamSupervisorActor.Shutdown => - context stop self - context become Actor.ignoringBehavior - - case Done => - throw new IllegalStateException("Stream terminated when it shouldn't") - - } -} diff --git a/core/src/main/scala/aecor/util/Clock.scala b/core/src/main/scala/aecor/util/Clock.scala new file mode 100644 index 00000000..fcf90eb0 --- /dev/null +++ b/core/src/main/scala/aecor/util/Clock.scala @@ -0,0 +1,16 @@ +package aecor.util + +import java.time._ + +import cats.Apply + +trait Clock[F[_]] { + def zone: F[ZoneId] + def instant: F[Instant] + def zonedDateTime(implicit F: Apply[F]): F[ZonedDateTime] = + F.map2(instant, zone)(ZonedDateTime.ofInstant) + def offsetDateTime(implicit F: Apply[F]): F[OffsetDateTime] = + F.map2(instant, zone)(OffsetDateTime.ofInstant) + def localDateTime(implicit F: Apply[F]): F[LocalDateTime] = + F.map2(instant, zone)(LocalDateTime.ofInstant) +} diff --git a/core/src/main/scala/aecor/util/JavaTimeClock.scala b/core/src/main/scala/aecor/util/JavaTimeClock.scala new file mode 100644 index 00000000..5047ef53 --- /dev/null +++ b/core/src/main/scala/aecor/util/JavaTimeClock.scala @@ -0,0 +1,15 @@ +package aecor.util + +import java.time.{ Instant, ZoneId } + +import aecor.effect.Capture + +class JavaTimeClock[F[_]](underlying: java.time.Clock)(implicit F: Capture[F]) extends Clock[F] { + override def zone: F[ZoneId] = F.capture(underlying.getZone) + override def instant: F[Instant] = F.capture(underlying.instant()) +} + +object JavaTimeClock { + def apply[F[_]: Capture](underlying: java.time.Clock): Clock[F] = + new JavaTimeClock[F](underlying) +} diff --git a/core/src/main/scala/aecor/util/KeyValueStore.scala b/core/src/main/scala/aecor/util/KeyValueStore.scala new file mode 100644 index 00000000..2ac3e92c --- /dev/null +++ b/core/src/main/scala/aecor/util/KeyValueStore.scala @@ -0,0 +1,22 @@ +package aecor.util +import cats.{ Functor, ~> } + +trait KeyValueStore[F[_], K, A] { self => + def setValue(key: K, value: A): F[Unit] + def getValue(key: K): F[Option[A]] + + def contramap[K2](f: K2 => K): KeyValueStore[F, K2, A] = new KeyValueStore[F, K2, A] { + override def setValue(key: K2, value: A): F[Unit] = self.setValue(f(key), value) + override def getValue(key: K2): F[Option[A]] = self.getValue(f(key)) + } + def mapK[G[_]](f: F ~> G): KeyValueStore[G, K, A] = new KeyValueStore[G, K, A] { + override def setValue(key: K, value: A): G[Unit] = f(self.setValue(key, value)) + override def getValue(key: K): G[Option[A]] = f(self.getValue(key)) + } + def imap[B](ab: A => B, ba: B => A)(implicit F: Functor[F]): KeyValueStore[F, K, B] = + new KeyValueStore[F, K, B] { + override def setValue(key: K, value: B): F[Unit] = self.setValue(key, ba(value)) + + override def getValue(key: K): F[Option[B]] = F.map(self.getValue(key))(_.map(ab)) + } +} diff --git a/core/src/main/scala/aecor/util/NoopKeyValueStore.scala b/core/src/main/scala/aecor/util/NoopKeyValueStore.scala new file mode 100644 index 00000000..b87ce9f8 --- /dev/null +++ b/core/src/main/scala/aecor/util/NoopKeyValueStore.scala @@ -0,0 +1,16 @@ +package aecor.util + +import cats.Applicative +import cats.implicits._ + +sealed class NoopKeyValueStore[F[_]: Applicative, K, V] extends KeyValueStore[F, K, V] { + override def setValue(key: K, value: V): F[Unit] = + ().pure[F] + + override def getValue(key: K): F[Option[V]] = + none[V].pure[F] +} + +object NoopKeyValueStore { + def apply[F[_]: Applicative, K, V]: NoopKeyValueStore[F, K, V] = new NoopKeyValueStore[F, K, V] +} diff --git a/distributed-processing/src/main/protobuf/DistributedProcessing.proto b/distributed-processing/src/main/protobuf/DistributedProcessing.proto new file mode 100644 index 00000000..50aec790 --- /dev/null +++ b/distributed-processing/src/main/protobuf/DistributedProcessing.proto @@ -0,0 +1,11 @@ +syntax = "proto2"; + +package aecor.distributedprocessing.serialization; +option java_package = "aecor.distributedprocessing.serialization.msg"; +option optimize_for = SPEED; + +message KeepRunning { + required uint32 workerId = 1; +} + + diff --git a/distributed-processing/src/main/resources/reference.conf b/distributed-processing/src/main/resources/reference.conf new file mode 100644 index 00000000..b918bf44 --- /dev/null +++ b/distributed-processing/src/main/resources/reference.conf @@ -0,0 +1,13 @@ +akka { + actor { + serialization-identifiers { + "aecor.distributedprocessing.serialization.MessageSerializer" = 130 + } + serializers { + distributed-processing-message-serializer = "aecor.distributedprocessing.serialization.MessageSerializer" + } + serialization-bindings { + "aecor.distributedprocessing.serialization.Message" = distributed-processing-message-serializer + } + } +} \ No newline at end of file diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/AkkaStreamProcess.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/AkkaStreamProcess.scala new file mode 100644 index 00000000..8722e265 --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/AkkaStreamProcess.scala @@ -0,0 +1,28 @@ +package aecor.distributedprocessing + +import aecor.distributedprocessing.DistributedProcessing._ +import aecor.effect.{ Capture, CaptureFuture } +import akka.NotUsed +import akka.stream.scaladsl.{ Flow, Keep, Sink, Source } +import akka.stream.{ KillSwitches, Materializer } + +object AkkaStreamProcess { + final class Mk[F[_]] { + def apply[A]( + source: Source[A, NotUsed], + flow: Flow[A, Unit, NotUsed] + )(implicit mat: Materializer, F0: CaptureFuture[F], F1: Capture[F]): Process[F] = + Process(run = Capture[F].capture { + val (killSwitch, terminated) = source + .viaMat(KillSwitches.single)(Keep.right) + .via(flow) + .toMat(Sink.ignore)(Keep.both) + .run() + RunningProcess( + CaptureFuture[F].captureFuture(terminated.map(_ => ())(mat.executionContext)), + () => killSwitch.shutdown() + ) + }) + } + def apply[F[_]]: Mk[F] = new Mk[F] +} diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessing.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessing.scala new file mode 100644 index 00000000..48a231a3 --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessing.scala @@ -0,0 +1,82 @@ +package aecor.distributedprocessing + +import java.net.URLEncoder +import java.nio.charset.StandardCharsets + +import aecor.distributedprocessing.DistributedProcessing.{ Process, ProcessKillSwitch } +import aecor.distributedprocessing.DistributedProcessingWorker.KeepRunning +import aecor.effect.{ Async, Capture, CaptureFuture } +import akka.actor.{ ActorSystem, SupervisorStrategy } +import akka.cluster.sharding.{ ClusterSharding, ClusterShardingSettings } +import akka.pattern.{ BackoffSupervisor, ask } +import akka.util.Timeout +import cats.Functor +import cats.implicits._ +import scala.collection.immutable._ +import scala.concurrent.duration.{ FiniteDuration, _ } + +class DistributedProcessing(system: ActorSystem) { + def start[F[_]: Functor: Async: Capture: CaptureFuture](name: String, + processes: Seq[Process[F]], + settings: DistributedProcessingSettings = + DistributedProcessingSettings( + minBackoff = 3.seconds, + maxBackoff = 10.seconds, + randomFactor = 0.2, + shutdownTimeout = 10.seconds, + numberOfShards = 100, + heartbeatInterval = 2.seconds, + clusterShardingSettings = + ClusterShardingSettings(system) + )): F[ProcessKillSwitch[F]] = + Capture[F].capture { + + val props = BackoffSupervisor.propsWithSupervisorStrategy( + DistributedProcessingWorker.props(x => processes(x)), + "worker", + settings.minBackoff, + settings.maxBackoff, + settings.randomFactor, + SupervisorStrategy.stoppingStrategy + ) + + val region = ClusterSharding(system).start( + typeName = name, + entityProps = props, + settings = settings.clusterShardingSettings, + extractEntityId = { + case c @ KeepRunning(workerId) => (workerId.toString, c) + }, + extractShardId = { + case KeepRunning(workerId) => (workerId % settings.numberOfShards).toString + case other => throw new IllegalArgumentException(s"Unexpected messge [$other]") + } + ) + + val regionSupervisor = system.actorOf( + DistributedProcessingSupervisor.props(processes.size, region, settings.heartbeatInterval), + "DistributedProcessingSupervisor-" + URLEncoder.encode(name, StandardCharsets.UTF_8.name()) + ) + implicit val timeout = Timeout(settings.shutdownTimeout) + ProcessKillSwitch { + CaptureFuture[F].captureFuture { + regionSupervisor ? DistributedProcessingSupervisor.GracefulShutdown + }.void + } + } +} + +object DistributedProcessing { + def apply(system: ActorSystem): DistributedProcessing = new DistributedProcessing(system) + final case class ProcessKillSwitch[F[_]](shutdown: F[Unit]) + final case class RunningProcess[F[_]](watchTermination: F[Unit], shutdown: () => Unit) + final case class Process[F[_]](run: F[RunningProcess[F]]) +} + +final case class DistributedProcessingSettings(minBackoff: FiniteDuration, + maxBackoff: FiniteDuration, + randomFactor: Double, + shutdownTimeout: FiniteDuration, + numberOfShards: Int, + heartbeatInterval: FiniteDuration, + clusterShardingSettings: ClusterShardingSettings) diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingSupervisor.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingSupervisor.scala new file mode 100644 index 00000000..a37b8217 --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingSupervisor.scala @@ -0,0 +1,60 @@ +package aecor.distributedprocessing + +import aecor.distributedprocessing.DistributedProcessingSupervisor.{ + GracefulShutdown, + ShutdownCompleted, + Tick +} +import aecor.distributedprocessing.DistributedProcessingWorker.KeepRunning +import akka.actor.{ Actor, ActorLogging, ActorRef, Props, Terminated } +import akka.cluster.sharding.ShardRegion + +import scala.concurrent.duration.{ FiniteDuration, _ } + +object DistributedProcessingSupervisor { + private final case object Tick + final case object GracefulShutdown + final case object ShutdownCompleted + + def props(processCount: Int, shardRegion: ActorRef, heartbeatInterval: FiniteDuration): Props = + Props(new DistributedProcessingSupervisor(processCount, shardRegion, heartbeatInterval)) +} + +final class DistributedProcessingSupervisor(processCount: Int, + shardRegion: ActorRef, + heartbeatInterval: FiniteDuration) + extends Actor + with ActorLogging { + + import context.dispatcher + + private val heartbeat = + context.system.scheduler.schedule(0.seconds, heartbeatInterval, self, Tick) + + context.watch(shardRegion) + + override def postStop(): Unit = { + heartbeat.cancel() + () + } + + override def receive: Receive = { + case Tick => + (0 until processCount).foreach { processId => + shardRegion ! KeepRunning(processId) + } + case Terminated(`shardRegion`) => + context.stop(self) + case GracefulShutdown => + log.info(s"Performing graceful shutdown of [$shardRegion]") + shardRegion ! ShardRegion.GracefulShutdown + val replyTo = sender() + context.become { + case Terminated(`shardRegion`) => + log.info(s"Graceful shutdown completed for [$shardRegion]") + context.stop(self) + replyTo ! ShutdownCompleted + } + + } +} diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingWorker.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingWorker.scala new file mode 100644 index 00000000..1a2a637a --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/DistributedProcessingWorker.scala @@ -0,0 +1,54 @@ +package aecor.distributedprocessing + +import aecor.distributedprocessing.DistributedProcessing._ +import aecor.distributedprocessing.DistributedProcessingWorker.KeepRunning +import aecor.distributedprocessing.serialization.Message +import aecor.effect.Async +import aecor.effect.Async.ops._ +import akka.actor.{ Actor, ActorLogging, Props, Status } +import akka.pattern._ + +private[aecor] object DistributedProcessingWorker { + def props[F[_]: Async](processWithId: Int => Process[F]): Props = + Props(new DistributedProcessingWorker[F](processWithId)) + + final case class KeepRunning(workerId: Int) extends Message +} + +private[aecor] class DistributedProcessingWorker[F[_]: Async](processFor: Int => Process[F]) + extends Actor + with ActorLogging { + import context.dispatcher + + case class ProcessStarted(process: RunningProcess[F]) + case object ProcessTerminated + + var killSwitch: Option[() => Unit] = None + + override def postStop: Unit = + killSwitch.foreach(_.apply()) + + def receive: Receive = { + case KeepRunning(workerId) => + log.info("[{}] Starting process", workerId) + processFor(workerId).run.unsafeRun.map(ProcessStarted) pipeTo self + context.become { + case ProcessStarted(RunningProcess(watchTermination, terminate)) => + log.info("[{}] Process started", workerId) + killSwitch = Some(terminate) + watchTermination.unsafeRun.map(_ => ProcessTerminated) pipeTo self + context.become { + case Status.Failure(e) => + log.error(e, "Process failed") + throw e + + case ProcessTerminated => + throw new IllegalStateException("Process terminated") + } + case Status.Failure(e) => + log.error(e, "Process failed to start") + throw e + case KeepRunning(_) => + } + } +} diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/Message.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/Message.scala new file mode 100644 index 00000000..1e67bc48 --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/Message.scala @@ -0,0 +1,6 @@ +package aecor.distributedprocessing.serialization + +/** + * Marker trait for all protobuf-serializable messages in `aecor.distributedprocessing`. + */ +trait Message diff --git a/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/MessageSerializer.scala b/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/MessageSerializer.scala new file mode 100644 index 00000000..815e06b7 --- /dev/null +++ b/distributed-processing/src/main/scala/aecor/distributedprocessing/serialization/MessageSerializer.scala @@ -0,0 +1,27 @@ +package aecor.distributedprocessing.serialization + +import aecor.distributedprocessing.DistributedProcessingWorker.KeepRunning +import akka.actor.ExtendedActorSystem +import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } + +class MessageSerializer(val system: ExtendedActorSystem) + extends SerializerWithStringManifest + with BaseSerializer { + val KeepRunningManifest = "A" + override def manifest(o: AnyRef): String = o match { + case KeepRunning(_) => KeepRunningManifest + case x => throw new IllegalArgumentException(s"Serialization of [$x] is not supported") + } + + override def toBinary(o: AnyRef): Array[Byte] = o match { + case KeepRunning(workerId) => msg.KeepRunning(workerId).toByteArray + case x => throw new IllegalArgumentException(s"Serialization of [$x] is not supported") + } + + override def fromBinary(bytes: Array[Byte], manifest: String): AnyRef = + manifest match { + case KeepRunningManifest => + KeepRunning(msg.KeepRunning.parseFrom(bytes).workerId) + case other => throw new IllegalArgumentException(s"Unknown manifest [$other]") + } +} diff --git a/example/src/main/protobuf/Account.proto b/example/src/main/protobuf/Account.proto deleted file mode 100644 index 7f69edd8..00000000 --- a/example/src/main/protobuf/Account.proto +++ /dev/null @@ -1,36 +0,0 @@ -syntax = "proto2"; - -option java_package = "com.chesnok.contract.account"; -option optimize_for = SPEED; - -message Event { - message AccountOpened { - required string accountId = 1; - } - message HoldPlaced { - required string accountId = 1; - required string transactionId = 2; - required uint32 amount = 3; - } - message HoldCancelled { - required string accountId = 1; - required string transactionId = 2; - } - message HoldCleared { - required string accountId = 1; - required string transactionId = 2; - required uint32 amount = 3; - } - message AccountCredited { - required string accountId = 1; - required string transactionId = 2; - required uint32 amount = 3; - } - oneof payload { - AccountOpened accountOpened = 1; - HoldPlaced holdPlaced = 2; - HoldCancelled holdCancelled = 3; - HoldCleared holdCleared = 4; - AccountCredited accountCredited = 5; - } -} diff --git a/example/src/main/resources/application.conf b/example/src/main/resources/application.conf index 4c5d3d23..f99d8ab5 100644 --- a/example/src/main/resources/application.conf +++ b/example/src/main/resources/application.conf @@ -4,7 +4,7 @@ http { } cluster { - system-name = "aecor-example" + system-name = "test" host = "127.0.0.1" port = 51000 } @@ -23,10 +23,10 @@ cassandra-snapshot-store { } cassandra-query-journal { - refresh-interval = 100ms + refresh-interval = 50ms first-time-bucket = "20160812" delayed-event-timeout = 30s - eventual-consistency-delay = 1s + eventual-consistency-delay = 0ms } akka { @@ -36,6 +36,18 @@ akka { logging-filter = "akka.event.slf4j.Slf4jLoggingFilter" actor { provider = "akka.cluster.ClusterActorRefProvider" + kryo { + type = "nograph" + idstrategy = "default" + implicit-registration-logging = true + } + serializers { + # Define kryo serializer + kryo = "com.romix.akka.serialization.kryo.KryoSerializer" + } + serialization-bindings { + "scala.Serializable" = kryo + } } persistence { journal.plugin = "cassandra-journal" @@ -62,8 +74,7 @@ akka { cluster { seed-nodes = [ "akka://"${cluster.system-name}"@127.0.0.1:51000" -// "akka://"${cluster.system-name}"@127.0.0.1:51001", -// "akka://"${cluster.system-name}"@127.0.0.1:51002" ] } + extensions = ["com.romix.akka.serialization.kryo.KryoSerializationExtension$"] } \ No newline at end of file diff --git a/example/src/main/resources/logback.xml b/example/src/main/resources/logback.xml index 366c627c..bfa22d98 100644 --- a/example/src/main/resources/logback.xml +++ b/example/src/main/resources/logback.xml @@ -6,7 +6,7 @@ - + diff --git a/example/src/main/resources/requests/AuthorizePayment.json b/example/src/main/resources/requests/AuthorizePayment.json deleted file mode 100644 index b4e0c6ee..00000000 --- a/example/src/main/resources/requests/AuthorizePayment.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "AuthorizePayment": { - "cardAuthorizationId": "2", - "accountId": "DMITRY", - "amount": 5000, - "acquireId": 1, - "terminalId": 1 - } -} diff --git a/example/src/main/resources/requests/CreditAccount.json b/example/src/main/resources/requests/CreditAccount.json deleted file mode 100644 index 1aaf12de..00000000 --- a/example/src/main/resources/requests/CreditAccount.json +++ /dev/null @@ -1,7 +0,0 @@ -{ - "CreditAccount": { - "accountId": "DMITRY", - "transactionId": "credit-evian-1", - "amount": "1000000" - } -} \ No newline at end of file diff --git a/example/src/main/resources/requests/OpenAccount.json b/example/src/main/resources/requests/OpenAccount.json deleted file mode 100644 index b456c1cc..00000000 --- a/example/src/main/resources/requests/OpenAccount.json +++ /dev/null @@ -1,5 +0,0 @@ -{ - "OpenAccount": { - "accountId": "DMITRY" - } -} \ No newline at end of file diff --git a/example/src/main/scala/aecor/example/AccountAPI.scala b/example/src/main/scala/aecor/example/AccountAPI.scala deleted file mode 100644 index 0e9f0dc0..00000000 --- a/example/src/main/scala/aecor/example/AccountAPI.scala +++ /dev/null @@ -1,74 +0,0 @@ -package aecor.example - -import aecor.example.domain._ -import akka.Done -import akka.http.scaladsl.model.StatusCodes -import akka.http.scaladsl.server.Directives._ -import akka.http.scaladsl.server.Route -import cats.implicits._ -import cats.~> -import de.heikoseeberger.akkahttpcirce.CirceSupport._ -import io.circe.generic.JsonCodec - -import scala.concurrent.{ExecutionContext, Future} - -class AccountAPI(account: AccountAggregateOp ~> Future) { - - import AccountAPI._ - - def openAccount(dto: DTO.OpenAccount)( - implicit ec: ExecutionContext): Future[String Either Done] = dto match { - case DTO.OpenAccount(accountId) => - account(AccountAggregateOp.OpenAccount(AccountId(accountId))) - .map(_.leftMap(_.toString)) - } - - def creditAccount(dto: DTO.CreditAccount)( - implicit ec: ExecutionContext): Future[String Either Done] = dto match { - case DTO.CreditAccount(accountId, transactionId, amount) => - account( - AccountAggregateOp.CreditAccount(AccountId(accountId), - TransactionId(transactionId), - Amount(amount))) - .map(_.leftMap(_.toString)) - } -} - -object AccountAPI { - - @JsonCodec sealed trait DTO - - object DTO { - case class CreditAccount(accountId: String, - transactionId: String, - amount: Long) - extends DTO - case class OpenAccount(accountId: String) extends DTO - } - - val route: AccountAPI => Route = { api => - path("accounts") { - extractExecutionContext { implicit ec => - post { - entity(as[DTO]) { - case dto: DTO.CreditAccount => - complete { - api.creditAccount(dto).map { - case Left(e) => StatusCodes.BadRequest -> e.toString - case Right(result) => StatusCodes.OK -> "" - } - } - case dto: DTO.OpenAccount => - complete { - api.openAccount(dto).map { - case Left(e) => StatusCodes.BadRequest -> e.toString - case Right(result) => StatusCodes.OK -> "" - } - } - } - } - } - } - } - -} diff --git a/example/src/main/scala/aecor/example/AccountEndpoint.scala b/example/src/main/scala/aecor/example/AccountEndpoint.scala new file mode 100644 index 00000000..a66c1aa7 --- /dev/null +++ b/example/src/main/scala/aecor/example/AccountEndpoint.scala @@ -0,0 +1,44 @@ +package aecor.example + +import aecor.example.MonixSupport._ +import aecor.example.domain.account.{ AccountAggregate, AccountId } +import akka.http.scaladsl.model.StatusCodes +import akka.http.scaladsl.server.Directives._ +import akka.http.scaladsl.server.Route +import cats.implicits._ +import de.heikoseeberger.akkahttpcirce.FailFastCirceSupport._ +import io.circe.generic.JsonCodec +import monix.eval.Task +import AnyValCirceEncoding._ +import aecor.example.AccountEndpoint.AccountApiRequest.OpenAccountRequest + +class AccountEndpoint(account: AccountAggregate[Task]) { + + def openAccount(accountId: AccountId): Task[String Either Unit] = + account + .openAccount(accountId) + .map(_.leftMap(_.toString)) +} + +object AccountEndpoint { + + sealed abstract class AccountApiRequest + object AccountApiRequest { + @JsonCodec final case class OpenAccountRequest(accountId: AccountId) extends AccountApiRequest + } + + def route(api: AccountEndpoint): Route = + pathPrefix("account") { + post { + (path("open") & entity(as[AccountApiRequest.OpenAccountRequest])) { + case OpenAccountRequest(accountId) => + complete { + api.openAccount(accountId).map { + case Left(e) => StatusCodes.BadRequest -> e.toString + case Right(result) => StatusCodes.OK -> "" + } + } + } + } + } +} diff --git a/example/src/main/scala/aecor/example/AnyValCirceEncoding.scala b/example/src/main/scala/aecor/example/AnyValCirceEncoding.scala new file mode 100644 index 00000000..c6c96b65 --- /dev/null +++ b/example/src/main/scala/aecor/example/AnyValCirceEncoding.scala @@ -0,0 +1,26 @@ +package aecor.example + +import io.circe.{ Decoder, Encoder } +import shapeless.Unwrapped + +trait AnyValCirceEncoding { + implicit def anyValEncoder[V, U](implicit ev: V <:< AnyVal, + V: Unwrapped.Aux[V, U], + encoder: Encoder[U]): Encoder[V] = { + val _ = ev + encoder.contramap(V.unwrap) + } + + implicit def anyValDecoder[V, U](implicit ev: V <:< AnyVal, + V: Unwrapped.Aux[V, U], + decoder: Decoder[U]): Decoder[V] = { + val _ = ev + decoder.map(V.wrap) + } +} + +object AnyValCirceEncoding extends AnyValCirceEncoding + +object CirceSupport + extends de.heikoseeberger.akkahttpcirce.FailFastCirceSupport + with AnyValCirceEncoding diff --git a/example/src/main/scala/aecor/example/App.scala b/example/src/main/scala/aecor/example/App.scala index fef8e05a..cb96c72e 100644 --- a/example/src/main/scala/aecor/example/App.scala +++ b/example/src/main/scala/aecor/example/App.scala @@ -1,16 +1,139 @@ package aecor.example +import java.time.Clock + +import aecor.data._ +import aecor.distributedprocessing.{ AkkaStreamProcess, DistributedProcessing } +import aecor.effect.monix._ +import aecor.example.domain.TransactionProcess.{ Input, TransactionProcessFailure } +import aecor.example.domain._ +import aecor.example.domain.account.{ AccountAggregate, AccountEvent, EventsourcedAccountAggregate } +import aecor.example.domain.transaction.{ + EventsourcedTransactionAggregate, + TransactionAggregate, + TransactionEvent +} +import aecor.runtime.akkapersistence.{ AkkaPersistenceRuntime, CassandraOffsetStore } +import aecor.util.JavaTimeClock import akka.actor.ActorSystem +import akka.event.Logging +import akka.http.scaladsl.Http +import akka.http.scaladsl.model.StatusCodes +import akka.http.scaladsl.server.Directives.{ complete, get, path, _ } +import akka.persistence.cassandra.DefaultJournalCassandraSession +import akka.stream.scaladsl.Flow +import akka.stream.{ ActorMaterializer, Materializer } import com.typesafe.config.ConfigFactory +import monix.cats._ +import monix.eval.Task +import monix.execution.Scheduler.Implicits.global object App { def main(args: Array[String]): Unit = { + val config = ConfigFactory.load() - val actorSystem = ActorSystem(config.getString("cluster.system-name")) - actorSystem.actorOf(AppActor.props, "root") - actorSystem.registerOnTermination { + implicit val system: ActorSystem = ActorSystem(config.getString("cluster.system-name")) + system.registerOnTermination { System.exit(1) } + implicit val materializer: Materializer = ActorMaterializer() + + val taskClock = JavaTimeClock[Task](Clock.systemUTC()) + + val offsetStoreConfig = + CassandraOffsetStore.Config(config.getString("cassandra-journal.keyspace")) + + val cassandraSession = + DefaultJournalCassandraSession( + system, + "app-session", + CassandraOffsetStore.createTable(offsetStoreConfig) + ) + + val transactionAggregateRuntime = AkkaPersistenceRuntime( + system, + "Transaction", + Correlation[TransactionAggregate.TransactionAggregateOp](_.transactionId.value), + EventsourcedTransactionAggregate.behavior[Task](taskClock), + EventsourcedTransactionAggregate.tagging + ) + + val startTransactions: Task[TransactionAggregate[Task]] = + transactionAggregateRuntime.start + .map(TransactionAggregate.fromFunctionK) + + val offsetStore = CassandraOffsetStore[Task](cassandraSession, offsetStoreConfig) + + val accountAggregateRuntime = AkkaPersistenceRuntime( + system, + "Account", + Correlation[AccountAggregate.AccountAggregateOp](_.accountId.value), + EventsourcedAccountAggregate.behavior[Task](taskClock), + Tagging.partitioned[AccountEvent](20, EventTag("Account"))(_.accountId.value) + ) + + val startAccounts: Task[AccountAggregate[Task]] = + accountAggregateRuntime.start + .map(AccountAggregate.fromFunctionK) + + def startTransactionProcessing( + accounts: AccountAggregate[Task], + transactions: TransactionAggregate[Task] + ): Task[DistributedProcessing.ProcessKillSwitch[Task]] = { + val failure = TransactionProcessFailure.withMonadError[Task] + + val process: (Input) => Task[Unit] = + TransactionProcess(transactions, accounts, failure) + + val processes = + EventsourcedTransactionAggregate.tagging.tags.map { tag => + AkkaStreamProcess[Task]( + transactionAggregateRuntime.journal + .committable(offsetStore) + .eventsByTag(tag, ConsumerId("processing")) + .map(_.map(_.event)), + Flow[Committable[Task, TransactionEvent]] + .mapAsync(30) { + _.traverse(process).runAsync + } + .mapAsync(1)(_.commit.runAsync) + ) + } + + DistributedProcessing(system).start[Task]("TransactionProcessing", processes) + } + + def startHttpServer(accounts: AccountAggregate[Task], + transactions: TransactionAggregate[Task]): Task[Http.ServerBinding] = + Task.defer { + Task.fromFuture { + val transactionEndpoint = + new TransactionEndpoint(transactions, Logging(system, classOf[TransactionEndpoint])) + val accountApi = new AccountEndpoint(accounts) + + val route = path("check") { + get { + complete(StatusCodes.OK) + } + } ~ + TransactionEndpoint.route(transactionEndpoint) ~ + AccountEndpoint.route(accountApi) + + Http() + .bindAndHandle(route, config.getString("http.interface"), config.getInt("http.port")) + } + } + + val app = for { + transactions <- startTransactions + accounts <- startAccounts + _ <- startTransactionProcessing(accounts, transactions) + bindResult <- startHttpServer(accounts, transactions) + _ = system.log.info("Bind result [{}]", bindResult) + } yield () + + app.runAsync + () } } diff --git a/example/src/main/scala/aecor/example/AppActor.scala b/example/src/main/scala/aecor/example/AppActor.scala deleted file mode 100644 index 15063c90..00000000 --- a/example/src/main/scala/aecor/example/AppActor.scala +++ /dev/null @@ -1,121 +0,0 @@ -package aecor.example - -import java.time.Clock - -import aecor.aggregate._ -import aecor.example.domain.CardAuthorizationAggregateEvent.CardAuthorizationCreated -import aecor.example.domain._ -import aecor.streaming._ -import akka.NotUsed -import akka.actor.{ Actor, ActorLogging, ActorSystem, Props } -import akka.event.Logging -import akka.http.scaladsl.Http -import akka.http.scaladsl.model.StatusCodes -import akka.http.scaladsl.server.Directives._ -import akka.persistence.cassandra.DefaultJournalCassandraSession -import akka.stream.scaladsl.{ Flow, Sink } -import akka.stream.{ ActorMaterializer, Materializer } -import cats.~> -import com.typesafe.config.Config - -import scala.concurrent.Future - -object AppActor { - def props: Props = Props(new AppActor) -} - -class AppActor extends Actor with ActorLogging { - override def receive: Receive = Actor.emptyBehavior - - implicit val system: ActorSystem = context.system - implicit val materializer: Materializer = ActorMaterializer() - - import materializer.executionContext - - val config: Config = system.settings.config - - val offsetStoreConfig = - CassandraOffsetStore.Config(config.getString("cassandra-journal.keyspace")) - - val cassandraSession = - DefaultJournalCassandraSession( - system, - "app-session", - CassandraOffsetStore.createTable(offsetStoreConfig) - ) - - val offsetStore = CassandraOffsetStore(cassandraSession, offsetStoreConfig) - - val journal = CassandraAggregateJournal(system) - - val authorizationRegion: CardAuthorizationAggregateOp ~> Future = - AkkaRuntime(system).start( - CardAuthorizationAggregate.entityName, - CardAuthorizationAggregate.commandHandler, - CardAuthorizationAggregate.correlation, - Tagging(CardAuthorizationAggregate.entityNameTag) - ) - - val accountRegion: AccountAggregateOp ~> Future = - AkkaRuntime(system).start( - AccountAggregate.entityName, - AccountAggregate.commandHandler(Clock.systemUTC()), - AccountAggregate.correlation, - Tagging(AccountAggregate.entityNameTag) - ) - - val scheduleEntityName = "Schedule3" - - val cardAuthorizationEventStream = - new DefaultEventStream( - system, - journal - .eventsByTag[CardAuthorizationAggregateEvent]( - CardAuthorizationAggregate.entityNameTag, - Option.empty - ) - .map(_.event) - ) - - val authorizePaymentAPI = new AuthorizePaymentAPI( - authorizationRegion, - cardAuthorizationEventStream, - Logging(system, classOf[AuthorizePaymentAPI]) - ) - val accountApi = new AccountAPI(accountRegion) - - import freek._ - - def authorizationProcessFlow[PassThrough] - : Flow[(CardAuthorizationCreated, PassThrough), PassThrough, NotUsed] = - AuthorizationProcess.flow(8, accountRegion :&: authorizationRegion) - - journal - .committableEventsByTag( - offsetStore, - CardAuthorizationAggregate.entityNameTag, - ConsumerId("processing") - ) - .collect { - case x @ Committable(_, JournalEntry(offset, _, _, e: CardAuthorizationCreated)) => - (e, x) - } - .via(authorizationProcessFlow) - .mapAsync(1)(_.commit()) - .runWith(Sink.ignore) - - val route = path("check") { - get { - complete(StatusCodes.OK) - } - } ~ - AuthorizePaymentAPI.route(authorizePaymentAPI) ~ - AccountAPI.route(accountApi) - - Http() - .bindAndHandle(route, config.getString("http.interface"), config.getInt("http.port")) - .onComplete { result => - log.info("Bind result [{}]", result) - } - -} diff --git a/example/src/main/scala/aecor/example/AuthorizePaymentAPI.scala b/example/src/main/scala/aecor/example/AuthorizePaymentAPI.scala deleted file mode 100644 index a043d346..00000000 --- a/example/src/main/scala/aecor/example/AuthorizePaymentAPI.scala +++ /dev/null @@ -1,118 +0,0 @@ -package aecor.example - -import aecor.example.AuthorizePaymentAPI._ -import aecor.example.domain.CardAuthorizationAggregateEvent.{ - CardAuthorizationAccepted, - CardAuthorizationDeclined -} -import aecor.example.domain.CardAuthorizationAggregateOp.{ - CreateCardAuthorization, - CreateCardAuthorizationRejection -} -import aecor.example.domain._ -import akka.event.LoggingAdapter -import akka.http.scaladsl.model.StatusCodes -import akka.http.scaladsl.server.Directives._ -import akka.http.scaladsl.server.Route -import cats.~> -import de.heikoseeberger.akkahttpcirce.CirceSupport._ -import io.circe.generic.JsonCodec - -import scala.concurrent.duration._ -import scala.concurrent.{ExecutionContext, Future} - -class AuthorizePaymentAPI( - authorization: CardAuthorizationAggregateOp ~> Future, - eventStream: EventStream[CardAuthorizationAggregateEvent], - log: LoggingAdapter) { - - import DTO._ - - def authorizePayment(dto: AuthorizePayment)( - implicit ec: ExecutionContext): Future[ - Either[CreateCardAuthorizationRejection, AuthorizePaymentAPI.ApiResult]] = - dto match { - case AuthorizePayment(cardAuthorizationId, - accountId, - amount, - acquireId, - terminalId) => - val command = CreateCardAuthorization( - CardAuthorizationId(cardAuthorizationId), - AccountId(accountId), - Amount(amount), - AcquireId(acquireId), - TerminalId(terminalId) - ) - log.debug("Sending command [{}]", command) - val start = System.nanoTime() - eventStream - .registerObserver(30.seconds) { - case e: CardAuthorizationDeclined - if e.cardAuthorizationId.value == cardAuthorizationId => - AuthorizePaymentAPI.ApiResult.Declined(e.reason.toString) - case e: CardAuthorizationAccepted - if e.cardAuthorizationId.value == cardAuthorizationId => - AuthorizePaymentAPI.ApiResult.Authorized - } - .flatMap { observer => - authorization(command).flatMap { - case Left(rejection) => Future.successful(Left(rejection)) - case _ => observer.result.map(Right(_)) - }.map { x => - log.debug("Command [{}] processed with result [{}] in [{}]", - command, - x, - (System.nanoTime() - start) / 1000000) - x - } - } - - } -} - -object AuthorizePaymentAPI { - - sealed trait ApiResult - object ApiResult { - case object Authorized extends ApiResult - case class Declined(reason: String) extends ApiResult - } - - @JsonCodec sealed trait DTO - - object DTO { - - case class AuthorizePayment(cardAuthorizationId: String, - accountId: String, - amount: Long, - acquireId: Long, - terminalId: Long) - extends DTO - - } - - val route: AuthorizePaymentAPI => Route = { api => - path("authorization") { - extractExecutionContext { implicit ec => - post { - entity(as[DTO]) { - case dto: DTO.AuthorizePayment => - complete { - api.authorizePayment(dto).map { - case Left(e) => StatusCodes.BadRequest -> e.toString - case Right(result) => - result match { - case ApiResult.Authorized => - StatusCodes.OK -> "Authorized" - case ApiResult.Declined(reason) => - StatusCodes.BadRequest -> s"Declined: $reason" - } - } - } - } - } - } - } - } -} diff --git a/example/src/main/scala/aecor/example/EventStream.scala b/example/src/main/scala/aecor/example/EventStream.scala deleted file mode 100644 index d51d92cd..00000000 --- a/example/src/main/scala/aecor/example/EventStream.scala +++ /dev/null @@ -1,109 +0,0 @@ -package aecor.example - -import java.util.UUID -import java.util.concurrent.TimeoutException - -import aecor.example.EventStream.ObserverControl -import aecor.example.EventStreamObserverRegistry._ -import akka.Done -import akka.actor.{Actor, ActorLogging, ActorSystem, Props} -import akka.stream.Materializer -import akka.stream.scaladsl.{Sink, Source} -import akka.util.Timeout - -import scala.concurrent.duration.FiniteDuration -import scala.concurrent.{Future, Promise} - -object EventStream { - - case class ObserverControl[A](id: ObserverId, result: Future[A]) - - type ObserverId = String -} - -trait EventStream[Event] { - def registerObserver[A](timeout: FiniteDuration)(f: PartialFunction[Event, A]): Future[ObserverControl[A]] -} - -class DefaultEventStream[Event](actorSystem: ActorSystem, source: Source[Event, Any])(implicit materializer: Materializer) extends EventStream[Event] { - - import akka.pattern.ask - - val actor = actorSystem.actorOf(Props(new EventStreamObserverRegistry[Event]), "event-stream-observer-registry") - source.map(HandleEvent(_)).runWith(Sink.actorRefWithAck(actor, Init, Done, ShutDown)) - - override def registerObserver[A](timeout: FiniteDuration)(f: PartialFunction[Event, A]): Future[ObserverControl[A]] = { - import materializer.executionContext - implicit val askTimeout = Timeout(timeout) - (actor ? RegisterObserver(f, timeout)).mapTo[ObserverRegistered[A]].map(_.control) - } -} - -object EventStreamObserverRegistry { - - sealed trait Command[+Event] - - case object Init extends Command[Nothing] - - case class RegisterObserver[Event, A](f: PartialFunction[Event, A], timeout: FiniteDuration) extends Command[Event] - - case class DeregisterObserver(id: String) extends Command[Nothing] - - case class HandleEvent[Event](event: Event) extends Command[Event] - - case object ShutDown extends Command[Nothing] - - case class ObserverRegistered[A](control: ObserverControl[A]) - -} - -class EventStreamObserverRegistry[Event] extends Actor with ActorLogging { - - import EventStreamObserverRegistry._ - - def scheduler = context.system.scheduler - - implicit def executionContext = context.dispatcher - - case class Observer(f: PartialFunction[Event, Any], promise: Promise[Any]) { - def handleEvent(event: Event): Boolean = { - val handled = f.isDefinedAt(event) - if (handled) { - promise.success(f(event)) - } - handled - } - } - - var observers = Map.empty[String, Observer] - - override def receive: Receive = { - case command: Command[Event] => handleCommand(command) - } - - def handleCommand(command: Command[Event]): Unit = command match { - case Init => - sender() ! Done - case RegisterObserver(f, timeout) => - val id = UUID.randomUUID().toString - val promise = Promise[Any] - observers = observers.updated(id, Observer(f.asInstanceOf[PartialFunction[Event, Any]], promise)) - scheduler.scheduleOnce(timeout) { - if (!promise.isCompleted) { - promise.failure(new TimeoutException()) - self ! DeregisterObserver(id) - } - } - sender() ! ObserverRegistered(ObserverControl(id, promise.future)) - case HandleEvent(event) => - observers = observers.filterNot { - case (id, observer) => observer.handleEvent(event) - } - sender() ! Done - case DeregisterObserver(id) => - observers = observers - id - case ShutDown => - observers.values.foreach(_.promise.failure(new TimeoutException())) - sender() ! Done - } -} diff --git a/example/src/main/scala/aecor/example/MonixSupport.scala b/example/src/main/scala/aecor/example/MonixSupport.scala new file mode 100644 index 00000000..58922074 --- /dev/null +++ b/example/src/main/scala/aecor/example/MonixSupport.scala @@ -0,0 +1,22 @@ +package aecor.example + +import akka.http.scaladsl.marshalling.{ Marshaller, ToResponseMarshallable, ToResponseMarshaller } +import monix.eval.Task +import monix.execution.Scheduler +trait MonixSupport { + implicit def taskToResponseMarshallable[A]( + task: Task[A] + )(implicit A: ToResponseMarshaller[A]): ToResponseMarshallable = + new ToResponseMarshallable { + override implicit def marshaller: ToResponseMarshaller[Task[A]] = + Marshaller { implicit ec => task => + task.runAsync(Scheduler(ec)).flatMap(A(_)) + } + + override def value: Task[A] = task + + override type T = Task[A] + } +} + +object MonixSupport extends MonixSupport diff --git a/example/src/main/scala/aecor/example/ScheduleApp.scala b/example/src/main/scala/aecor/example/ScheduleApp.scala index cdbed265..1d5d4027 100644 --- a/example/src/main/scala/aecor/example/ScheduleApp.scala +++ b/example/src/main/scala/aecor/example/ScheduleApp.scala @@ -1,30 +1,38 @@ package aecor.example -import java.time.{ Clock, LocalDate, LocalDateTime } +import java.time.{ Clock, LocalDate } import java.util.UUID +import aecor.data.ConsumerId +import aecor.distributedprocessing.{ AkkaStreamProcess, DistributedProcessing } +import aecor.effect.Async.ops._ +import aecor.effect.monix._ +import aecor.effect.{ Async, Capture, CaptureFuture } +import aecor.runtime.akkapersistence.CassandraOffsetStore import aecor.schedule.{ CassandraScheduleEntryRepository, Schedule } -import aecor.streaming.{ CassandraAggregateJournal, CassandraOffsetStore, ConsumerId } -import akka.Done -import akka.actor.{ ActorSystem } +import aecor.util.JavaTimeClock +import akka.NotUsed +import akka.actor.ActorSystem import akka.persistence.cassandra.{ CassandraSessionInitSerialization, DefaultJournalCassandraSession } -import akka.stream.{ ActorMaterializer } -import akka.stream.scaladsl.{ Sink, Source } -import cats.data.Reader +import akka.stream.ActorMaterializer +import akka.stream.scaladsl.{ Flow, Sink, Source } +import cats.implicits._ +import cats.{ Functor, Monad } +import monix.eval.Task +import monix.execution.Scheduler +import monix.cats._ -import scala.concurrent.Future +import scala.concurrent.Await import scala.concurrent.duration._ object ScheduleApp extends App { - implicit val system = ActorSystem("aecor-example") + implicit val system = ActorSystem("test") implicit val materializer = ActorMaterializer() - - val clock = Clock.systemUTC() - - import materializer.executionContext + implicit val scheduler = Scheduler(materializer.executionContext) + def clock[F[_]: Capture] = JavaTimeClock[F](Clock.systemUTC()) val offsetStoreConfig = CassandraOffsetStore.Config("aecor_example") val scheduleEntryRepositoryQueries = @@ -37,56 +45,79 @@ object ScheduleApp extends App { CassandraScheduleEntryRepository.init(scheduleEntryRepositoryQueries) ) ) - val offsetStore = CassandraOffsetStore(cassandraSession, offsetStoreConfig) - val scheduleEntryRepository = - CassandraScheduleEntryRepository(cassandraSession, scheduleEntryRepositoryQueries) - def runSchedule: Reader[Unit, Schedule] = + def runSchedule[F[_]: Async: CaptureFuture: Capture: Monad]: F[Schedule[F]] = Schedule.start( entityName = "Schedule", - clock = clock, dayZero = LocalDate.of(2016, 5, 10), - bucketLength = 1.day, - refreshInterval = 100.millis, - eventualConsistencyDelay = 5.seconds, - repository = scheduleEntryRepository, - aggregateJournal = CassandraAggregateJournal(system), - offsetStore = offsetStore + clock = clock, + repository = + CassandraScheduleEntryRepository[F](cassandraSession, scheduleEntryRepositoryQueries), + offsetStore = CassandraOffsetStore(cassandraSession, offsetStoreConfig) ) - def runAdder(schedule: Schedule): Reader[Unit, Any] = - Reader { _ => + def runAdder[F[_]: Async: Capture: Monad](schedule: Schedule[F]): F[Unit] = + Capture[F].capture { Source - .tick(0.seconds, 200.millis, ()) + .tick(0.seconds, 2.seconds, ()) .mapAsync(1) { _ => - schedule.addScheduleEntry( - "Test", - UUID.randomUUID().toString, - "test", - LocalDateTime.now(clock).plusSeconds(20) - ) + Async[F].unsafeRun { + clock[F].localDateTime.flatMap { now => + schedule.addScheduleEntry( + "Test", + UUID.randomUUID().toString, + "test", + now.plusSeconds(20) + ) + } + + } } .runWith(Sink.ignore) - } + }.void - def runEventWatch(schedule: Schedule): Reader[Unit, Future[Done]] = - Reader { _ => + def runEventWatch[F[_]: Async: Capture: Functor](schedule: Schedule[F]): F[Unit] = + Capture[F].capture { schedule .committableScheduleEvents("SubscriptionInvoicing", ConsumerId("println")) .mapAsync(1) { x => println(x.value) - x.commit() + x.commit.unsafeRun } .runWith(Sink.ignore) - } + }.void - val app: Reader[Unit, Unit] = + def mkApp[F[_]: Async: CaptureFuture: Capture: Monad]: F[Unit] = for { - schedule <- runSchedule - _ <- runAdder(schedule) -// _ <- runRepositoryScanStream -// _ <- runEventWatch(schedule) + schedule <- runSchedule[F] + _ <- runAdder[F](schedule) + _ <- runEventWatch[F](schedule) } yield () - app.run(()) + val app: Task[Unit] = + mkApp[Task] + + val processes = (0 to 10).map { x => + AkkaStreamProcess[Task]( + Source.tick(0.seconds, 2.seconds, x).mapMaterializedValue(_ => NotUsed), + Flow[Int].map { x => + system.log.info(s"Worker $x") + () + } + ) + } + + val distributed = DistributedProcessing(system) + .start[Task]("TestProcesses", processes) + + val app2: Task[Unit] = for { + killswtich <- distributed + x <- killswtich.shutdown.delayExecution(10.seconds) + _ <- { + system.log.info(s"$x") + app2 + } + } yield () + + Await.result(app.runAsync, Duration.Inf) } diff --git a/example/src/main/scala/aecor/example/TransactionEndpoint.scala b/example/src/main/scala/aecor/example/TransactionEndpoint.scala new file mode 100644 index 00000000..65c03274 --- /dev/null +++ b/example/src/main/scala/aecor/example/TransactionEndpoint.scala @@ -0,0 +1,126 @@ +package aecor.example + +import java.util.UUID + +import aecor.example.MonixSupport._ +import aecor.example.TransactionEndpoint.TransactionEndpointRequest.CreateTransactionRequest +import aecor.example.TransactionEndpoint._ +import aecor.example.domain._ +import aecor.example.domain.account.{ AccountId, EventsourcedAccountAggregate } +import aecor.example.domain.transaction._ +import akka.event.LoggingAdapter +import akka.http.scaladsl.marshalling.ToResponseMarshallable +import akka.http.scaladsl.model.StatusCodes +import akka.http.scaladsl.server.Directives._ +import akka.http.scaladsl.server.Route +import CirceSupport._ +import monix.eval.Task +import io.circe.Decoder +import io.circe.generic.decoding.DerivedDecoder +import shapeless.Lazy + +import scala.concurrent.duration._ + +class TransactionEndpoint(transactions: TransactionAggregate[Task], log: LoggingAdapter) { + + import TransactionEndpointRequest._ + + def authorizePayment(request: CreateTransactionRequest): Task[TransactionEndpoint.ApiResult] = + request match { + case CreateTransactionRequest(transactionId, fromAccountId, toAccountId, amount) => + log.debug("Processing request [{}]", request) + val start = System.nanoTime() + transactions + .createTransaction(transactionId, fromAccountId, toAccountId, amount) + .flatMap { _ => + transactions + .getTransactionInfo(transactionId) + .flatMap { + case Some(t) => Task.pure(t) + case None => Task.raiseError(new IllegalStateException("Something went bad")) + } + .delayExecution(5.millis) + .restartUntil(_.succeeded.isDefined) + .timeout(30.seconds) + .map(_.succeeded.get) + } + .map { succeeded => + if (succeeded) { + ApiResult.Authorized + } else { + ApiResult.Declined("You suck") + } + } + .map { x => + log.debug( + "Request [{}] processed with result [{}] in [{}ms]", + request, + x, + (System.nanoTime() - start) / 1000000 + ) + x + } + + } +} + +object TransactionEndpoint { + + sealed trait ApiResult + object ApiResult { + case object Authorized extends ApiResult + case class Declined(reason: String) extends ApiResult + } + + sealed trait TransactionEndpointRequest + + object TransactionEndpointRequest { + + case class CreateTransactionRequest(transactionId: TransactionId, + from: From[AccountId], + to: To[AccountId], + amount: Amount) + extends TransactionEndpointRequest + + } + + implicit def requestDecoder[A <: TransactionEndpointRequest]( + implicit A: Lazy[DerivedDecoder[A]] + ): Decoder[A] = A.value + + def route(api: TransactionEndpoint): Route = + pathPrefix("transaction") { + post { + (path("create") & entity(as[CreateTransactionRequest])) { request => + complete { + api.authorizePayment(request).map[ToResponseMarshallable] { + case ApiResult.Authorized => + StatusCodes.OK -> "Authorized" + case ApiResult.Declined(reason) => + StatusCodes.BadRequest -> s"Declined: $reason" + } + } + } ~ path("test") { + complete { + api + .authorizePayment( + CreateTransactionRequest( + TransactionId(UUID.randomUUID.toString), +// From(AccountId("foo" + scala.util.Random.nextInt(20))), + From(EventsourcedAccountAggregate.rootAccountId), + To(AccountId("foo" + scala.util.Random.nextInt(20))), + Amount(1) + ) + ) + .map[ToResponseMarshallable] { + case ApiResult.Authorized => + StatusCodes.OK -> "Authorized" + case ApiResult.Declined(reason) => + StatusCodes.BadRequest -> s"Declined: $reason" + } + } + } + } + } + +} diff --git a/example/src/main/scala/aecor/example/domain/AccountAggregate.scala b/example/src/main/scala/aecor/example/domain/AccountAggregate.scala deleted file mode 100644 index 9e586469..00000000 --- a/example/src/main/scala/aecor/example/domain/AccountAggregate.scala +++ /dev/null @@ -1,138 +0,0 @@ -package aecor.example.domain - -import java.time.Clock - -import aecor.aggregate.{ Correlation, CorrelationIdF, Folder } -import aecor.data.Folded.syntax._ -import aecor.data.{ EventTag, Folded, Handler } -import aecor.example.domain.AccountAggregateEvent._ -import aecor.example.domain.AccountAggregateOp._ -import akka.Done -import cats.arrow.FunctionK -import cats.~> - -import scala.collection.immutable.Seq - -case class AccountId(value: String) extends AnyVal - -object AccountAggregate { - - def correlation: Correlation[AccountAggregateOp] = { - def mk[A](fa: AccountAggregateOp[A]): CorrelationIdF[A] = - fa.accountId.value - FunctionK.lift(mk _) - } - - def applyEvent(state: Option[Account])(event: AccountAggregateEvent): Folded[Option[Account]] = - state match { - case None => - event match { - case AccountOpened(accountId) => - Some(Account(accountId, Amount(0), Map.empty, Set.empty)).next - case other => - impossible - } - case Some(Account(id, balance, holds, transactions)) => - event match { - case e: AccountOpened => impossible - case e: TransactionAuthorized => - state - .map( - _.copy( - holds = holds + (e.transactionId -> e.amount), - balance = balance - e.amount, - transactions = transactions + e.transactionId - ) - ) - .next - case e: TransactionVoided => - holds - .get(e.transactionId) - .map( - holdAmount => - state - .map(_.copy(holds = holds - e.transactionId, balance = balance + holdAmount)) - ) - .getOrElse(state) - .next - case e: AccountCredited => - state.map(_.copy(balance = balance + e.amount)).next - case e: TransactionCaptured => - state.map(_.copy(holds = holds - e.transactionId)).next - } - } - - case class Account(id: AccountId, - balance: Amount, - holds: Map[TransactionId, Amount], - transactions: Set[TransactionId]) - - object Account { - implicit val folder: Folder[Folded, AccountAggregateEvent, Option[Account]] = - Folder.instance(Option.empty[Account])(applyEvent) - } - - val entityName: String = "Account" - - val entityNameTag: EventTag[AccountAggregateEvent] = EventTag(entityName) - - def commandHandler(clock: Clock) = - new (AccountAggregateOp ~> Handler[Option[AccountAggregate.Account], AccountAggregateEvent, ?]) { - def accept[R, E](events: E*): (Seq[E], Either[R, Done]) = - (events.toVector, Right(Done)) - - def reject[R, E](rejection: R): (Seq[E], Either[R, Done]) = - (Seq.empty, Left(rejection)) - - override def apply[A](fa: AccountAggregateOp[A]) = - fa match { - case OpenAccount(accountId) => - Handler { - case None => - accept(AccountOpened(accountId)) - case _ => - reject(AccountExists) - } - - case AuthorizeTransaction(_, transactionId, amount) => - Handler { - case None => - reject(AccountDoesNotExist) - case Some(Account(id, balance, holds, transactions)) => - if (transactions.contains(transactionId)) { - reject(DuplicateTransaction) - } else if (balance > amount) { - accept(TransactionAuthorized(id, transactionId, amount)) - } else { - reject(InsufficientFunds) - } - } - case VoidTransaction(_, transactionId) => - Handler { - case None => - reject(AccountDoesNotExist) - case Some(Account(id, balance, holds, transactions)) => - accept(TransactionVoided(id, transactionId)) - } - case CaptureTransaction(_, transactionId, clearingAmount) => - Handler { - case None => - reject(AccountDoesNotExist) - case Some(Account(id, balance, holds, transactions)) => - holds.get(transactionId) match { - case Some(amount) => - accept(TransactionCaptured(id, transactionId, clearingAmount)) - case None => - reject(HoldNotFound) - } - } - case CreditAccount(_, transactionId, amount) => - Handler { - case None => - reject(AccountDoesNotExist) - case Some(Account(id, balance, holds, transactions)) => - accept(AccountCredited(id, transactionId, amount)) - } - } - } -} diff --git a/example/src/main/scala/aecor/example/domain/AccountAggregateEvent.scala b/example/src/main/scala/aecor/example/domain/AccountAggregateEvent.scala deleted file mode 100644 index 30d28693..00000000 --- a/example/src/main/scala/aecor/example/domain/AccountAggregateEvent.scala +++ /dev/null @@ -1,34 +0,0 @@ -package aecor.example.domain - -import aecor.example.persistentEncoderUtil -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentEncoder } -import io.circe.generic.auto._ - -sealed abstract class AccountAggregateEvent extends Product with Serializable { - def accountId: AccountId -} - -object AccountAggregateEvent { - case class AccountOpened(accountId: AccountId) extends AccountAggregateEvent - - case class TransactionAuthorized(accountId: AccountId, - transactionId: TransactionId, - amount: Amount) - extends AccountAggregateEvent - - case class TransactionVoided(accountId: AccountId, transactionId: TransactionId) - extends AccountAggregateEvent - - case class TransactionCaptured(accountId: AccountId, - transactionId: TransactionId, - amount: Amount) - extends AccountAggregateEvent - - case class AccountCredited(accountId: AccountId, transactionId: TransactionId, amount: Amount) - extends AccountAggregateEvent - - implicit val encoder: PersistentEncoder[AccountAggregateEvent] = - persistentEncoderUtil.circePersistentEncoder - implicit val decoder: PersistentDecoder[AccountAggregateEvent] = - persistentEncoderUtil.circePersistentDecoder -} diff --git a/example/src/main/scala/aecor/example/domain/AccountAggregateOp.scala b/example/src/main/scala/aecor/example/domain/AccountAggregateOp.scala deleted file mode 100644 index e4c28d43..00000000 --- a/example/src/main/scala/aecor/example/domain/AccountAggregateOp.scala +++ /dev/null @@ -1,49 +0,0 @@ -package aecor.example.domain - -import akka.Done - -object AccountAggregateOp { - case class OpenAccount(accountId: AccountId) - extends AccountAggregateOp[Either[Rejection, Done]] - - case class AuthorizeTransaction(accountId: AccountId, - transactionId: TransactionId, - amount: Amount) - extends AccountAggregateOp[Either[AuthorizeTransactionRejection, Done]] - - case class VoidTransaction(accountId: AccountId, - transactionId: TransactionId) - extends AccountAggregateOp[Rejection Either Done] - - case class CaptureTransaction(accountId: AccountId, - transactionId: TransactionId, - amount: Amount) - extends AccountAggregateOp[Rejection Either Done] - - case class CreditAccount(accountId: AccountId, - transactionId: TransactionId, - amount: Amount) - extends AccountAggregateOp[Rejection Either Done] - - sealed trait Rejection - - sealed trait AuthorizeTransactionRejection - - case object DuplicateTransaction extends AuthorizeTransactionRejection - - case object AccountDoesNotExist - extends Rejection - with AuthorizeTransactionRejection - - case object InsufficientFunds - extends Rejection - with AuthorizeTransactionRejection - - case object AccountExists extends Rejection - - case object HoldNotFound extends Rejection -} - -sealed trait AccountAggregateOp[R] { - def accountId: AccountId -} diff --git a/example/src/main/scala/aecor/example/domain/Amount.scala b/example/src/main/scala/aecor/example/domain/Amount.scala index de206949..5b8d9459 100644 --- a/example/src/main/scala/aecor/example/domain/Amount.scala +++ b/example/src/main/scala/aecor/example/domain/Amount.scala @@ -1,8 +1,13 @@ package aecor.example.domain -case class Amount(value: Long) extends AnyVal { +case class Amount(value: BigDecimal) extends AnyVal { def >(other: Amount): Boolean = value > other.value def <=(other: Amount): Boolean = value <= other.value + def >=(other: Amount): Boolean = value >= other.value def -(other: Amount): Amount = Amount(value - other.value) def +(other: Amount): Amount = Amount(value + other.value) } + +object Amount { + val zero: Amount = Amount(0) +} diff --git a/example/src/main/scala/aecor/example/domain/AuthorizationProcess.scala b/example/src/main/scala/aecor/example/domain/AuthorizationProcess.scala deleted file mode 100644 index 8cd231d4..00000000 --- a/example/src/main/scala/aecor/example/domain/AuthorizationProcess.scala +++ /dev/null @@ -1,72 +0,0 @@ -package aecor.example.domain - -import aecor.example.domain.AccountAggregateOp._ -import aecor.example.domain.CardAuthorizationAggregateEvent.CardAuthorizationCreated -import aecor.example.domain.CardAuthorizationAggregateOp._ -import akka.stream.scaladsl.Flow -import akka.{ Done, NotUsed } -import cats.free.Free -import freek._ - -import scala.concurrent.{ ExecutionContext, Future } - -object AuthorizationProcess { - type PRG = - AccountAggregateOp :|: CardAuthorizationAggregateOp :|: NilDSL - - val PRG = DSL.Make[PRG] - - def pure[A](a: A): Free[PRG.Cop, A] = Free.pure(a) - - def handleEvent: CardAuthorizationCreated => Free[PRG.Cop, Done] = { - case CardAuthorizationCreated(cardAuthorizationId, accountId, amount, _, _, transactionId) => - AuthorizeTransaction(accountId, transactionId, amount).freek[PRG].flatMap { - case Right(_) => - AcceptCardAuthorization(cardAuthorizationId).freek[PRG].flatMap { - case Left(rejection) => - rejection match { - case AlreadyDeclined => - VoidTransaction(accountId, transactionId).freek[PRG].map(_ => Done) - case DoesNotExists => - VoidTransaction(accountId, transactionId).freek[PRG].map(_ => Done) - case AlreadyAccepted => pure(Done) - } - case _ => - pure(Done) - } - case Left(rejection) => - rejection match { - case AccountAggregateOp.AccountDoesNotExist => - DeclineCardAuthorization(cardAuthorizationId, "Unknown account") - .freek[PRG] - .map(_ => Done) - case AccountAggregateOp.InsufficientFunds => - DeclineCardAuthorization(cardAuthorizationId, "InsufficientFunds") - .freek[PRG] - .map(_ => Done) - case AccountAggregateOp.DuplicateTransaction => - AcceptCardAuthorization(cardAuthorizationId).freek[PRG].flatMap { - case Left(r) => - r match { - case AlreadyDeclined => - VoidTransaction(accountId, transactionId).freek[PRG].map(_ => Done) - case DoesNotExists => - VoidTransaction(accountId, transactionId).freek[PRG].map(_ => Done) - case AlreadyAccepted => pure(Done) - } - case _ => pure(Done) - } - } - } - } - - def flow[PassThrough, F2[_] <: CopK[_]](parallelism: Int, interpreter: Interpreter[F2, Future])( - implicit sub: SubCop[PRG.Cop, F2], - ec: ExecutionContext - ): Flow[(CardAuthorizationCreated, PassThrough), PassThrough, NotUsed] = - Flow[(CardAuthorizationCreated, PassThrough)].mapAsync(parallelism) { - case (e, ps) => - import cats.instances.future._ - handleEvent(e).interpret(interpreter).map(_ => ps) - } -} diff --git a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregate.scala b/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregate.scala deleted file mode 100644 index 9f8bfc5d..00000000 --- a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregate.scala +++ /dev/null @@ -1,138 +0,0 @@ -package aecor.example.domain -import java.util.UUID - -import aecor.aggregate.{ Correlation, CorrelationIdF, Folder } -import aecor.data.Folded.syntax._ -import aecor.data.{ EventTag, Folded, Handler } -import aecor.example.domain.CardAuthorizationAggregate.State.{ - Accepted, - Created, - Declined, - Initial -} -import aecor.example.domain.CardAuthorizationAggregateEvent.{ - CardAuthorizationAccepted, - CardAuthorizationCreated, - CardAuthorizationDeclined -} -import aecor.example.domain.CardAuthorizationAggregateOp._ -import akka.Done -import cats.arrow.FunctionK -import cats.~> - -import scala.collection.immutable.Seq - -case class TransactionId(value: String) extends AnyVal - -object CardAuthorizationAggregate { - sealed trait State { - def applyEvent(event: CardAuthorizationAggregateEvent): Folded[State] = - this match { - case Initial => - event match { - case e: CardAuthorizationCreated => - Created(e.cardAuthorizationId).next - case _ => - impossible - } - case self: Created => - event match { - case e: CardAuthorizationCreated => impossible - case e: CardAuthorizationAccepted => Accepted(self.id).next - case e: CardAuthorizationDeclined => Declined(self.id).next - } - case self: Accepted => - event match { - case e: CardAuthorizationCreated => impossible - case e: CardAuthorizationAccepted => Accepted(self.id).next - case e: CardAuthorizationDeclined => Declined(self.id).next - } - case self: Declined => - event match { - case e: CardAuthorizationCreated => impossible - case e: CardAuthorizationAccepted => Accepted(self.id).next - case e: CardAuthorizationDeclined => Declined(self.id).next - } - } - } - object State { - case object Initial extends State - case class Created(id: CardAuthorizationId) extends State - case class Accepted(id: CardAuthorizationId) extends State - case class Declined(id: CardAuthorizationId) extends State - implicit val folder: Folder[Folded, CardAuthorizationAggregateEvent, State] = - Folder.instance(Initial: State)(_.applyEvent) - - } - - def correlation: Correlation[CardAuthorizationAggregateOp] = { - def mk[A](fa: CardAuthorizationAggregateOp[A]): CorrelationIdF[A] = - fa.cardAuthorizationId.value - FunctionK.lift(mk _) - } - - val entityName: String = "CardAuthorization" - - val entityNameTag: EventTag[CardAuthorizationAggregateEvent] = EventTag(entityName) - - def commandHandler = - new (CardAuthorizationAggregateOp ~> Handler[State, CardAuthorizationAggregateEvent, ?]) { - def accept[R, E](events: E*): (Seq[E], Either[R, Done]) = - (events.toVector, Right(Done)) - - def reject[R, E](rejection: R): (Seq[E], Either[R, Done]) = - (Seq.empty, Left(rejection)) - override def apply[A](command: CardAuthorizationAggregateOp[A]) = - Handler { - case Initial => - command match { - case CreateCardAuthorization( - cardAuthorizationId, - accountId, - amount, - acquireId, - terminalId - ) => - accept( - CardAuthorizationCreated( - cardAuthorizationId, - accountId, - amount, - acquireId, - terminalId, - TransactionId(UUID.randomUUID().toString) - ) - ) - case c: AcceptCardAuthorization => - reject(DoesNotExists) - case c: DeclineCardAuthorization => - reject(DoesNotExists) - } - case Created(id) => - command match { - case e: AcceptCardAuthorization => - accept(CardAuthorizationAccepted(id)) - case e: DeclineCardAuthorization => - accept(CardAuthorizationDeclined(id, e.reason)) - case e: CreateCardAuthorization => - reject(AlreadyExists) - } - case Accepted(id) => - command match { - case e: AcceptCardAuthorization => reject(AlreadyAccepted) - case e: DeclineCardAuthorization => reject(AlreadyAccepted) - case e: CreateCardAuthorization => reject(AlreadyExists) - } - case Declined(id) => - command match { - case e: AcceptCardAuthorization => reject(AlreadyDeclined) - case e: DeclineCardAuthorization => reject(AlreadyDeclined) - case e: CreateCardAuthorization => reject(AlreadyExists) - } - } - } -} -case class CardAuthorizationId(value: String) extends AnyVal -case class CardNumber(value: String) extends AnyVal -case class AcquireId(value: Long) extends AnyVal -case class TerminalId(value: Long) extends AnyVal diff --git a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateEvent.scala b/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateEvent.scala deleted file mode 100644 index 4ca89d44..00000000 --- a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateEvent.scala +++ /dev/null @@ -1,27 +0,0 @@ -package aecor.example.domain - -import aecor.example.persistentEncoderUtil -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentEncoder } -import io.circe.generic.auto._ -sealed trait CardAuthorizationAggregateEvent { - def cardAuthorizationId: CardAuthorizationId -} - -object CardAuthorizationAggregateEvent { - case class CardAuthorizationCreated(cardAuthorizationId: CardAuthorizationId, - accountId: AccountId, - amount: Amount, - acquireId: AcquireId, - terminalId: TerminalId, - transactionId: TransactionId) - extends CardAuthorizationAggregateEvent - case class CardAuthorizationDeclined(cardAuthorizationId: CardAuthorizationId, reason: String) - extends CardAuthorizationAggregateEvent - case class CardAuthorizationAccepted(cardAuthorizationId: CardAuthorizationId) - extends CardAuthorizationAggregateEvent - - implicit val encoder: PersistentEncoder[CardAuthorizationAggregateEvent] = - persistentEncoderUtil.circePersistentEncoder - implicit val decoder: PersistentDecoder[CardAuthorizationAggregateEvent] = - persistentEncoderUtil.circePersistentDecoder -} diff --git a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateOp.scala b/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateOp.scala deleted file mode 100644 index af09ea04..00000000 --- a/example/src/main/scala/aecor/example/domain/CardAuthorizationAggregateOp.scala +++ /dev/null @@ -1,36 +0,0 @@ -package aecor.example.domain - -import akka.Done - -sealed trait CardAuthorizationAggregateOp[Response] { - def cardAuthorizationId: CardAuthorizationId -} - -object CardAuthorizationAggregateOp { - type CommandResult[+Rejection] = Either[Rejection, Done] - - case class CreateCardAuthorization(cardAuthorizationId: CardAuthorizationId, - accountId: AccountId, - amount: Amount, - acquireId: AcquireId, - terminalId: TerminalId) - extends CardAuthorizationAggregateOp[CommandResult[CreateCardAuthorizationRejection]] - case class DeclineCardAuthorization(cardAuthorizationId: CardAuthorizationId, reason: String) - extends CardAuthorizationAggregateOp[CommandResult[DeclineCardAuthorizationRejection]] - case class AcceptCardAuthorization(cardAuthorizationId: CardAuthorizationId) - extends CardAuthorizationAggregateOp[CommandResult[AcceptCardAuthorizationRejection]] - - sealed trait CreateCardAuthorizationRejection - sealed trait DeclineCardAuthorizationRejection - sealed trait AcceptCardAuthorizationRejection - case object DoesNotExists - extends DeclineCardAuthorizationRejection - with AcceptCardAuthorizationRejection - case object AlreadyExists extends CreateCardAuthorizationRejection - case object AlreadyDeclined - extends DeclineCardAuthorizationRejection - with AcceptCardAuthorizationRejection - case object AlreadyAccepted - extends DeclineCardAuthorizationRejection - with AcceptCardAuthorizationRejection -} diff --git a/example/src/main/scala/aecor/example/domain/TransactionProcess.scala b/example/src/main/scala/aecor/example/domain/TransactionProcess.scala new file mode 100644 index 00000000..78cf357b --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/TransactionProcess.scala @@ -0,0 +1,74 @@ +package aecor.example.domain + +import aecor.example.domain.account.{ + AccountAggregate, + AccountTransactionId, + AccountTransactionKind +} +import aecor.example.domain.transaction.TransactionAggregate.TransactionInfo +import aecor.example.domain.transaction.{ TransactionAggregate, TransactionEvent } +import cats.implicits._ +import cats.{ Monad, MonadError } +import io.aecor.liberator.macros.{ algebra, term } + +object TransactionProcess { + type Input = TransactionEvent + + @algebra + @term + trait TransactionProcessFailure[F[_]] { + def failProcess[A](reason: String): F[A] + } + + object TransactionProcessFailure { + def withMonadError[F[_]](implicit F: MonadError[F, Throwable]): TransactionProcessFailure[F] = + new TransactionProcessFailure[F] { + override def failProcess[A](reason: String): F[A] = + F.raiseError(new RuntimeException(reason)) + } + } + + def apply[F[_]: Monad](transaction: TransactionAggregate[F], + account: AccountAggregate[F], + failure: TransactionProcessFailure[F]): Input => F[Unit] = { + case TransactionEvent.TransactionCreated(transactionId, from, to, amount, _) => + for { + out <- account.debitAccount( + from.value, + AccountTransactionId(transactionId, AccountTransactionKind.Normal), + amount + ) + _ <- out match { + case Left(rejection) => + transaction.failTransaction(transactionId, rejection.toString) + case Right(_) => + transaction.authorizeTransaction(transactionId) + } + } yield () + case TransactionEvent.TransactionAuthorized(transactionId, _) => + for { + txn <- transaction.getTransactionInfo(transactionId).flatMap { + case Some(x) => x.pure[F] + case None => + failure.failProcess[TransactionInfo](s"Transaction [$transactionId] not found") + } + creditResult <- account.creditAccount( + txn.toAccountId.value, + AccountTransactionId(transactionId, AccountTransactionKind.Normal), + txn.amount + ) + _ <- creditResult match { + case Left(rejection) => + account.debitAccount( + txn.fromAccountId.value, + AccountTransactionId(txn.transactionId, AccountTransactionKind.Revert), + txn.amount + ) >> transaction.failTransaction(transactionId, rejection.toString) + case Right(_) => + transaction.succeedTransaction(transactionId) + } + } yield () + case other => + ().pure[F] + } +} diff --git a/example/src/main/scala/aecor/example/domain/account/AccountAggregate.scala b/example/src/main/scala/aecor/example/domain/account/AccountAggregate.scala new file mode 100644 index 00000000..8c78286e --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/account/AccountAggregate.scala @@ -0,0 +1,52 @@ +package aecor.example.domain.account + +import aecor.example.AnyValCirceEncoding +import aecor.example.domain.Amount +import aecor.example.domain.account.AccountAggregate.Rejection +import aecor.example.domain.transaction.TransactionId +import io.aecor.liberator.macros.algebra +import io.circe.{ Decoder, Encoder } + +@algebra('accountId) +trait AccountAggregate[F[_]] { + def openAccount(accountId: AccountId): F[Either[Rejection, Unit]] + def creditAccount(accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount): F[Either[Rejection, Unit]] + def debitAccount(accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount): F[Either[Rejection, Unit]] +} + +final case class AccountTransactionId(baseTransactionId: TransactionId, + kind: AccountTransactionKind) +object AccountTransactionId extends AnyValCirceEncoding { + implicit def decoder: Decoder[AccountTransactionId] = io.circe.generic.semiauto.deriveDecoder + implicit def encoder: Encoder[AccountTransactionId] = io.circe.generic.semiauto.deriveEncoder +} +sealed abstract class AccountTransactionKind +object AccountTransactionKind { + case object Normal extends AccountTransactionKind + case object Revert extends AccountTransactionKind + implicit val decoder: Decoder[AccountTransactionKind] = Decoder[String].emap { + case "Normal" => Right(Normal) + case "Revert" => Right(Revert) + case _ => Left("Unknown AccountTransactionKind") + } + implicit val encoder: Encoder[AccountTransactionKind] = Encoder[String].contramap(_.toString) +} +object AccountAggregate { + sealed trait Rejection extends Product with Serializable + + sealed trait AuthorizeTransactionRejection + + case object DuplicateTransaction extends AuthorizeTransactionRejection + + case object AccountDoesNotExist extends Rejection with AuthorizeTransactionRejection + + case object InsufficientFunds extends Rejection with AuthorizeTransactionRejection + + case object AccountExists extends Rejection + + case object HoldNotFound extends Rejection +} diff --git a/example/src/main/scala/aecor/example/domain/account/AccountEvent.scala b/example/src/main/scala/aecor/example/domain/account/AccountEvent.scala new file mode 100644 index 00000000..04bcc3c9 --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/account/AccountEvent.scala @@ -0,0 +1,34 @@ +package aecor.example.domain.account + +import java.time.Instant + +import aecor.example.domain.Amount +import aecor.example.persistentEncoderUtil +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentEncoder } +import io.circe.java8.time._ +import io.circe.generic.auto._ + +sealed abstract class AccountEvent extends Product with Serializable { + def accountId: AccountId +} + +object AccountEvent { + case class AccountOpened(accountId: AccountId, timestamp: Instant) extends AccountEvent + + case class AccountDebited(accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount, + timestamp: Instant) + extends AccountEvent + + case class AccountCredited(accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount, + timestamp: Instant) + extends AccountEvent + + implicit val encoder: PersistentEncoder[AccountEvent] = + persistentEncoderUtil.circePersistentEncoder(io.circe.generic.semiauto.deriveEncoder) + implicit val decoder: PersistentDecoder[AccountEvent] = + persistentEncoderUtil.circePersistentDecoder(io.circe.generic.semiauto.deriveDecoder) +} diff --git a/example/src/main/scala/aecor/example/domain/account/AccountId.scala b/example/src/main/scala/aecor/example/domain/account/AccountId.scala new file mode 100644 index 00000000..23b6c2b1 --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/account/AccountId.scala @@ -0,0 +1,3 @@ +package aecor.example.domain.account + +final case class AccountId(value: String) extends AnyVal diff --git a/example/src/main/scala/aecor/example/domain/account/EventsourcedAccountAggregate.scala b/example/src/main/scala/aecor/example/domain/account/EventsourcedAccountAggregate.scala new file mode 100644 index 00000000..e23cf3cf --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/account/EventsourcedAccountAggregate.scala @@ -0,0 +1,107 @@ +package aecor.example.domain.account + +import aecor.data.Folded.syntax._ +import aecor.data.{ EventsourcedBehavior, Folded, Folder, Handler } +import aecor.example.domain.Amount +import aecor.example.domain.account.AccountAggregate.{ AccountDoesNotExist, InsufficientFunds } +import aecor.example.domain.account.AccountEvent._ +import aecor.example.domain.account.EventsourcedAccountAggregate.Account +import aecor.util.Clock +import cats.Applicative +import cats.implicits._ + +import scala.collection.immutable._ + +class EventsourcedAccountAggregate[F[_]](clock: Clock[F])(implicit F: Applicative[F]) + extends AccountAggregate[Handler[F, Option[Account], AccountEvent, ?]] { + + import F._ + + override def openAccount( + accountId: AccountId + ): Handler[F, Option[Account], AccountEvent, Either[AccountAggregate.Rejection, Unit]] = + Handler { + case None => + clock.instant.map { now => + Seq(AccountOpened(accountId, now)) -> ().asRight + } + case Some(x) => pure(Seq.empty -> AccountAggregate.AccountExists.asLeft) + } + + override def creditAccount( + accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount + ): Handler[F, Option[Account], AccountEvent, Either[AccountAggregate.Rejection, Unit]] = + Handler { + case Some(account) => + clock.instant.map { now => + if (account.processedTransactions.contains(transactionId)) { + Seq.empty -> ().asRight + } else { + Seq(AccountCredited(accountId, transactionId, amount, now)) -> ().asRight + } + } + case None => + pure(Seq.empty -> AccountAggregate.AccountDoesNotExist.asLeft) + } + + override def debitAccount( + accountId: AccountId, + transactionId: AccountTransactionId, + amount: Amount + ): Handler[F, Option[Account], AccountEvent, Either[AccountAggregate.Rejection, Unit]] = + Handler { + case Some(account) => + clock.instant.map { now => + if (account.processedTransactions.contains(transactionId)) { + Seq.empty -> ().asRight + } else { + if (account.hasFunds(amount) || accountId == EventsourcedAccountAggregate.rootAccountId) { + Seq(AccountDebited(accountId, transactionId, amount, now)) -> ().asRight + } else { + Seq.empty -> InsufficientFunds.asLeft + } + } + } + case None => + pure(Seq.empty -> AccountDoesNotExist.asLeft) + } +} + +object EventsourcedAccountAggregate { + + def behavior[F[_]: Applicative]( + clock: Clock[F] + ): EventsourcedBehavior[F, AccountAggregate.AccountAggregateOp, Option[Account], AccountEvent] = + EventsourcedBehavior( + AccountAggregate.toFunctionK(new EventsourcedAccountAggregate[F](clock)), + Account.folder + ) + final val rootAccountId: AccountId = AccountId("ROOT") + final case class Account(balance: Amount, processedTransactions: Set[AccountTransactionId]) { + def hasFunds(amount: Amount): Boolean = + balance >= amount + def applyEvent(event: AccountEvent): Folded[Account] = event match { + case AccountOpened(_, _) => impossible + case AccountDebited(_, transactionId, amount, _) => + copy( + balance = balance - amount, + processedTransactions = processedTransactions + transactionId + ).next + case AccountCredited(_, transactionId, amount, _) => + copy( + balance = balance + amount, + processedTransactions = processedTransactions + transactionId + ).next + } + } + object Account { + def fromEvent(event: AccountEvent): Folded[Account] = event match { + case AccountOpened(_, _) => Account(Amount.zero, Set.empty).next + case _ => impossible + } + def folder: Folder[Folded, AccountEvent, Option[Account]] = + Folder.optionInstance(fromEvent)(x => x.applyEvent) + } +} diff --git a/example/src/main/scala/aecor/example/domain/transaction/EventsourcedTransactionAggregate.scala b/example/src/main/scala/aecor/example/domain/transaction/EventsourcedTransactionAggregate.scala new file mode 100644 index 00000000..6405d4cf --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/transaction/EventsourcedTransactionAggregate.scala @@ -0,0 +1,153 @@ +package aecor.example.domain.transaction + +import aecor.data.Folded.syntax._ +import aecor.data._ +import aecor.example.domain.Amount +import aecor.example.domain.account.AccountId +import aecor.example.domain.transaction.EventsourcedTransactionAggregate.Transaction +import aecor.example.domain.transaction.EventsourcedTransactionAggregate.TransactionStatus.{ + Authorized, + Failed, + Requested, + Succeeded +} +import aecor.example.domain.transaction.TransactionAggregate._ +import aecor.example.domain.transaction.TransactionEvent.{ + TransactionAuthorized, + TransactionCreated, + TransactionFailed, + TransactionSucceeded +} +import aecor.util.Clock +import cats.Applicative +import cats.implicits._ + +import scala.collection.immutable._ + +class EventsourcedTransactionAggregate[F[_]](clock: Clock[F])(implicit F: Applicative[F]) + extends TransactionAggregate[Handler[F, Option[Transaction], TransactionEvent, ?]] { + import F._ + + override def createTransaction( + transactionId: TransactionId, + fromAccountId: From[AccountId], + toAccountId: To[AccountId], + amount: Amount + ): Handler[F, Option[Transaction], TransactionEvent, Unit] = + Handler { + case None => + clock.instant.map { now => + Seq( + TransactionEvent + .TransactionCreated(transactionId, fromAccountId, toAccountId, amount, now) + ) -> (()) + } + + case Some(_) => pure(Seq.empty -> (())) + } + + override def authorizeTransaction( + transactionId: TransactionId + ): Handler[F, Option[Transaction], TransactionEvent, Either[String, Unit]] = + Handler { + case Some(transaction) => + clock.instant.map { now => + if (transaction.status == Requested) { + Seq(TransactionAuthorized(transactionId, now)) -> ().asRight + } else if (transaction.status == Authorized) { + Seq() -> ().asRight + } else { + Seq() -> "Illegal transition".asLeft + } + } + case None => + pure(Seq() -> "Transaction not found".asLeft) + } + + override def failTransaction( + transactionId: TransactionId, + reason: String + ): Handler[F, Option[Transaction], TransactionEvent, Either[String, Unit]] = + Handler { + case Some(transaction) => + clock.instant.map { now => + if (transaction.status == Failed) { + Seq.empty -> ().asRight + } else { + Seq(TransactionFailed(transactionId, reason, now)) -> ().asRight + } + } + case None => + pure(Seq.empty -> "Transaction not found".asLeft) + } + + override def succeedTransaction( + transactionId: TransactionId + ): Handler[F, Option[Transaction], TransactionEvent, Either[String, Unit]] = + Handler { + case Some(transaction) => + clock.instant.map { now => + if (transaction.status == Succeeded) { + Seq.empty -> ().asRight + } else if (transaction.status == Authorized) { + Seq(TransactionSucceeded(transactionId, now)) -> ().asRight + } else { + Seq.empty -> "Illegal transition".asLeft + } + } + case None => + pure(Seq.empty -> "Transaction not found".asLeft) + } + + override def getTransactionInfo( + transactionId: TransactionId + ): Handler[F, Option[Transaction], TransactionEvent, Option[TransactionInfo]] = + Handler.readOnly(_.map { + case Transaction(status, from, to, amount) => + TransactionInfo(transactionId, from, to, amount, Some(status).collect { + case Succeeded => true + case Failed => false + }) + }) +} + +object EventsourcedTransactionAggregate { + def behavior[F[_]: Applicative]( + clock: Clock[F] + ): EventsourcedBehavior[F, TransactionAggregateOp, Option[Transaction], TransactionEvent] = + EventsourcedBehavior( + TransactionAggregate.toFunctionK(new EventsourcedTransactionAggregate[F](clock)), + Transaction.folder + ) + + def tagging: Tagging.Partitioned[TransactionEvent] = + Tagging.partitioned[TransactionEvent](20, EventTag("Transaction"))(_.transactionId.value) + sealed abstract class TransactionStatus + object TransactionStatus { + case object Requested extends TransactionStatus + case object Authorized extends TransactionStatus + case object Failed extends TransactionStatus + case object Succeeded extends TransactionStatus + } + final case class Transaction(status: TransactionStatus, + from: From[AccountId], + to: To[AccountId], + amount: Amount) { + def applyEvent(event: TransactionEvent): Folded[Transaction] = event match { + case TransactionCreated(_, _, _, _, _) => impossible + case TransactionAuthorized(_, _) => copy(status = TransactionStatus.Authorized).next + case TransactionFailed(_, _, _) => copy(status = TransactionStatus.Failed).next + case TransactionSucceeded(_, _) => copy(status = TransactionStatus.Succeeded).next + } + } + object Transaction { + def fromEvent(event: TransactionEvent): Folded[Transaction] = event match { + case TransactionEvent.TransactionCreated(transactionId, fromAccount, toAccount, amount, _) => + Transaction(TransactionStatus.Requested, fromAccount, toAccount, amount).next + case _ => impossible + } + def folder: Folder[Folded, TransactionEvent, Option[Transaction]] = + Folder.optionInstance(fromEvent)(x => x.applyEvent) + } + +} diff --git a/example/src/main/scala/aecor/example/domain/transaction/TransactionAggregate.scala b/example/src/main/scala/aecor/example/domain/transaction/TransactionAggregate.scala new file mode 100644 index 00000000..49034352 --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/transaction/TransactionAggregate.scala @@ -0,0 +1,30 @@ +package aecor.example.domain.transaction + +import aecor.example.domain.Amount +import aecor.example.domain.account.AccountId +import aecor.example.domain.transaction.TransactionAggregate.TransactionInfo +import io.aecor.liberator.macros.{ algebra } + +final case class TransactionId(value: String) extends AnyVal +final case class From[A](value: A) extends AnyVal +final case class To[A](value: A) extends AnyVal + +@algebra('transactionId) +trait TransactionAggregate[F[_]] { + def createTransaction(transactionId: TransactionId, + fromAccountId: From[AccountId], + toAccountId: To[AccountId], + amount: Amount): F[Unit] + def authorizeTransaction(transactionId: TransactionId): F[Either[String, Unit]] + def failTransaction(transactionId: TransactionId, reason: String): F[Either[String, Unit]] + def succeedTransaction(transactionId: TransactionId): F[Either[String, Unit]] + def getTransactionInfo(transactionId: TransactionId): F[Option[TransactionInfo]] +} + +object TransactionAggregate { + final case class TransactionInfo(transactionId: TransactionId, + fromAccountId: From[AccountId], + toAccountId: To[AccountId], + amount: Amount, + succeeded: Option[Boolean]) +} diff --git a/example/src/main/scala/aecor/example/domain/transaction/TransactionEvent.scala b/example/src/main/scala/aecor/example/domain/transaction/TransactionEvent.scala new file mode 100644 index 00000000..55d28b12 --- /dev/null +++ b/example/src/main/scala/aecor/example/domain/transaction/TransactionEvent.scala @@ -0,0 +1,36 @@ +package aecor.example.domain.transaction + +import java.time.Instant + +import aecor.example.domain.Amount +import aecor.example.domain.account.AccountId +import aecor.example.persistentEncoderUtil +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentEncoder } +import io.circe.java8.time._ +import io.circe.generic.auto._ + +sealed abstract class TransactionEvent extends Product with Serializable { + def transactionId: TransactionId +} + +object TransactionEvent { + final case class TransactionCreated(transactionId: TransactionId, + fromAccount: From[AccountId], + toAccount: To[AccountId], + amount: Amount, + timestamp: Instant) + extends TransactionEvent + + final case class TransactionAuthorized(transactionId: TransactionId, timestamp: Instant) + extends TransactionEvent + + case class TransactionFailed(transactionId: TransactionId, reason: String, timestamp: Instant) + extends TransactionEvent + case class TransactionSucceeded(transactionId: TransactionId, timestamp: Instant) + extends TransactionEvent + + implicit val encoder: PersistentEncoder[TransactionEvent] = + persistentEncoderUtil.circePersistentEncoder[TransactionEvent] + implicit val decoder: PersistentDecoder[TransactionEvent] = + persistentEncoderUtil.circePersistentDecoder[TransactionEvent] +} diff --git a/example/src/main/scala/aecor/example/persistentEncoderUtil.scala b/example/src/main/scala/aecor/example/persistentEncoderUtil.scala index ab3833ec..37d9531a 100644 --- a/example/src/main/scala/aecor/example/persistentEncoderUtil.scala +++ b/example/src/main/scala/aecor/example/persistentEncoderUtil.scala @@ -3,13 +3,14 @@ package aecor.example import java.nio.ByteBuffer import java.nio.charset.StandardCharsets -import aecor.aggregate.serialization.{ +import aecor.runtime.akkapersistence.serialization.{ DecodingFailure, PersistentDecoder, PersistentEncoder, PersistentRepr } import io.circe.{ Decoder, Encoder, jawn } + object persistentEncoderUtil { def circePersistentEncoder[A](implicit encoder: Encoder[A]): PersistentEncoder[A] = PersistentEncoder.instance( diff --git a/project/build.properties b/project/build.properties index 40868d71..3e8bd1a5 100644 --- a/project/build.properties +++ b/project/build.properties @@ -1 +1 @@ -sbt.version = 0.13.13-RC2 +sbt.version = 0.13.15 diff --git a/project/plugins.sbt b/project/plugins.sbt index a91f19e6..eed8461b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,13 +1,12 @@ logLevel := Level.Warn -addSbtPlugin("com.thesamet" % "sbt-protoc" % "0.99.1") +addSbtPlugin("com.thesamet" % "sbt-protoc" % "0.99.6") -addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.2.0-M7") -addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.2") +addSbtPlugin("com.typesafe.sbt" % "sbt-native-packager" % "1.2.0-M9") addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.3") addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.8.5") addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "1.1") -libraryDependencies += "com.trueaccord.scalapb" %% "compilerplugin" % "0.5.45-p3" +libraryDependencies += "com.trueaccord.scalapb" %% "compilerplugin" % "0.6.1" diff --git a/schedule/src/main/scala/aecor/schedule/CassandraScheduleEntryRepository.scala b/schedule/src/main/scala/aecor/schedule/CassandraScheduleEntryRepository.scala index 14832643..f7660581 100644 --- a/schedule/src/main/scala/aecor/schedule/CassandraScheduleEntryRepository.scala +++ b/schedule/src/main/scala/aecor/schedule/CassandraScheduleEntryRepository.scala @@ -5,20 +5,26 @@ import java.time.format.DateTimeFormatter import aecor.schedule.CassandraScheduleEntryRepository.{ Queries, TimeBucket } import aecor.schedule.ScheduleEntryRepository.ScheduleEntry +import aecor.effect.{ Async, CaptureFuture } import akka.NotUsed import akka.persistence.cassandra._ import akka.persistence.cassandra.session.scaladsl.CassandraSession -import akka.stream.scaladsl.Source +import akka.stream.Materializer +import akka.stream.scaladsl.{ Sink, Source } import com.datastax.driver.core.{ Row, Session } import com.datastax.driver.extras.codecs.jdk8.InstantCodec import org.slf4j.LoggerFactory +import Async.ops._ import scala.concurrent.{ ExecutionContext, Future } -class CassandraScheduleEntryRepository(cassandraSession: CassandraSession, queries: Queries)( - implicit executionContext: ExecutionContext -) extends ScheduleEntryRepository { - private val log = LoggerFactory.getLogger(classOf[CassandraScheduleEntryRepository]) +class CassandraScheduleEntryRepository[F[_]: Async: CaptureFuture]( + cassandraSession: CassandraSession, + queries: Queries +)(implicit materializer: Materializer) + extends ScheduleEntryRepository[F] { + import materializer.executionContext + private val log = LoggerFactory.getLogger(classOf[CassandraScheduleEntryRepository[F]]) private val preparedInsertEntry = cassandraSession.prepare(queries.insertEntry) private val preparedSelectEntries = cassandraSession.prepare(queries.selectEntries) private val preparedSelectEntry = cassandraSession.prepare(queries.selectEntry) @@ -26,48 +32,52 @@ class CassandraScheduleEntryRepository(cassandraSession: CassandraSession, queri override def insertScheduleEntry(scheduleName: String, scheduleBucket: String, entryId: String, - dueDate: LocalDateTime): Future[Unit] = - preparedInsertEntry - .map( - _.bind() - .setString("schedule_name", scheduleName) - .setString("schedule_bucket", scheduleBucket) - .setString("entry_id", entryId) - .setString("time_bucket", TimeBucket(dueDate.toLocalDate).key) - .set("due_date", dueDate.atOffset(ZoneOffset.UTC).toInstant, classOf[Instant]) - .setBool("fired", false) - ) - .flatMap(cassandraSession.executeWrite) - .map(_ => ()) + dueDate: LocalDateTime): F[Unit] = + CaptureFuture[F].captureFuture { + preparedInsertEntry + .map( + _.bind() + .setString("schedule_name", scheduleName) + .setString("schedule_bucket", scheduleBucket) + .setString("entry_id", entryId) + .setString("time_bucket", TimeBucket(dueDate.toLocalDate).key) + .set("due_date", dueDate.toInstant(ZoneOffset.UTC), classOf[Instant]) + .setBool("fired", false) + ) + .flatMap(cassandraSession.executeWrite) + .map(_ => ()) + } override def markScheduleEntryAsFired(scheduleName: String, scheduleBucket: String, - entryId: String): Future[Unit] = - preparedSelectEntry - .map( - _.bind() - .setString("schedule_name", scheduleName) - .setString("schedule_bucket", scheduleBucket) - .setString("entry_id", entryId) - ) - .flatMap(cassandraSession.selectOne) - .flatMap { - case Some(row) => - preparedInsertEntry - .map( - _.bind() - .setString("schedule_name", scheduleName) - .setString("schedule_bucket", scheduleBucket) - .setString("entry_id", entryId) - .setString("time_bucket", row.getString("time_bucket")) - .setTimestamp("due_date", row.getTimestamp("due_date")) - .setBool("fired", true) - ) - .flatMap(cassandraSession.executeWrite) - .map(_ => ()) - case None => - Future.successful(()) - } + entryId: String): F[Unit] = + CaptureFuture[F].captureFuture { + preparedSelectEntry + .map( + _.bind() + .setString("schedule_name", scheduleName) + .setString("schedule_bucket", scheduleBucket) + .setString("entry_id", entryId) + ) + .flatMap(cassandraSession.selectOne) + .flatMap { + case Some(row) => + preparedInsertEntry + .map( + _.bind() + .setString("schedule_name", scheduleName) + .setString("schedule_bucket", scheduleBucket) + .setString("entry_id", entryId) + .setString("time_bucket", row.getString("time_bucket")) + .setTimestamp("due_date", row.getTimestamp("due_date")) + .setBool("fired", true) + ) + .flatMap(cassandraSession.executeWrite) + .map(_ => ()) + case None => + Future.successful(()) + } + } private def getBucket(timeBucket: TimeBucket, from: LocalDateTime, @@ -85,7 +95,7 @@ class CassandraScheduleEntryRepository(cassandraSession: CassandraSession, queri .map(fromRow) .named(s"getBucket($timeBucket, $from, $to)") - override def getEntries( + private def getEntries( from: LocalDateTime, to: LocalDateTime ): Source[ScheduleEntryRepository.ScheduleEntry, NotUsed] = @@ -108,6 +118,15 @@ class CassandraScheduleEntryRepository(cassandraSession: CassandraSession, queri rec(TimeBucket(from.toLocalDate)).named(s"getEntries($from, $to)") } + override def processEntries(from: LocalDateTime, to: LocalDateTime, parallelism: Int)( + f: (ScheduleEntry) => F[Unit] + ): F[Option[ScheduleEntry]] = + CaptureFuture[F].captureFuture { + getEntries(from, to) + .mapAsync(parallelism)(x => f(x).unsafeRun.map(_ => x)) + .runWith(Sink.lastOption) + } + private def fromRow(row: Row): ScheduleEntry = ScheduleEntry( row.getString("schedule_name"), @@ -121,9 +140,9 @@ class CassandraScheduleEntryRepository(cassandraSession: CassandraSession, queri object CassandraScheduleEntryRepository { - def apply(cassandraSession: CassandraSession, queries: Queries)( - implicit executionContext: ExecutionContext - ): CassandraScheduleEntryRepository = + def apply[F[_]: Async: CaptureFuture](cassandraSession: CassandraSession, queries: Queries)( + implicit materializer: Materializer + ): CassandraScheduleEntryRepository[F] = new CassandraScheduleEntryRepository(cassandraSession, queries) final case class TimeBucket(day: LocalDate, key: String) { diff --git a/schedule/src/main/scala/aecor/schedule/DefaultSchedule.scala b/schedule/src/main/scala/aecor/schedule/DefaultSchedule.scala index 8e377bd1..9bf68ea6 100644 --- a/schedule/src/main/scala/aecor/schedule/DefaultSchedule.scala +++ b/schedule/src/main/scala/aecor/schedule/DefaultSchedule.scala @@ -1,45 +1,55 @@ package aecor.schedule -import java.time.{ Clock, LocalDateTime } +import java.time.LocalDateTime import java.util.UUID -import aecor.aggregate.CorrelationId -import aecor.data.EventTag -import aecor.streaming._ +import aecor.data._ +import aecor.effect.Async +import aecor.effect.Async.ops._ +import aecor.runtime.akkapersistence.{ CommittableEventJournalQuery, JournalEntry } +import aecor.util.Clock import akka.NotUsed import akka.stream.scaladsl.Source - -import scala.concurrent.Future +import cats.Monad +import cats.implicits._ import scala.concurrent.duration.FiniteDuration -private[schedule] class DefaultSchedule(clock: Clock, - aggregate: ScheduleAggregate[Future], - bucketLength: FiniteDuration, - aggregateJournal: AggregateJournal[UUID], - offsetStore: OffsetStore[UUID], - eventTag: EventTag[ScheduleEvent]) - extends Schedule { +private[schedule] class DefaultSchedule[F[_]: Async: Monad]( + clock: Clock[F], + aggregate: ScheduleAggregate[F], + bucketLength: FiniteDuration, + aggregateJournal: CommittableEventJournalQuery[F, UUID, ScheduleEvent], + eventTag: EventTag +) extends Schedule[F] { override def addScheduleEntry(scheduleName: String, entryId: String, correlationId: CorrelationId, - dueDate: LocalDateTime): Future[Unit] = { - val scheduleBucket = - dueDate.atZone(clock.getZone).toEpochSecond / bucketLength.toSeconds - aggregate - .addScheduleEntry(scheduleName, scheduleBucket.toString, entryId, correlationId, dueDate) - } + dueDate: LocalDateTime): F[Unit] = + for { + zone <- clock.zone + scheduleBucket = dueDate.atZone(zone).toEpochSecond / bucketLength.toSeconds + _ <- aggregate + .addScheduleEntry( + scheduleName, + scheduleBucket.toString, + entryId, + correlationId, + dueDate + ) + } yield () override def committableScheduleEvents( scheduleName: String, consumerId: ConsumerId - ): Source[Committable[JournalEntry[UUID, ScheduleEvent]], NotUsed] = + ): Source[Committable[F, JournalEntry[UUID, ScheduleEvent]], NotUsed] = aggregateJournal - .committableEventsByTag[ScheduleEvent]( - offsetStore, - eventTag, - ConsumerId(scheduleName + consumerId.value) - ) - .collect { - case m if m.value.event.scheduleName == scheduleName => m + .eventsByTag(eventTag, ConsumerId(scheduleName + consumerId.value)) + .flatMapConcat { + case m if m.value.event.scheduleName == scheduleName => Source.single(m) + case other => + Source + .fromFuture(other.commit.unsafeRun) + .flatMapConcat(_ => Source.empty[Committable[F, JournalEntry[UUID, ScheduleEvent]]]) } + } diff --git a/schedule/src/main/scala/aecor/schedule/DefaultScheduleAggregate.scala b/schedule/src/main/scala/aecor/schedule/DefaultScheduleAggregate.scala new file mode 100644 index 00000000..5d8abdee --- /dev/null +++ b/schedule/src/main/scala/aecor/schedule/DefaultScheduleAggregate.scala @@ -0,0 +1,150 @@ +package aecor.schedule + +import java.time.{ Instant, LocalDateTime, ZonedDateTime } + +import aecor.data.Folded.syntax._ +import aecor.data._ +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentEncoder } +import aecor.schedule.ScheduleEvent.{ ScheduleEntryAdded, ScheduleEntryFired } +import aecor.schedule.ScheduleState._ +import aecor.schedule.protobuf.ScheduleEventCodec +import cats.Functor +import cats.implicits._ + +object DefaultScheduleAggregate { + + def apply[F[_]: Functor]( + clock: F[ZonedDateTime] + ): ScheduleAggregate[Handler[F, ScheduleState, ScheduleEvent, ?]] = + new DefaultScheduleAggregate(clock) + + def correlation: Correlation[ScheduleOp] = + c => CorrelationId.composite("-", c.scheduleName, c.scheduleBucket) + + def behavior[F[_]: Functor]( + clock: F[ZonedDateTime] + ): EventsourcedBehavior[F, ScheduleOp, ScheduleState, ScheduleEvent] = + EventsourcedBehavior(DefaultScheduleAggregate(clock).asFunctionK, ScheduleState.folder) + +} + +class DefaultScheduleAggregate[F[_]: Functor](clock: F[ZonedDateTime]) + extends ScheduleAggregate[Handler[F, ScheduleState, ScheduleEvent, ?]] { + + override def addScheduleEntry( + scheduleName: String, + scheduleBucket: String, + entryId: String, + correlationId: CorrelationId, + dueDate: LocalDateTime + ): Handler[F, ScheduleState, ScheduleEvent, Unit] = + Handler { state => + clock.map { zdt => + val timestamp = zdt.toInstant + val now = zdt.toLocalDateTime + if (state.unfired.get(entryId).isDefined || state.fired.contains(entryId)) { + Vector.empty -> (()) + } else { + val scheduleEntryAdded = ScheduleEntryAdded( + scheduleName, + scheduleBucket, + entryId, + correlationId, + dueDate, + timestamp + ) + val firedEvent = if (dueDate.isEqual(now) || dueDate.isBefore(now)) { + Vector( + ScheduleEntryFired(scheduleName, scheduleBucket, entryId, correlationId, timestamp) + ) + } else { + Vector.empty + } + + (scheduleEntryAdded +: firedEvent, ()) + } + } + + } + override def fireEntry(scheduleName: String, + scheduleBucket: String, + entryId: String): Handler[F, ScheduleState, ScheduleEvent, Unit] = + Handler { state => + clock.map(_.toInstant).map { timestamp => + state + .findEntry(entryId) + .map( + entry => + ScheduleEntryFired( + scheduleName, + scheduleBucket, + entry.id, + entry.correlationId, + timestamp + ) + ) + .toVector -> (()) + } + } +} + +sealed abstract class ScheduleEvent extends Product with Serializable { + def scheduleName: String + def scheduleBucket: String + def entryId: String + def timestamp: Instant +} + +object ScheduleEvent extends ScheduleEventInstances { + final case class ScheduleEntryAdded(scheduleName: String, + scheduleBucket: String, + entryId: String, + correlationId: CorrelationId, + dueDate: LocalDateTime, + timestamp: Instant) + extends ScheduleEvent + + final case class ScheduleEntryFired(scheduleName: String, + scheduleBucket: String, + entryId: String, + correlationId: CorrelationId, + timestamp: Instant) + extends ScheduleEvent +} + +trait ScheduleEventInstances { + implicit val persistentEncoder: PersistentEncoder[ScheduleEvent] = + PersistentEncoder.fromCodec(ScheduleEventCodec) + implicit val persistentDecoder: PersistentDecoder[ScheduleEvent] = + PersistentDecoder.fromCodec(ScheduleEventCodec) +} + +private[aecor] case class ScheduleState(unfired: Map[String, ScheduleEntry], fired: Set[String]) { + def addEntry(entryId: String, + correlationId: CorrelationId, + dueDate: LocalDateTime): ScheduleState = + copy(unfired = unfired + (entryId -> ScheduleEntry(entryId, correlationId, dueDate))) + + def markEntryAsFired(entryId: String): ScheduleState = + copy(unfired = unfired - entryId, fired = fired + entryId) + + def findEntry(entryId: String): Option[ScheduleEntry] = + unfired.get(entryId) + + def update(event: ScheduleEvent): Folded[ScheduleState] = event match { + case ScheduleEntryAdded(_, _, entryId, correlationId, dueDate, _) => + addEntry(entryId, correlationId, dueDate).next + case e: ScheduleEntryFired => + markEntryAsFired(e.entryId).next + } +} + +private[aecor] object ScheduleState { + + def initial: ScheduleState = ScheduleState(Map.empty, Set.empty) + + case class ScheduleEntry(id: String, correlationId: CorrelationId, dueDate: LocalDateTime) + + val folder: Folder[Folded, ScheduleEvent, ScheduleState] = + Folder.curried(initial)(_.update) +} diff --git a/schedule/src/main/scala/aecor/schedule/Schedule.scala b/schedule/src/main/scala/aecor/schedule/Schedule.scala index bbd9ba7d..a5f8903b 100644 --- a/schedule/src/main/scala/aecor/schedule/Schedule.scala +++ b/schedule/src/main/scala/aecor/schedule/Schedule.scala @@ -1,111 +1,116 @@ package aecor.schedule -import java.time.{ Clock, LocalDate, LocalDateTime } +import java.time.{ Clock => _, _ } import java.util.UUID -import aecor.aggregate.{ AkkaRuntime, CorrelationId, Tagging } -import aecor.data.EventTag -import aecor.streaming._ +import aecor.data._ +import aecor.effect.{ Async, Capture, CaptureFuture } +import aecor.runtime.akkapersistence._ +import aecor.schedule.process.{ + DefaultScheduleEventJournal, + PeriodicProcessRuntime, + ScheduleProcess +} +import aecor.util.{ Clock, KeyValueStore } import akka.NotUsed import akka.actor.ActorSystem import akka.stream.Materializer import akka.stream.scaladsl.Source -import cats.data.Reader +import cats.Monad +import cats.implicits._ +import com.datastax.driver.core.utils.UUIDs -import scala.concurrent.Future import scala.concurrent.duration._ -trait Schedule { +trait Schedule[F[_]] { def addScheduleEntry(scheduleName: String, entryId: String, correlationId: CorrelationId, - dueDate: LocalDateTime): Future[Unit] + dueDate: LocalDateTime): F[Unit] def committableScheduleEvents( scheduleName: String, consumerId: ConsumerId - ): Source[Committable[JournalEntry[UUID, ScheduleEvent]], NotUsed] + ): Source[Committable[F, JournalEntry[UUID, ScheduleEvent]], NotUsed] } -private[schedule] class ConfiguredSchedule( - system: ActorSystem, - entityName: String, - clock: Clock, - dayZero: LocalDate, - bucketLength: FiniteDuration, - refreshInterval: FiniteDuration, - eventualConsistencyDelay: FiniteDuration, - repository: ScheduleEntryRepository, - aggregateJournal: AggregateJournal[UUID], - offsetStore: OffsetStore[UUID], - consumerId: ConsumerId -)(implicit materializer: Materializer) { +object Schedule { + final case class ScheduleSettings(bucketLength: FiniteDuration, + refreshInterval: FiniteDuration, + eventualConsistencyDelay: FiniteDuration, + consumerId: ConsumerId) - private val runtime = AkkaRuntime(system) + def start[F[_]: Async: CaptureFuture: Capture: Monad]( + entityName: String, + dayZero: LocalDate, + clock: Clock[F], + repository: ScheduleEntryRepository[F], + offsetStore: KeyValueStore[F, TagConsumer, UUID], + settings: ScheduleSettings = ScheduleSettings( + 1.day, + 10.seconds, + 40.seconds, + ConsumerId("io.aecor.schedule.ScheduleProcess") + ) + )(implicit system: ActorSystem, materializer: Materializer): F[Schedule[F]] = { - private val eventTag = EventTag[ScheduleEvent](entityName) + val eventTag = EventTag(entityName) - private val startAggregate = Reader { _: Unit => - ScheduleAggregate.fromFunctionK( - runtime.start( - entityName, - DefaultScheduleAggregate(clock).asFunctionK, - DefaultScheduleAggregate.correlation, - Tagging(eventTag) - ) + val runtime = AkkaPersistenceRuntime( + system, + entityName, + DefaultScheduleAggregate.correlation, + DefaultScheduleAggregate.behavior(clock.zonedDateTime), + Tagging.const(eventTag) ) - } - private def startProcess(aggregate: ScheduleAggregate[Future]) = - ScheduleProcess( - clock = clock, - entityName = entityName, - consumerId = consumerId, - dayZero = dayZero, - refreshInterval = refreshInterval, - eventualConsistencyDelay = eventualConsistencyDelay, - parallelism = 8, - offsetStore = offsetStore, - repository = repository, - scheduleAggregate = aggregate, - aggregateJournal = aggregateJournal, - eventTag = eventTag - ).run(system) + def uuidToLocalDateTime(zoneId: ZoneId): KeyValueStore[F, TagConsumer, LocalDateTime] = + offsetStore.imap( + uuid => LocalDateTime.ofInstant(Instant.ofEpochMilli(UUIDs.unixTimestamp(uuid)), zoneId), + value => UUIDs.startOf(value.atZone(zoneId).toInstant.toEpochMilli) + ) + + def startAggregate = + for { + f <- runtime.start + } yield ScheduleAggregate.fromFunctionK(f) - private def createSchedule(aggregate: ScheduleAggregate[Future]): Schedule = - new DefaultSchedule(clock, aggregate, bucketLength, aggregateJournal, offsetStore, eventTag) + def startProcess(aggregate: ScheduleAggregate[F]) = clock.zone.map { zone => + val journal = + DefaultScheduleEventJournal[F]( + settings.consumerId, + 8, + runtime.journal.committable(offsetStore), + eventTag + ) + + val process = ScheduleProcess( + journal, + dayZero, + settings.consumerId, + uuidToLocalDateTime(zone), + settings.eventualConsistencyDelay, + repository, + aggregate, + clock.localDateTime, + 8 + ) + PeriodicProcessRuntime(entityName, settings.refreshInterval, process).run(system) + } + + def createSchedule(aggregate: ScheduleAggregate[F]): Schedule[F] = + new DefaultSchedule( + clock, + aggregate, + settings.bucketLength, + runtime.journal.committable(offsetStore), + eventTag + ) - def start: Reader[Unit, Schedule] = for { aggregate <- startAggregate _ <- startProcess(aggregate) schedule = createSchedule(aggregate) } yield schedule -} + } -object Schedule { - def start( - entityName: String, - clock: Clock, - dayZero: LocalDate, - bucketLength: FiniteDuration, - refreshInterval: FiniteDuration, - eventualConsistencyDelay: FiniteDuration, - repository: ScheduleEntryRepository, - aggregateJournal: AggregateJournal[UUID], - offsetStore: OffsetStore[UUID], - consumerId: ConsumerId = ConsumerId("io.aecor.schedule.ScheduleProcess") - )(implicit system: ActorSystem, materializer: Materializer): Reader[Unit, Schedule] = - new ConfiguredSchedule( - system, - entityName, - clock, - dayZero, - bucketLength, - refreshInterval, - eventualConsistencyDelay, - repository, - aggregateJournal, - offsetStore, - consumerId - ).start } diff --git a/schedule/src/main/scala/aecor/schedule/ScheduleAggregate.scala b/schedule/src/main/scala/aecor/schedule/ScheduleAggregate.scala index 1b7a5928..a0e70715 100644 --- a/schedule/src/main/scala/aecor/schedule/ScheduleAggregate.scala +++ b/schedule/src/main/scala/aecor/schedule/ScheduleAggregate.scala @@ -1,93 +1,29 @@ package aecor.schedule -import java.time.{ Clock, Instant, LocalDateTime } +import java.time.LocalDateTime -import aecor.aggregate._ -import aecor.aggregate.serialization.{ PersistentDecoder, PersistentEncoder } -import aecor.data.Folded.syntax._ -import aecor.data.{ Folded, Handler } -import aecor.schedule.ScheduleCommand.{ AddScheduleEntry, FireEntry } -import aecor.schedule.ScheduleEvent.{ ScheduleEntryAdded, ScheduleEntryFired } -import aecor.schedule.ScheduleState.ScheduleEntry -import aecor.schedule.protobuf.ScheduleEventCodec -import cats.arrow.FunctionK +import aecor.data.CorrelationId +import aecor.schedule.ScheduleOp.{ AddScheduleEntry, FireEntry } import cats.~> -sealed abstract class ScheduleEvent extends Product with Serializable { +sealed abstract class ScheduleOp[A] extends Product with Serializable { def scheduleName: String def scheduleBucket: String - def entryId: String - def timestamp: Instant } -object ScheduleEvent extends ScheduleEventInstances { - final case class ScheduleEntryAdded(scheduleName: String, - scheduleBucket: String, - entryId: String, - correlationId: CorrelationId, - dueDate: LocalDateTime, - timestamp: Instant) - extends ScheduleEvent - - final case class ScheduleEntryFired(scheduleName: String, - scheduleBucket: String, - entryId: String, - correlationId: CorrelationId, - timestamp: Instant) - extends ScheduleEvent -} - -trait ScheduleEventInstances { - implicit val persistentEncoder: PersistentEncoder[ScheduleEvent] = - PersistentEncoder.fromCodec(ScheduleEventCodec) - implicit val persistentDecoder: PersistentDecoder[ScheduleEvent] = - PersistentDecoder.fromCodec(ScheduleEventCodec) -} - -sealed abstract class ScheduleCommand[A] extends Product with Serializable -object ScheduleCommand { +object ScheduleOp { final case class AddScheduleEntry(scheduleName: String, scheduleBucket: String, entryId: String, correlationId: CorrelationId, dueDate: LocalDateTime) - extends ScheduleCommand[Unit] + extends ScheduleOp[Unit] final case class FireEntry(scheduleName: String, scheduleBucket: String, entryId: String) - extends ScheduleCommand[Unit] -} - -private[aecor] case class ScheduleState(entries: Map[String, ScheduleEntry]) { - def addEntry(entryId: String, - correlationId: CorrelationId, - dueDate: LocalDateTime): ScheduleState = - copy(entries = entries + (entryId -> ScheduleEntry(entryId, correlationId, dueDate))) - - def removeEntry(entryId: String): ScheduleState = - copy(entries = entries - entryId) - - def findEntry(entryId: String): Option[ScheduleEntry] = - entries.get(entryId) - - def update(event: ScheduleEvent): Folded[ScheduleState] = event match { - case ScheduleEntryAdded(_, _, entryId, correlationId, dueDate, _) => - addEntry(entryId, correlationId, dueDate).next - case e: ScheduleEntryFired => - removeEntry(e.entryId).next - } + extends ScheduleOp[Unit] } -private[aecor] object ScheduleState { - - def initial: ScheduleState = ScheduleState(Map.empty) - - case class ScheduleEntry(id: String, correlationId: CorrelationId, dueDate: LocalDateTime) - - implicit val folder: Folder[Folded, ScheduleEvent, ScheduleState] = - Folder.instance(ScheduleState(Map.empty))(_.update) -} - -private[schedule] trait ScheduleAggregate[F[_]] { +private[aecor] trait ScheduleAggregate[F[_]] { def addScheduleEntry(scheduleName: String, scheduleBucket: String, entryId: String, @@ -96,8 +32,8 @@ private[schedule] trait ScheduleAggregate[F[_]] { def fireEntry(scheduleName: String, scheduleBucket: String, entryId: String): F[Unit] - def asFunctionK: ScheduleCommand ~> F = - λ[ScheduleCommand ~> F] { + final def asFunctionK: ScheduleOp ~> F = + λ[ScheduleOp ~> F] { case AddScheduleEntry(scheduleName, scheduleBucket, entryId, correlationId, dueDate) => addScheduleEntry(scheduleName, scheduleBucket, entryId, correlationId, dueDate) case FireEntry(scheduleName, scheduleBucket, entryId) => @@ -105,8 +41,8 @@ private[schedule] trait ScheduleAggregate[F[_]] { } } -private[schedule] object ScheduleAggregate { - def fromFunctionK[F[_]](f: ScheduleCommand ~> F): ScheduleAggregate[F] = +private[aecor] object ScheduleAggregate { + def fromFunctionK[F[_]](f: ScheduleOp ~> F): ScheduleAggregate[F] = new ScheduleAggregate[F] { override def addScheduleEntry(scheduleName: String, scheduleBucket: String, @@ -121,77 +57,3 @@ private[schedule] object ScheduleAggregate { f(FireEntry(scheduleName, scheduleBucket, entryId)) } } - -private[aecor] object DefaultScheduleAggregate { - - def apply(clock: Clock): ScheduleAggregate[Handler[ScheduleState, ScheduleEvent, ?]] = - new DefaultScheduleAggregate(clock) - - def correlation: Correlation[ScheduleCommand] = { - def mk[A](c: ScheduleCommand[A]): CorrelationIdF[A] = - c match { - case AddScheduleEntry(scheduleName, scheduleBucket, _, _, _) => - CorrelationId.composite("-", scheduleName, scheduleBucket) - case FireEntry(scheduleName, scheduleBucket, _) => - CorrelationId.composite("-", scheduleName, scheduleBucket) - } - FunctionK.lift(mk _) - } - -} - -private[schedule] class DefaultScheduleAggregate(clock: Clock) - extends ScheduleAggregate[Handler[ScheduleState, ScheduleEvent, ?]] { - - private def timestamp = clock.instant() - - override def addScheduleEntry( - scheduleName: String, - scheduleBucket: String, - entryId: String, - correlationId: CorrelationId, - dueDate: LocalDateTime - ): Handler[ScheduleState, ScheduleEvent, Unit] = - Handler { state => - if (state.entries.get(entryId).isDefined) { - Vector.empty -> (()) - } else { - val scheduleEntryAdded = ScheduleEntryAdded( - scheduleName, - scheduleBucket, - entryId, - correlationId, - dueDate, - timestamp - ) - val now = LocalDateTime.now(clock) - val firedEvent = if (dueDate.isEqual(now) || dueDate.isBefore(now)) { - Vector( - ScheduleEntryFired(scheduleName, scheduleBucket, entryId, correlationId, timestamp) - ) - } else { - Vector.empty - } - - (scheduleEntryAdded +: firedEvent, ()) - } - } - override def fireEntry(scheduleName: String, - scheduleBucket: String, - entryId: String): Handler[ScheduleState, ScheduleEvent, Unit] = - Handler { state => - state - .findEntry(entryId) - .map( - entry => - ScheduleEntryFired( - scheduleName, - scheduleBucket, - entry.id, - entry.correlationId, - timestamp - ) - ) - .toVector -> (()) - } -} diff --git a/schedule/src/main/scala/aecor/schedule/ScheduleEntryRepository.scala b/schedule/src/main/scala/aecor/schedule/ScheduleEntryRepository.scala index fd87a569..23a171e4 100644 --- a/schedule/src/main/scala/aecor/schedule/ScheduleEntryRepository.scala +++ b/schedule/src/main/scala/aecor/schedule/ScheduleEntryRepository.scala @@ -3,20 +3,18 @@ package aecor.schedule import java.time.LocalDateTime import aecor.schedule.ScheduleEntryRepository.ScheduleEntry -import akka.NotUsed -import akka.stream.scaladsl.Source -import scala.concurrent.Future - -trait ScheduleEntryRepository { +trait ScheduleEntryRepository[F[_]] { def insertScheduleEntry(scheduleName: String, scheduleBucket: String, entryId: String, - dueDate: LocalDateTime): Future[Unit] + dueDate: LocalDateTime): F[Unit] def markScheduleEntryAsFired(scheduleName: String, scheduleBucket: String, - entryId: String): Future[Unit] - def getEntries(from: LocalDateTime, to: LocalDateTime): Source[ScheduleEntry, NotUsed] + entryId: String): F[Unit] + def processEntries(from: LocalDateTime, to: LocalDateTime, parallelism: Int)( + f: (ScheduleEntry) => F[Unit] + ): F[Option[ScheduleEntry]] } object ScheduleEntryRepository { diff --git a/schedule/src/main/scala/aecor/schedule/ScheduleProcess.scala b/schedule/src/main/scala/aecor/schedule/ScheduleProcess.scala deleted file mode 100644 index 7f58665e..00000000 --- a/schedule/src/main/scala/aecor/schedule/ScheduleProcess.scala +++ /dev/null @@ -1,141 +0,0 @@ -package aecor.schedule - -import java.time._ -import java.time.temporal.ChronoUnit -import java.util.UUID - -import aecor.data.EventTag -import aecor.schedule.ScheduleEvent.{ ScheduleEntryAdded, ScheduleEntryFired } -import aecor.streaming.StreamSupervisor.StreamKillSwitch -import aecor.streaming._ -import akka.actor.ActorSystem -import akka.stream.Materializer -import akka.stream.scaladsl.{ Flow, Sink, Source } -import cats.data.Reader -import com.datastax.driver.core.utils.UUIDs -import scala.concurrent.Future -import scala.concurrent.duration.FiniteDuration -import cats.instances.future._ -import org.slf4j.LoggerFactory - -private[schedule] class ScheduleProcess( - clock: Clock, - entityName: String, - consumerId: ConsumerId, - dayZero: LocalDate, - refreshInterval: FiniteDuration, - eventualConsistencyDelay: FiniteDuration, - parallelism: Int, - offsetStore: OffsetStore[UUID], - repository: ScheduleEntryRepository, - scheduleAggregate: ScheduleAggregate[Future], - aggregateJournal: AggregateJournal[UUID], - eventTag: EventTag[ScheduleEvent] -)(implicit materializer: Materializer) { - - import materializer.executionContext - - private val log = LoggerFactory.getLogger(classOf[ScheduleProcess]) - - private val scheduleEntriesTag = "io.aecor.ScheduleDueEntries" - - private def source = - Source - .single(()) - .mapAsync(1)(_ => offsetStore.getOffset(scheduleEntriesTag, consumerId)) - .map { - case Some(offset) => - LocalDateTime.ofInstant(Instant.ofEpochMilli(UUIDs.unixTimestamp(offset)), clock.getZone) - case None => - dayZero.atStartOfDay() - } - .mapAsync(1)(runProcessCycle) - - private def runProcessCycle(from: LocalDateTime): Future[Unit] = - for { - updatedCounter <- updateRepository - _ = log.debug(s"Schedule entries view updated, new entries = [$updatedCounter]") - now = LocalDateTime.now(clock) - _ <- fireEntries(from, now) - _ <- afterRefreshInterval { - runProcessCycle(now.minus(eventualConsistencyDelay.toMillis, ChronoUnit.MILLIS)) - } - } yield () - - private def updateRepository: Future[Int] = - aggregateJournal - .committableCurrentEventsByTag(offsetStore, eventTag, consumerId) - .map(_.map(_.event)) - .mapAsync(parallelism)(_.traverse { - case ScheduleEntryAdded(scheduleName, scheduleBucket, entryId, _, dueDate, _) => - repository - .insertScheduleEntry(scheduleName, scheduleBucket, entryId, dueDate) - .map(_ => 1) - case ScheduleEntryFired(scheduleName, scheduleBucket, entryId, _, _) => - repository.markScheduleEntryAsFired(scheduleName, scheduleBucket, entryId).map(_ => 0) - }) - .fold(Committable.pure(0)) { (acc, x) => - x.copy(value = acc.value + x.value) - } - .mapAsync(1)(x => x.commit().map(_ => x.value)) - .runWith(Sink.head) - - private def fireEntries(from: LocalDateTime, to: LocalDateTime) = - repository - .getEntries(from, to) - .map { entry => - val offset = UUIDs.startOf(entry.dueDate.atZone(clock.getZone).toInstant.toEpochMilli) - Committable(() => offsetStore.setOffset(scheduleEntriesTag, consumerId, offset), entry) - } - .mapAsync(parallelism)(_.traverse { entry => - if (entry.fired) - Future.successful(()) - else - scheduleAggregate.fireEntry(entry.scheduleName, entry.scheduleBucket, entry.entryId) - }) - .mapAsync(1)(_.commit()) - .runWith(Sink.ignore) - - private def afterRefreshInterval[A](f: => Future[A]): Future[A] = - Source - .tick(refreshInterval, refreshInterval, ()) - .take(1) - .mapAsync(1)(_ => f) - .runWith(Sink.head) - - def run(system: ActorSystem): Reader[Unit, StreamKillSwitch] = Reader { _ => - StreamSupervisor(system) - .startClusterSingleton(s"$entityName-Process", source, Flow[Unit]) - } -} - -object ScheduleProcess { - def apply( - clock: Clock, - entityName: String, - consumerId: ConsumerId, - dayZero: LocalDate, - refreshInterval: FiniteDuration, - eventualConsistencyDelay: FiniteDuration, - parallelism: Int, - offsetStore: OffsetStore[UUID], - repository: ScheduleEntryRepository, - scheduleAggregate: ScheduleAggregate[Future], - aggregateJournal: AggregateJournal[UUID], - eventTag: EventTag[ScheduleEvent] - )(implicit materializer: Materializer): ScheduleProcess = - new ScheduleProcess( - clock, - entityName, - consumerId, - dayZero, - refreshInterval, - eventualConsistencyDelay, - parallelism, - offsetStore, - repository, - scheduleAggregate, - aggregateJournal, - eventTag - ) -} diff --git a/schedule/src/main/scala/aecor/schedule/process/DefaultScheduleEventJournal.scala b/schedule/src/main/scala/aecor/schedule/process/DefaultScheduleEventJournal.scala new file mode 100644 index 00000000..999ff402 --- /dev/null +++ b/schedule/src/main/scala/aecor/schedule/process/DefaultScheduleEventJournal.scala @@ -0,0 +1,43 @@ +package aecor.schedule.process + +import java.util.UUID + +import aecor.data.{ Committable, ConsumerId, EventTag } +import aecor.effect.Async.ops._ +import aecor.effect.{ Async, CaptureFuture } +import aecor.runtime.akkapersistence.{ CommittableEventJournalQuery } +import aecor.schedule.ScheduleEvent + +import akka.stream.Materializer +import akka.stream.scaladsl.{ Keep, Sink } +import cats.Applicative +import cats.implicits._ + +object DefaultScheduleEventJournal { + def apply[F[_]: Async: CaptureFuture: Applicative]( + consumerId: ConsumerId, + parallelism: Int, + aggregateJournal: CommittableEventJournalQuery[F, UUID, ScheduleEvent], + eventTag: EventTag + )(implicit materializer: Materializer): DefaultScheduleEventJournal[F] = + new DefaultScheduleEventJournal(consumerId, parallelism, aggregateJournal, eventTag) +} + +class DefaultScheduleEventJournal[F[_]: Async: CaptureFuture: Applicative]( + consumerId: ConsumerId, + parallelism: Int, + aggregateJournal: CommittableEventJournalQuery[F, UUID, ScheduleEvent], + eventTag: EventTag +)(implicit materializer: Materializer) + extends ScheduleEventJournal[F] { + import materializer.executionContext + override def processNewEvents(f: (ScheduleEvent) => F[Unit]): F[Unit] = + CaptureFuture[F].captureFuture { + aggregateJournal + .currentEventsByTag(eventTag, consumerId) + .mapAsync(parallelism)(_.map(_.event).traverse(f.andThen(_.unsafeRun))) + .fold(Committable.unit[F])(Keep.right) + .mapAsync(1)(_.commit.unsafeRun) + .runWith(Sink.ignore) + }.void +} diff --git a/schedule/src/main/scala/aecor/schedule/process/PeriodicProcessRuntime.scala b/schedule/src/main/scala/aecor/schedule/process/PeriodicProcessRuntime.scala new file mode 100644 index 00000000..b691a4e7 --- /dev/null +++ b/schedule/src/main/scala/aecor/schedule/process/PeriodicProcessRuntime.scala @@ -0,0 +1,43 @@ +package aecor.schedule.process + +import aecor.effect.{ Async, Capture, CaptureFuture } + +import scala.collection.immutable._ +import akka.actor.ActorSystem +import akka.stream.Materializer +import akka.stream.scaladsl.{ Flow, Source } +import cats.Monad +import Async.ops._ +import aecor.distributedprocessing.{ AkkaStreamProcess, DistributedProcessing } +import akka.NotUsed + +import scala.concurrent.duration.FiniteDuration + +import scala.concurrent.duration._ + +object PeriodicProcessRuntime { + def apply[F[_]: Async: CaptureFuture: Capture: Monad]( + name: String, + tickInterval: FiniteDuration, + processCycle: F[Unit] + )(implicit materializer: Materializer): PeriodicProcessRuntime[F] = + new PeriodicProcessRuntime[F](name, tickInterval, processCycle) +} + +class PeriodicProcessRuntime[F[_]: Async: CaptureFuture: Capture: Monad]( + name: String, + tickInterval: FiniteDuration, + processCycle: F[Unit] +)(implicit materializer: Materializer) { + + private def source = + Source + .tick(0.seconds, tickInterval, processCycle) + .mapAsync(1)(_.unsafeRun) + .mapMaterializedValue(_ => NotUsed) + + def run(system: ActorSystem): F[DistributedProcessing.ProcessKillSwitch[F]] = + DistributedProcessing(system) + .start[F](s"$name-Process", Seq(AkkaStreamProcess[F](source, Flow[Unit]))) + +} diff --git a/schedule/src/main/scala/aecor/schedule/process/ScheduleEventJournal.scala b/schedule/src/main/scala/aecor/schedule/process/ScheduleEventJournal.scala new file mode 100644 index 00000000..1eefc5e3 --- /dev/null +++ b/schedule/src/main/scala/aecor/schedule/process/ScheduleEventJournal.scala @@ -0,0 +1,7 @@ +package aecor.schedule.process + +import aecor.schedule.ScheduleEvent + +trait ScheduleEventJournal[F[_]] { + def processNewEvents(f: ScheduleEvent => F[Unit]): F[Unit] +} diff --git a/schedule/src/main/scala/aecor/schedule/process/ScheduleProcess.scala b/schedule/src/main/scala/aecor/schedule/process/ScheduleProcess.scala new file mode 100644 index 00000000..8a779b7c --- /dev/null +++ b/schedule/src/main/scala/aecor/schedule/process/ScheduleProcess.scala @@ -0,0 +1,64 @@ +package aecor.schedule.process + +import java.time.temporal.ChronoUnit +import java.time.{ Clock => _, _ } + +import aecor.data.{ ConsumerId, EventTag, TagConsumer } +import aecor.schedule.ScheduleEvent.{ ScheduleEntryAdded, ScheduleEntryFired } +import aecor.schedule.{ ScheduleAggregate, ScheduleEntryRepository } +import aecor.util.KeyValueStore +import cats.Monad +import cats.implicits._ + +import scala.concurrent.duration.FiniteDuration + +object ScheduleProcess { + def apply[F[_]: Monad](journal: ScheduleEventJournal[F], + dayZero: LocalDate, + consumerId: ConsumerId, + offsetStore: KeyValueStore[F, TagConsumer, LocalDateTime], + eventualConsistencyDelay: FiniteDuration, + repository: ScheduleEntryRepository[F], + scheduleAggregate: ScheduleAggregate[F], + clock: F[LocalDateTime], + parallelism: Int): F[Unit] = { + val scheduleEntriesTag = EventTag("io.aecor.ScheduleDueEntries") + + val tagConsumerId = TagConsumer(scheduleEntriesTag, consumerId) + + val updateRepository: F[Unit] = + journal.processNewEvents { + case ScheduleEntryAdded(scheduleName, scheduleBucket, entryId, _, dueDate, _) => + repository + .insertScheduleEntry(scheduleName, scheduleBucket, entryId, dueDate) + case ScheduleEntryFired(scheduleName, scheduleBucket, entryId, _, _) => + repository.markScheduleEntryAsFired(scheduleName, scheduleBucket, entryId) + } + def fireEntries(from: LocalDateTime, + to: LocalDateTime): F[Option[ScheduleEntryRepository.ScheduleEntry]] = + repository.processEntries(from, to, parallelism) { entry => + if (entry.fired) + ().pure[F] + else + scheduleAggregate + .fireEntry(entry.scheduleName, entry.scheduleBucket, entry.entryId) + } + + val loadOffset: F[LocalDateTime] = + offsetStore + .getValue(tagConsumerId) + .map(_.getOrElse(dayZero.atStartOfDay())) + + def saveOffset(value: LocalDateTime): F[Unit] = + offsetStore.setValue(tagConsumerId, value) + + for { + _ <- updateRepository + from <- loadOffset + now <- clock + entry <- fireEntries(from.minus(eventualConsistencyDelay.toMillis, ChronoUnit.MILLIS), now) + _ <- entry.map(_.dueDate).traverse(saveOffset) + } yield () + } + +} diff --git a/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventCodec.scala b/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventCodec.scala index e03aa0aa..46ae1d17 100644 --- a/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventCodec.scala +++ b/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventCodec.scala @@ -2,11 +2,11 @@ package aecor.schedule.protobuf import java.time.{ Instant, LocalDateTime, ZoneOffset } -import aecor.aggregate.serialization.Codec +import aecor.runtime.akkapersistence.serialization.Codec import aecor.schedule.ScheduleEvent import aecor.schedule.serialization.protobuf.msg.{ ScheduleEntryAdded, ScheduleEntryFired } -import scala.util.Try +import scala.util.{ Failure, Try } object ScheduleEventCodec extends Codec[ScheduleEvent] { val ScheduleEntryAddedManifest = "A" @@ -51,6 +51,7 @@ object ScheduleEventCodec extends Codec[ScheduleEvent] { Instant.ofEpochMilli(timestamp) ) } + case other => Failure(new IllegalArgumentException(s"Unknown manifest [$other]")) } override def encode(o: ScheduleEvent): Array[Byte] = o match { diff --git a/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventSerializer.scala b/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventSerializer.scala index b5db8695..6b5ffe2a 100644 --- a/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventSerializer.scala +++ b/schedule/src/main/scala/aecor/schedule/protobuf/ScheduleEventSerializer.scala @@ -1,6 +1,6 @@ package aecor.schedule.protobuf -import aecor.aggregate.serialization.PersistentReprSerializer +import aecor.runtime.akkapersistence.serialization.PersistentReprSerializer import akka.actor.ExtendedActorSystem /** diff --git a/tests/src/test/scala/aecor/tests/AkkaPersistenceRuntimeSpec.scala b/tests/src/test/scala/aecor/tests/AkkaPersistenceRuntimeSpec.scala new file mode 100644 index 00000000..0fee842b --- /dev/null +++ b/tests/src/test/scala/aecor/tests/AkkaPersistenceRuntimeSpec.scala @@ -0,0 +1,66 @@ +package aecor.tests + +import aecor.data.Tagging +import aecor.tests.e2e.{ CounterEvent, CounterOp, CounterOpHandler } +import akka.actor.ActorSystem +import akka.testkit.TestKit +import com.typesafe.config.{ Config, ConfigFactory } +import monix.eval.Task +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.{ BeforeAndAfterAll, FunSuiteLike, Matchers } +import monix.cats._ +import aecor.effect.monix._ +import aecor.runtime.akkapersistence.AkkaPersistenceRuntime +import monix.execution.Scheduler + +import scala.concurrent.duration._ + +object AkkaPersistenceRuntimeSpec { + def conf: Config = ConfigFactory.parseString(s""" + cluster.system-name=test + akka.persistence.journal.plugin=akka.persistence.journal.inmem + akka.persistence.snapshot-store.plugin=akka.persistence.no-snapshot-store + aecor.akka-runtime.idle-timeout = 1s + cluster.seed-nodes = ["akka://test@127.0.0.1:51000"] + """).withFallback(ConfigFactory.load()) +} + +class AkkaPersistenceRuntimeSpec + extends TestKit(ActorSystem("test", AkkaPersistenceRuntimeSpec.conf)) + with FunSuiteLike + with Matchers + with ScalaFutures + with BeforeAndAfterAll { + + override implicit val patienceConfig = PatienceConfig(15.seconds, 150.millis) + + implicit val scheduler = Scheduler(system.dispatcher) + + override def afterAll: Unit = + TestKit.shutdownActorSystem(system) + + val runtime = AkkaPersistenceRuntime( + system, + "Counter", + CounterOp.correlation, + CounterOpHandler.behavior[Task], + Tagging.const(CounterEvent.tag) + ) + + test("Runtime should work") { + val program = for { + runtime <- runtime.start + _ <- runtime(CounterOp.Increment("1")) + _ <- runtime(CounterOp.Increment("2")) + _2 <- runtime(CounterOp.GetValue("2")) + _ <- runtime(CounterOp.Decrement("1")) + _1 <- runtime(CounterOp.GetValue("1")) + afterPassivation <- runtime(CounterOp.GetValue("2")).delayExecution(2.seconds) + } yield (_1, _2, afterPassivation) + + val (_1, _2, afterPassivation) = program.runAsync.futureValue + _1 shouldBe 0L + _2 shouldBe 1L + afterPassivation shouldBe 1L + } +} diff --git a/tests/src/test/scala/aecor/tests/CompositeCorrelationIdSpec.scala b/tests/src/test/scala/aecor/tests/CompositeCorrelationIdSpec.scala index 3d2f5645..dcab73cd 100644 --- a/tests/src/test/scala/aecor/tests/CompositeCorrelationIdSpec.scala +++ b/tests/src/test/scala/aecor/tests/CompositeCorrelationIdSpec.scala @@ -1,6 +1,6 @@ package aecor.tests -import aecor.aggregate.CorrelationId +import aecor.data.CorrelationId import org.scalatest.{ FlatSpec, Matchers } class CompositeCorrelationIdSpec extends FlatSpec with Matchers { diff --git a/tests/src/test/scala/aecor/tests/DistributedSourceWorkerSpec.scala b/tests/src/test/scala/aecor/tests/DistributedSourceWorkerSpec.scala new file mode 100644 index 00000000..6a806eb0 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/DistributedSourceWorkerSpec.scala @@ -0,0 +1,37 @@ +package aecor.tests + +import akka.actor.ActorSystem +import akka.stream.OverflowStrategy +import akka.stream.scaladsl.Source +import akka.testkit.TestKit +import com.typesafe.config.{ Config, ConfigFactory } +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.{ BeforeAndAfterAll, FunSuiteLike, Matchers } + +import scala.concurrent.duration._ + +object DistributedSourceWorkerSpec { + def conf: Config = ConfigFactory.parseString(s""" + cluster.system-name=test + akka.persistence.journal.plugin=akka.persistence.journal.inmem + akka.persistence.snapshot-store.plugin=akka.persistence.no-snapshot-store + aecor.akka-runtime.idle-timeout = 1s + cluster.seed-nodes = ["akka://test@127.0.0.1:51000"] + """).withFallback(ConfigFactory.load()) +} + +class DistributedSourceWorkerSpec + extends TestKit(ActorSystem("test", ShardedRuntimeSpec.conf)) + with FunSuiteLike + with Matchers + with ScalaFutures + with BeforeAndAfterAll { + + override implicit val patienceConfig = PatienceConfig(15.seconds, 150.millis) + + override def afterAll: Unit = + TestKit.shutdownActorSystem(system) + + val sink = Source.queue(10, OverflowStrategy.backpressure) + +} diff --git a/tests/src/test/scala/aecor/tests/EndToEndTest.scala b/tests/src/test/scala/aecor/tests/EndToEndTest.scala new file mode 100644 index 00000000..e18becf4 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/EndToEndTest.scala @@ -0,0 +1,207 @@ +package aecor.tests + +import java.time._ + +import aecor.data._ +import aecor.effect.Capture +import aecor.schedule.ScheduleEntryRepository.ScheduleEntry +import aecor.schedule._ +import aecor.schedule.process.{ ScheduleEventJournal, ScheduleProcess } +import aecor.testkit.StateEventJournal.State +import aecor.testkit.{ E2eSupport, StateClock, StateEventJournal, StateKeyValueStore } +import aecor.tests.e2e.CounterOp.{ Decrement, Increment } +import aecor.tests.e2e.TestCounterViewRepository.TestCounterViewRepositoryState +import aecor.tests.e2e._ +import aecor.tests.e2e.notification.{ NotificationEvent, NotificationOp } +import cats.data.StateT +import cats.implicits._ +import org.scalatest.{ FunSuite, Matchers } +import shapeless.Coproduct + +import scala.concurrent.duration._ + +class EndToEndTest extends FunSuite with Matchers with E2eSupport { + + def instant[F[_]: Capture]: F[Instant] = + Capture[F].capture(Instant.ofEpochMilli(System.currentTimeMillis())) + + case class SpecState(counterJournalState: StateEventJournal.State[CounterEvent], + notificationJournalState: StateEventJournal.State[NotificationEvent], + scheduleJournalState: StateEventJournal.State[ScheduleEvent], + counterViewState: TestCounterViewRepositoryState, + time: Instant, + scheduleEntries: Vector[ScheduleEntry], + offsetStoreState: Map[TagConsumer, LocalDateTime]) + + val offsetStore = + StateKeyValueStore[SpecF, SpecState, TagConsumer, LocalDateTime]( + _.offsetStoreState, + (s, os) => s.copy(offsetStoreState = os) + ) + + val clock = StateClock[SpecF, SpecState](ZoneOffset.UTC, _.time, (s, t) => s.copy(time = t)) + + def counterEventJournal = + mkJournal[CounterEvent](_.counterJournalState, (x, a) => x.copy(counterJournalState = a)) + + def counterBehavior = + mkBehavior[CounterOp, CounterState, CounterEvent]( + CounterOpHandler.behavior[StateT[SpecF, SpecState, ?]], + _.id, + Tagging.const(CounterEvent.tag), + counterEventJournal + ) + + def notificationEventJournal = + mkJournal[NotificationEvent]( + _.notificationJournalState, + (x, a) => x.copy(notificationJournalState = a) + ) + + def notificationBehavior = + mkBehavior( + notification.behavior, + NotificationOp.correlation, + Tagging.const(NotificationEvent.tag), + notificationEventJournal + ) + + def schduleEventJournal = + mkJournal[ScheduleEvent](_.scheduleJournalState, (x, a) => x.copy(scheduleJournalState = a)) + + val scheduleAggregate = mkBehavior[ScheduleOp, ScheduleState, ScheduleEvent]( + DefaultScheduleAggregate.behavior(clock.zonedDateTime), + DefaultScheduleAggregate.correlation, + Tagging.const(EventTag("Schedule")), + schduleEventJournal + ) + + val scheduleEntryRepository = TestScheduleEntryRepository[SpecF, SpecState]( + _.scheduleEntries, + (x, a) => x.copy(scheduleEntries = a) + ) + + val scheduleProcessConsumerId: ConsumerId = ConsumerId("NotificationProcess") + val wrappedEventJournal = new ScheduleEventJournal[StateT[SpecF, SpecState, ?]] { + override def processNewEvents( + f: (ScheduleEvent) => StateT[SpecF, SpecState, Unit] + ): StateT[SpecF, SpecState, Unit] = + schduleEventJournal + .eventsByTag(EventTag("Schedule"), scheduleProcessConsumerId) + .process(f) + } + + val scheduleProcess = ScheduleProcess[StateT[SpecF, SpecState, ?]]( + journal = wrappedEventJournal, + dayZero = LocalDate.now(), + consumerId = scheduleProcessConsumerId, + offsetStore = offsetStore, + eventualConsistencyDelay = 1.second, + repository = scheduleEntryRepository, + scheduleAggregate = ScheduleAggregate.fromFunctionK(scheduleAggregate), + clock = clock.localDateTime, + parallelism = 1 + ) + + val counterViewProcessConsumerId: ConsumerId = ConsumerId("CounterViewProcess") + + val notificationProcessConsumerId: ConsumerId = ConsumerId("NotificationProcess") + + override def otherStuff: Vector[StateT[SpecF, SpecState, Unit]] = + Vector(scheduleProcess) + + override def processes: Vector[WiredProcess[StateT[SpecF, SpecState, ?]]] = Vector( + wireProcess( + CounterViewProcess( + TestCounterViewRepository[SpecF, SpecState]( + _.counterViewState, + (x, a) => x.copy(counterViewState = a) + ), + counterBehavior + ), + counterEventJournal + .eventsByTag(CounterEvent.tag, counterViewProcessConsumerId) + ), + wireProcess( + NotificationProcess(counterBehavior, notificationBehavior), + counterEventJournal + .eventsByTag(CounterEvent.tag, notificationProcessConsumerId) + .map(Coproduct[NotificationProcess.Input](_)), + notificationEventJournal + .eventsByTag(NotificationEvent.tag, notificationProcessConsumerId) + .map(Coproduct[NotificationProcess.Input](_)) + ) + ) + + def tickSeconds(seconds: Long) = wired(clock.tick)(java.time.Duration.ofSeconds(seconds)) + + test("Process should react to events") { + + val counter = wiredK(counterBehavior) + + val program = for { + _ <- counter(Increment("1")) + _ <- counter(Increment("1")) + _ <- counter(Decrement("1")) + _ <- counter(Increment("2")) + _ <- counter(Increment("2")) + } yield () + + val Right((state, _)) = program + .run( + SpecState( + State.init, + State.init, + State.init, + TestCounterViewRepositoryState.init, + Instant.now(), + Vector.empty, + Map.empty + ) + ) + .value + .value + + state.counterViewState.value shouldBe Map("1" -> 1L, "2" -> 2L) + + state.notificationJournalState.eventsById + .getOrElse("1-2", Vector.empty) should have size (2) + } + + test("Schedule should fire") { + + val schedule = wiredK(scheduleAggregate) + + def program(n: Int): StateT[SpecF, SpecState, Unit] = + for { + now <- clock.localDateTime + _ <- schedule(ScheduleOp.AddScheduleEntry("foo", "b", "e1", "cid", now.plusSeconds(3))) + _ <- schedule(ScheduleOp.AddScheduleEntry("foo", "b", "e2", "cid", now.plusSeconds(5))) + _ <- tickSeconds(3) + _ <- tickSeconds(2) + _ <- if (n == 0) { + ().pure[StateT[SpecF, SpecState, ?]] + } else { + program(n - 1) + } + } yield () + + val Right((state, _)) = program(100) + .run( + SpecState( + State.init, + State.init, + State.init, + TestCounterViewRepositoryState.init, + Instant.now(Clock.systemUTC()), + Vector.empty, + Map.empty + ) + ) + .value + .value + + state.scheduleEntries.exists(e => e.entryId == "e1" && e.fired) shouldBe true + state.scheduleEntries.exists(e => e.entryId == "e2" && e.fired) shouldBe true + } +} diff --git a/tests/src/test/scala/aecor/tests/PersistentEncoderCirce.scala b/tests/src/test/scala/aecor/tests/PersistentEncoderCirce.scala new file mode 100644 index 00000000..4b41ada7 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/PersistentEncoderCirce.scala @@ -0,0 +1,30 @@ +package aecor.tests + +import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets + +import aecor.runtime.akkapersistence.serialization.{ + DecodingFailure, + PersistentDecoder, + PersistentEncoder, + PersistentRepr +} +import io.circe.{ Decoder, Encoder, jawn } + +object PersistentEncoderCirce { + def circePersistentEncoder[A](implicit encoder: Encoder[A]): PersistentEncoder[A] = + PersistentEncoder.instance( + e => PersistentRepr("", encoder(e).noSpaces.getBytes(StandardCharsets.UTF_8)) + ) + + def circePersistentDecoder[A](implicit decoder: Decoder[A]): PersistentDecoder[A] = + PersistentDecoder.instance( + repr => + jawn + .parseByteBuffer(ByteBuffer.wrap(repr.payload)) + .right + .flatMap(decoder.decodeJson) + .left + .map(DecodingFailure.fromThrowable) + ) +} diff --git a/tests/src/test/scala/aecor/tests/ScheduleAggregateSpec.scala b/tests/src/test/scala/aecor/tests/ScheduleAggregateSpec.scala index 4dff1929..ceda7e79 100644 --- a/tests/src/test/scala/aecor/tests/ScheduleAggregateSpec.scala +++ b/tests/src/test/scala/aecor/tests/ScheduleAggregateSpec.scala @@ -1,14 +1,15 @@ package aecor.tests -import java.time.{ Clock, Instant, LocalDateTime, ZoneId } +import java.time._ import scala.collection.immutable._ import aecor.schedule.{ DefaultScheduleAggregate, ScheduleEvent, ScheduleState } +import cats.Id import org.scalatest.{ FlatSpec, Matchers } class ScheduleAggregateSpec extends FlatSpec with Matchers { val clock = Clock.fixed(Instant.now, ZoneId.systemDefault()) - val aggregate = DefaultScheduleAggregate(clock) + val aggregate = DefaultScheduleAggregate[Id](ZonedDateTime.now(clock)) "ScheduleAggregate" should "fire entry when due date is before now" in { val handler = aggregate.addScheduleEntry( diff --git a/tests/src/test/scala/aecor/tests/ScheduleEventCodecSpec.scala b/tests/src/test/scala/aecor/tests/ScheduleEventCodecSpec.scala index ff5b2d7e..880a257d 100644 --- a/tests/src/test/scala/aecor/tests/ScheduleEventCodecSpec.scala +++ b/tests/src/test/scala/aecor/tests/ScheduleEventCodecSpec.scala @@ -13,8 +13,8 @@ import scala.util.Success class ScheduleEventCodecSpec extends AkkaSpec with PropertyChecks { val codec = ScheduleEventCodec - implicit val arbitraryLocalDateTime = Arbitrary(Gen.const(LocalDateTime.now())) - implicit val arbitraryInstant = Arbitrary(Gen.const(Instant.now())) + implicit val arbitraryLocalDateTime = Arbitrary(Gen.lzy(Gen.const(LocalDateTime.now()))) + implicit val arbitraryInstant = Arbitrary(Gen.lzy(Gen.const(Instant.now()))) "ScheduleEventCodec" must { "be able to encode and decode ScheduleEvent" in { diff --git a/tests/src/test/scala/aecor/tests/ShardedRuntimeSpec.scala b/tests/src/test/scala/aecor/tests/ShardedRuntimeSpec.scala new file mode 100644 index 00000000..a317592a --- /dev/null +++ b/tests/src/test/scala/aecor/tests/ShardedRuntimeSpec.scala @@ -0,0 +1,67 @@ +package aecor.tests + +import aecor.data.{ Behavior, EventsourcedBehavior } +import aecor.effect.monix._ +import aecor.runtime.akkageneric.GenericAkkaRuntime +import aecor.testkit.StateRuntime +import aecor.tests.e2e.{ CounterEvent, CounterOp, CounterOpHandler, CounterState } +import akka.actor.ActorSystem +import akka.testkit.TestKit +import com.typesafe.config.{ Config, ConfigFactory } +import monix.cats._ +import monix.eval.Task +import monix.execution.Scheduler +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.{ BeforeAndAfterAll, FunSuiteLike, Matchers } + +import scala.concurrent.duration._ + +object ShardedRuntimeSpec { + def conf: Config = ConfigFactory.parseString(s""" + cluster.system-name=test + akka.persistence.journal.plugin=akka.persistence.journal.inmem + akka.persistence.snapshot-store.plugin=akka.persistence.no-snapshot-store + aecor.akka-runtime.idle-timeout = 1s + cluster.seed-nodes = ["akka://test@127.0.0.1:51000"] + """).withFallback(ConfigFactory.load()) +} + +class ShardedRuntimeSpec + extends TestKit(ActorSystem("test", ShardedRuntimeSpec.conf)) + with FunSuiteLike + with Matchers + with ScalaFutures + with BeforeAndAfterAll { + + override implicit val patienceConfig = PatienceConfig(15.seconds, 150.millis) + + implicit val scheduler = Scheduler(system.dispatcher) + + override def afterAll: Unit = + TestKit.shutdownActorSystem(system) + + val behavior: Behavior[Task, CounterOp] = Behavior.fromState( + Vector.empty[CounterEvent], + StateRuntime.shared(EventsourcedBehavior(CounterOpHandler[Task], CounterState.folder)) + ) + + val startRuntime = + GenericAkkaRuntime[Task](system).start("Counter", CounterOp.correlation, behavior) + + test("Runtime should work") { + val program = for { + runtime <- startRuntime + _ <- runtime(CounterOp.Increment("1")) + _ <- runtime(CounterOp.Increment("2")) + second <- runtime(CounterOp.GetValue("2")) + _ <- runtime(CounterOp.Decrement("1")) + _1 <- runtime(CounterOp.GetValue("1")) + secondAfterPassivation <- runtime(CounterOp.GetValue("2")).delayExecution(2.seconds) + } yield (_1, second, secondAfterPassivation) + + val (_1, _2, afterPassivation) = program.runAsync.futureValue + _1 shouldBe 0L + _2 shouldBe 1L + afterPassivation shouldBe 0 + } +} diff --git a/tests/src/test/scala/aecor/tests/StateRuntimeSpec.scala b/tests/src/test/scala/aecor/tests/StateRuntimeSpec.scala index 785d6013..cef86cd2 100644 --- a/tests/src/test/scala/aecor/tests/StateRuntimeSpec.scala +++ b/tests/src/test/scala/aecor/tests/StateRuntimeSpec.scala @@ -1,52 +1,23 @@ package aecor.tests -import aecor.aggregate.{ Correlation, CorrelationIdF, Folder, StateRuntime } -import aecor.data.Handler -import cats.{ Id, Monad, ~> } -import org.scalatest.{ FunSuite, Matchers } +import aecor.data.EventsourcedBehavior +import aecor.testkit.StateRuntime +import aecor.tests.e2e.CounterEvent.{ CounterDecremented, CounterIncremented } +import aecor.tests.e2e.CounterOp.{ Decrement, Increment } +import aecor.tests.e2e.{ CounterEvent, CounterOp, CounterOpHandler, CounterState } import cats.implicits._ +import cats.{ Monad, ~> } +import org.scalatest.{ FunSuite, Matchers } class StateRuntimeSpec extends FunSuite with Matchers { - sealed trait CounterOp[A] { - def id: String - } - case class Increment(id: String) extends CounterOp[Long] - case class Decrement(id: String) extends CounterOp[Long] - - val correlation = new (Correlation[CounterOp]) { - override def apply[A](fa: CounterOp[A]): CorrelationIdF[A] = fa.id - } - - sealed trait CounterEvent - case class CounterIncremented(id: String) extends CounterEvent - case class CounterDecremented(id: String) extends CounterEvent - case class CounterState(value: Long) - object CounterState { - implicit val folder: Folder[Id, CounterEvent, CounterState] = - Folder.instance(CounterState(0)) { - case CounterState(x) => { - case CounterIncremented(_) => CounterState(x + 1) - case CounterDecremented(_) => CounterState(x - 1) - } - } - } - val behavior: CounterOp ~> Handler[CounterState, CounterEvent, ?] = - Lambda[CounterOp ~> Handler[CounterState, CounterEvent, ?]] { - case Increment(id) => - Handler { x => - Vector(CounterIncremented(id)) -> (x.value + 1) - } - case Decrement(id) => - Handler { x => - Vector(CounterDecremented(id)) -> (x.value - 1) - } - } val sharedRuntime = - StateRuntime.shared[CounterOp, CounterState, CounterEvent, Id](behavior) + StateRuntime.shared[Either[Throwable, ?], CounterOp, CounterState, CounterEvent]( + EventsourcedBehavior(CounterOpHandler[Either[Throwable, ?]], CounterState.folder) + ) val correlatedRuntime = - StateRuntime.correlated[CounterOp, CounterState, CounterEvent, Id](behavior, correlation) + StateRuntime.correlate(sharedRuntime, CounterOp.correlation) def mkProgram[F[_]: Monad](runtime: CounterOp ~> F): F[Long] = for { @@ -58,7 +29,7 @@ class StateRuntimeSpec extends FunSuite with Matchers { test("Shared runtime should execute all commands against shared sequence of events") { val program = mkProgram(sharedRuntime) - val (state, result) = program.run(Vector.empty) + val Right((state, result)) = program.run(Vector.empty) state shouldBe Vector( CounterIncremented("1"), @@ -73,7 +44,7 @@ class StateRuntimeSpec extends FunSuite with Matchers { ) { val program = mkProgram(correlatedRuntime) - val (state, result) = program.run(Map.empty) + val Right((state, result)) = program.run(Map.empty) state shouldBe Map( "1" -> Vector(CounterIncremented("1"), CounterDecremented("1")), diff --git a/tests/src/test/scala/aecor/tests/e2e/CounterOp.scala b/tests/src/test/scala/aecor/tests/e2e/CounterOp.scala new file mode 100644 index 00000000..bdcdf2a5 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/CounterOp.scala @@ -0,0 +1,69 @@ +package aecor.tests.e2e +import aecor.data._ +import aecor.runtime.akkapersistence.serialization.{ PersistentDecoder, PersistentEncoder } +import aecor.tests.PersistentEncoderCirce +import aecor.tests.e2e.CounterEvent.{ CounterDecremented, CounterIncremented } +import aecor.tests.e2e.CounterOp.{ Decrement, GetValue, Increment } +import cats.implicits._ +import cats.{ Applicative, ~> } +import io.circe.generic.auto._ + +import scala.collection.immutable.Seq + +sealed trait CounterOp[A] { + def id: String +} + +object CounterOp { + case class Increment(id: String) extends CounterOp[Long] + case class Decrement(id: String) extends CounterOp[Long] + case class GetValue(id: String) extends CounterOp[Long] + val correlation: Correlation[CounterOp] = Correlation[CounterOp](_.id) +} + +sealed trait CounterEvent +object CounterEvent { + case class CounterIncremented(id: String) extends CounterEvent + case class CounterDecremented(id: String) extends CounterEvent + val tag: EventTag = EventTag("Counter") + implicit def encoder: PersistentEncoder[CounterEvent] = + PersistentEncoderCirce.circePersistentEncoder[CounterEvent] + implicit def decoder: PersistentDecoder[CounterEvent] = + PersistentEncoderCirce.circePersistentDecoder[CounterEvent] +} + +case class CounterState(value: Long) +object CounterState { + def folder[F[_]: Applicative]: Folder[F, CounterEvent, CounterState] = + Folder.curried(CounterState(0)) { + case CounterState(x) => { + case CounterIncremented(_) => CounterState(x + 1).pure[F] + case CounterDecremented(_) => CounterState(x - 1).pure[F] + } + } +} + +object CounterOpHandler { + def apply[F[_]: Applicative]: CounterOp ~> Handler[F, CounterState, CounterEvent, ?] = + new CounterOpHandler[F] + + def behavior[F[_]: Applicative]: EventsourcedBehavior[F, CounterOp, CounterState, CounterEvent] = + EventsourcedBehavior(CounterOpHandler[F], CounterState.folder[Folded]) +} + +class CounterOpHandler[F[_]: Applicative] + extends (CounterOp ~> Handler[F, CounterState, CounterEvent, ?]) { + override def apply[A](fa: CounterOp[A]): Handler[F, CounterState, CounterEvent, A] = + fa match { + case Increment(id) => + Handler.lift { x => + Seq(CounterIncremented(id)) -> (x.value + 1) + } + case Decrement(id) => + Handler.lift { x => + Seq(CounterDecremented(id)) -> (x.value - 1) + } + case GetValue(id) => + Handler.lift(x => Seq.empty -> x.value) + } +} diff --git a/tests/src/test/scala/aecor/tests/e2e/CounterViewProcess.scala b/tests/src/test/scala/aecor/tests/e2e/CounterViewProcess.scala new file mode 100644 index 00000000..b4c697e4 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/CounterViewProcess.scala @@ -0,0 +1,21 @@ +package aecor.tests.e2e + +import aecor.tests.e2e.CounterEvent.{ CounterDecremented, CounterIncremented } +import cats.implicits._ +import cats.{ Monad, ~> } + +object CounterViewProcess { + def apply[F[_]: Monad](repo: CounterViewRepository[F], + counter: CounterOp ~> F): CounterEvent => F[Unit] = { + case CounterIncremented(id) => + for { + state <- repo.getCounterState(id) + _ <- repo.setCounterState(id, state.getOrElse(0L) + 1L) + } yield () + case CounterDecremented(id) => + for { + state <- repo.getCounterState(id) + _ <- repo.setCounterState(id, state.getOrElse(0L) - 1L) + } yield () + } +} diff --git a/tests/src/test/scala/aecor/tests/e2e/CounterViewRepository.scala b/tests/src/test/scala/aecor/tests/e2e/CounterViewRepository.scala new file mode 100644 index 00000000..7795d684 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/CounterViewRepository.scala @@ -0,0 +1,37 @@ +package aecor.tests.e2e + +import aecor.tests.e2e.TestCounterViewRepository.TestCounterViewRepositoryState +import cats.Applicative +import cats.data.StateT + +trait CounterViewRepository[F[_]] { + def getCounterState(id: String): F[Option[Long]] + def setCounterState(id: String, value: Long): F[Unit] +} + +object TestCounterViewRepository { + case class TestCounterViewRepositoryState(value: Map[String, Long]) + object TestCounterViewRepositoryState { + def init: TestCounterViewRepositoryState = TestCounterViewRepositoryState(Map.empty) + } + def apply[F[_]: Applicative, A]( + extract: A => TestCounterViewRepositoryState, + update: (A, TestCounterViewRepositoryState) => A + ): TestCounterViewRepository[F, A] = + new TestCounterViewRepository(extract, update) +} + +class TestCounterViewRepository[F[_]: Applicative, A]( + extract: A => TestCounterViewRepositoryState, + update: (A, TestCounterViewRepositoryState) => A +) extends CounterViewRepository[StateT[F, A, ?]] { + def getCounterState(id: String): StateT[F, A, Option[Long]] = + StateT + .get[F, TestCounterViewRepositoryState] + .map(_.value.get(id)) + .transformS(extract, update) + def setCounterState(id: String, value: Long): StateT[F, A, Unit] = + StateT + .modify[F, TestCounterViewRepositoryState](x => x.copy(x.value.updated(id, value))) + .transformS(extract, update) +} diff --git a/tests/src/test/scala/aecor/tests/e2e/NotificationProcess.scala b/tests/src/test/scala/aecor/tests/e2e/NotificationProcess.scala new file mode 100644 index 00000000..6cf98291 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/NotificationProcess.scala @@ -0,0 +1,39 @@ +package aecor.tests.e2e + +import aecor.tests.e2e.CounterEvent.CounterIncremented +import aecor.tests.e2e.notification.{ NotificationEvent, NotificationOp } +import cats.{ Monad, ~> } +import shapeless.{ :+:, CNil, HNil } +import aecor.util.FunctionBuilder.syntax._ +import cats.implicits._ + +object NotificationProcess { + type Input = CounterEvent :+: NotificationEvent :+: CNil + + def apply[F[_]: Monad](counterFacade: CounterOp ~> F, + notificationFacade: NotificationOp ~> F): Input => F[Unit] = + build { + at[CounterEvent] { + case CounterIncremented(counterId) => + for { + value <- counterFacade(CounterOp.GetValue(counterId)) + _ <- if (value % 2 == 0) { + notificationFacade( + NotificationOp.CreateNotification(s"$counterId-$value", counterId) + ) + } else { + ().pure[F] + } + } yield () + case _ => ().pure[F] + } :: + at[NotificationEvent] { + case NotificationEvent.NotificationCreated(nid, _) => + notificationFacade(NotificationOp.MarkAsSent(nid)) + case _ => + ().pure[F] + } :: + HNil + } + +} diff --git a/tests/src/test/scala/aecor/tests/e2e/TestScheduleEntryRepository.scala b/tests/src/test/scala/aecor/tests/e2e/TestScheduleEntryRepository.scala new file mode 100644 index 00000000..d8c22c20 --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/TestScheduleEntryRepository.scala @@ -0,0 +1,78 @@ +package aecor.tests.e2e + +import java.time.LocalDateTime + +import aecor.schedule.CassandraScheduleEntryRepository.TimeBucket +import aecor.schedule.ScheduleEntryRepository +import aecor.schedule.ScheduleEntryRepository.ScheduleEntry +import cats.Monad +import cats.data.StateT +import cats.implicits.none +import cats.implicits._ + +object TestScheduleEntryRepository { + def apply[F[_]: Monad, S]( + extract: S => Vector[ScheduleEntry], + update: (S, Vector[ScheduleEntry]) => S + ): ScheduleEntryRepository[StateT[F, S, ?]] = + new TestScheduleEntryRepository(extract, update) +} + +class TestScheduleEntryRepository[F[_]: Monad, S](extract: S => Vector[ScheduleEntry], + update: (S, Vector[ScheduleEntry]) => S) + extends ScheduleEntryRepository[StateT[F, S, ?]] { + override def insertScheduleEntry(scheduleName: String, + scheduleBucket: String, + entryId: String, + dueDate: LocalDateTime): StateT[F, S, Unit] = + StateT + .modify[F, Vector[ScheduleEntry]] { scheduleEntries => + scheduleEntries :+ ScheduleEntry( + scheduleName, + scheduleBucket, + entryId, + dueDate, + TimeBucket(dueDate.toLocalDate).key, + false + ) + } + .transformS(extract, update) + override def markScheduleEntryAsFired(scheduleName: String, + scheduleBucket: String, + entryId: String): StateT[F, S, Unit] = + StateT + .modify[F, Vector[ScheduleEntry]] { scheduleEntries => + scheduleEntries.map { e => + if (e.scheduleName == scheduleName && scheduleBucket == e.scheduleBucket && e.entryId == entryId) { + e.copy(fired = true) + } else { + e + } + } + } + .transformS(extract, update) + + override def processEntries(from: LocalDateTime, to: LocalDateTime, parallelism: Int)( + f: (ScheduleEntryRepository.ScheduleEntry) => StateT[F, S, Unit] + ): StateT[F, S, Option[ScheduleEntryRepository.ScheduleEntry]] = + StateT + .get[F, Vector[ScheduleEntry]] + .map { entries => + val x: StateT[F, S, Option[ScheduleEntry]] = entries + .foldLeft( + StateT + .pure[F, S, Option[ScheduleEntryRepository.ScheduleEntry]](none) + ) { (acc, entry) => + if (entry.dueDate.isAfter(from) && (entry.dueDate + .isBefore(to) || entry.dueDate == to)) { + acc.flatMap(_ => f(entry)).map(_ => entry.some) + } else { + acc + } + } + x + } + .transformS(extract, update) + .flatten + +} diff --git a/tests/src/test/scala/aecor/tests/e2e/notification.scala b/tests/src/test/scala/aecor/tests/e2e/notification.scala new file mode 100644 index 00000000..98b3362c --- /dev/null +++ b/tests/src/test/scala/aecor/tests/e2e/notification.scala @@ -0,0 +1,58 @@ +package aecor.tests.e2e + +import aecor.data._ +import aecor.tests.e2e.notification.NotificationEvent.{ NotificationCreated, NotificationSent } +import aecor.tests.e2e.notification.NotificationOp.{ CreateNotification, MarkAsSent } +import cats.implicits._ +import cats.{ Applicative, ~> } + +object notification { + sealed abstract class NotificationOp[A] extends Product with Serializable { + def notificationId: String + } + object NotificationOp { + final case class CreateNotification(notificationId: String, counterId: String) + extends NotificationOp[Unit] + final case class MarkAsSent(notificationId: String) extends NotificationOp[Unit] + val correlation: Correlation[NotificationOp] = Correlation[NotificationOp](_.notificationId) + } + + sealed trait NotificationEvent + object NotificationEvent { + case class NotificationCreated(notificationId: String, counterId: String) + extends NotificationEvent + case class NotificationSent(notificationId: String) extends NotificationEvent + val tag: EventTag = EventTag("Notification") + } + + case class NotificationState(sent: Boolean) + object NotificationState { + def folder[F[_]: Applicative]: Folder[F, NotificationEvent, NotificationState] = + Folder.curried(NotificationState(false)) { + case NotificationState(_) => { + case NotificationCreated(_, _) => NotificationState(false).pure[F] + case NotificationSent(_) => NotificationState(true).pure[F] + } + } + } + + def notificationOpHandler[F[_]: Applicative] = + new (NotificationOp ~> Handler[F, NotificationState, NotificationEvent, ?]) { + override def apply[A]( + fa: NotificationOp[A] + ): Handler[F, NotificationState, NotificationEvent, A] = + fa match { + case CreateNotification(nid, cid) => + Handler.lift { _ => + Vector(NotificationCreated(nid, cid)) -> (()) + } + case MarkAsSent(id) => + Handler.lift { _ => + Vector(NotificationSent(id)) -> (()) + } + } + } + def behavior[F[_]: Applicative] + : EventsourcedBehavior[F, NotificationOp, NotificationState, NotificationEvent] = + EventsourcedBehavior(notificationOpHandler[F], NotificationState.folder[Folded]) +}