Remove ApiMayChange for Replicated Event Sourcing, #30346 (#30347)

This commit is contained in:
Patrik Nordwall 2021-06-28 17:11:55 +02:00 committed by GitHub
parent bbae718ef7
commit 8573c70883
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
12 changed files with 36 additions and 48 deletions

View file

@ -15,12 +15,10 @@ import akka.util.ccompat.JavaConverters._
import java.util.{ Set => JSet }
import akka.actor.typed.Behavior
import akka.annotation.ApiMayChange
import akka.cluster.sharding.typed.internal.EntityTypeKeyImpl
import akka.persistence.typed.ReplicationId
import akka.persistence.typed.ReplicationId.Separator
@ApiMayChange
object ReplicatedEntityProvider {
/**
@ -138,7 +136,6 @@ object ReplicatedEntityProvider {
*
* @tparam M The type of messages the replicated entity accepts
*/
@ApiMayChange
final class ReplicatedEntityProvider[M] private (
val replicas: immutable.Seq[(ReplicatedEntity[M], String)],
val directReplication: Boolean) {
@ -155,7 +152,6 @@ final class ReplicatedEntityProvider[M] private (
}
@ApiMayChange
object ReplicatedEntity {
/**
@ -181,5 +177,4 @@ object ReplicatedEntity {
* Settings for a specific replica id in replicated sharding
* Currently only Entity's with ShardingEnvelope are supported but this may change in the future
*/
@ApiMayChange
final class ReplicatedEntity[M] private (val replicaId: ReplicaId, val entity: Entity[M, ShardingEnvelope[M]])

View file

@ -7,7 +7,6 @@ package akka.cluster.sharding.typed
import akka.actor.typed.ActorSystem
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
import akka.annotation.ApiMayChange
import akka.annotation.DoNotInherit
import akka.cluster.sharding.typed.internal.ReplicatedShardingExtensionImpl
import akka.cluster.sharding.typed.scaladsl.EntityRef
@ -18,7 +17,6 @@ import java.util.{ Map => JMap }
* Extension for running Replicated Event Sourcing in sharding by starting one separate instance of sharding per replica.
* The sharding instances can be confined to datacenters or cluster roles or run on the same set of cluster nodes.
*/
@ApiMayChange
object ReplicatedShardingExtension extends ExtensionId[ReplicatedShardingExtension] {
override def createExtension(system: ActorSystem[_]): ReplicatedShardingExtension =
@ -32,7 +30,6 @@ object ReplicatedShardingExtension extends ExtensionId[ReplicatedShardingExtensi
* Not for user extension.
*/
@DoNotInherit
@ApiMayChange
trait ReplicatedShardingExtension extends Extension {
/**
@ -61,7 +58,6 @@ trait ReplicatedShardingExtension extends Extension {
* Not for user extension.
*/
@DoNotInherit
@ApiMayChange
trait ReplicatedSharding[M] {
/**

View file

@ -31,6 +31,5 @@ These are the current complete modules marked as **may change**:
* @ref:[Multi Node Testing](../multi-node-testing.md)
* @ref:[Reliable Delivery](../typed/reliable-delivery.md)
* @ref:[Sharded Daemon Process](../typed/cluster-sharded-daemon-process.md)
* @ref:[Replicated Event Sourcing](../typed/replicated-eventsourcing.md)

View file

@ -1,13 +1,5 @@
# Replicated Event Sourcing
@@@ warning
This module is marked as @ref:[may change](../common/may-change.md) because it is a new feature that
needs feedback from real usage before finalizing the API. This means that API or semantics can change without
warning or deprecation period. It is also not recommended to use this module in production just yet.
@@@
@ref[Event Sourcing](./persistence.md) with `EventSourcedBehavior`s is based on the single writer principle, which means that there can only be one active instance of a `EventSourcedBehavior`
with a given `persistenceId`. Otherwise, multiple instances would store interleaving events based on different states, and when these events would later be replayed it would not be possible to reconstruct the correct state.

View file

@ -4,10 +4,13 @@
package docs.akka.persistence.typed
import scala.annotation.nowarn
import akka.actor.typed.ActorSystem
import akka.persistence.typed.ReplicaId
import akka.persistence.typed.ReplicationId
import akka.persistence.typed.scaladsl.{ EventSourcedBehavior, ReplicatedEventSourcing }
import scala.annotation.nowarn
import akka.persistence.typed.scaladsl.EventSourcedBehavior
import akka.persistence.typed.scaladsl.ReplicatedEventSourcing
@nowarn("msg=never used")
object ReplicatedEventSourcingCompileOnlySpec {
@ -24,21 +27,36 @@ object ReplicatedEventSourcingCompileOnlySpec {
trait State
trait Event
//#factory-shared
ReplicatedEventSourcing.commonJournalConfig(
ReplicationId("entityTypeHint", "entityId", DCA),
AllReplicas,
queryPluginId) { context =>
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
object Shared {
//#factory-shared
def apply(
system: ActorSystem[_],
entityId: String,
replicaId: ReplicaId): EventSourcedBehavior[Command, State, Event] = {
ReplicatedEventSourcing.commonJournalConfig(
ReplicationId("MyReplicatedEntity", entityId, replicaId),
AllReplicas,
queryPluginId) { replicationContext =>
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
}
}
//#factory-shared
}
//#factory-shared
//#factory
ReplicatedEventSourcing.perReplicaJournalConfig(
ReplicationId("entityTypeHint", "entityId", DCA),
Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { context =>
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
object PerReplica {
//#factory
def apply(
system: ActorSystem[_],
entityId: String,
replicaId: ReplicaId): EventSourcedBehavior[Command, State, Event] = {
ReplicatedEventSourcing.perReplicaJournalConfig(
ReplicationId("MyReplicatedEntity", entityId, replicaId),
Map(DCA -> "journalForDCA", DCB -> "journalForDCB")) { replicationContext =>
EventSourcedBehavior[Command, State, Event](???, ???, ???, ???)
}
}
//#factory
}
//#factory
}

View file

@ -3,9 +3,7 @@
*/
package akka.persistence.typed.crdt
import akka.annotation.ApiMayChange
@ApiMayChange
object Counter {
val empty: Counter = Counter(0)
@ -23,7 +21,6 @@ object Counter {
}
}
@ApiMayChange
final case class Counter(value: BigInt) extends OpCrdt[Counter.Updated] {
override type T = Counter

View file

@ -3,14 +3,12 @@
*/
package akka.persistence.typed.crdt
import akka.annotation.ApiMayChange
import akka.persistence.typed.ReplicaId
/**
* Utility class for comparing timestamp replica
* identifier when implementing last-writer wins.
*/
@ApiMayChange
final case class LwwTime(timestamp: Long, originReplica: ReplicaId) {
/**

View file

@ -7,12 +7,11 @@ package akka.persistence.typed.crdt
import scala.annotation.tailrec
import scala.collection.immutable
import akka.util.HashCode
import akka.annotation.{ ApiMayChange, InternalApi }
import akka.annotation.InternalApi
import akka.persistence.typed.ReplicaId
import akka.persistence.typed.crdt.ORSet.DeltaOp
import akka.persistence.typed.internal.{ ManyVersionVector, OneVersionVector, VersionVector }
@ApiMayChange
object ORSet {
def empty[A](originReplica: ReplicaId): ORSet[A] = new ORSet(originReplica.id, Map.empty, VersionVector.empty)
def apply[A](originReplica: ReplicaId): ORSet[A] = empty(originReplica)
@ -274,7 +273,6 @@ object ORSet {
*
* This class is immutable, i.e. "modifying" methods return a new instance.
*/
@ApiMayChange
final class ORSet[A] private[akka] (
val originReplica: String,
private[akka] val elementsMap: Map[A, ORSet.Dot],

View file

@ -4,9 +4,8 @@
package akka.persistence.typed.crdt
import akka.annotation.{ ApiMayChange, DoNotInherit }
import akka.annotation.DoNotInherit
@ApiMayChange
@DoNotInherit
trait OpCrdt[Operation] { self =>
type T <: OpCrdt[Operation] { type T = self.T }

View file

@ -305,7 +305,6 @@ object ReplicatedEventMetadata {
* For a journal supporting Replicated Event Sourcing needing to add test coverage, use this instance as metadata and defer
* to the built in serializer for serialization format
*/
@ApiMayChange
def instanceForJournalTest: Any = ReplicatedEventMetadata(ReplicaId("DC-A"), 1L, VersionVector.empty + "DC-A", true)
}
@ -328,7 +327,6 @@ object ReplicatedSnapshotMetadata {
* For a snapshot store supporting Replicated Event Sourcing needing to add test coverage, use this instance as metadata and defer
* to the built in serializer for serialization format
*/
@ApiMayChange
def instanceForSnapshotStoreTest: Any =
ReplicatedSnapshotMetadata(
VersionVector.empty + "DC-B" + "DC-A",

View file

@ -268,7 +268,7 @@ private[akka] object Running {
state: Running.RunningState[S],
envelope: ReplicatedEventEnvelope[E],
replication: ReplicationSetup): Behavior[InternalProtocol] = {
setup.internalLogger.infoN(
setup.internalLogger.debugN(
"Replica {} received replicated event. Replica seqs nrs: {}. Envelope {}",
setup.replication,
state.seenPerReplica,

View file

@ -9,14 +9,12 @@ import java.util.Optional
import akka.actor.typed.BackoffSupervisorStrategy
import akka.actor.typed.Behavior
import akka.actor.typed.TypedActorContext
import akka.annotation.ApiMayChange
import akka.annotation.InternalApi
import akka.persistence.typed.internal.ReplicationContextImpl
/**
* Base class for replicated event sourced behaviors.
*/
@ApiMayChange
abstract class ReplicatedEventSourcedBehavior[Command, Event, State](
replicationContext: ReplicationContext,
onPersistFailure: Optional[BackoffSupervisorStrategy])