Updated DistributedData Delete API to include optional request context.

#20140
This commit is contained in:
inakov 2017-01-14 14:30:58 +02:00
parent 4a9c753710
commit 65cba329d0
6 changed files with 95 additions and 41 deletions

View file

@ -318,7 +318,7 @@ object Replicator {
* *
* The subscriber will automatically be unregistered if it is terminated. * The subscriber will automatically be unregistered if it is terminated.
* *
* If the key is deleted the subscriber is notified with a [[DataDeleted]] * If the key is deleted the subscriber is notified with a [[Deleted]]
* message. * message.
*/ */
final case class Subscribe[A <: ReplicatedData](key: Key[A], subscriber: ActorRef) extends ReplicatorMessage final case class Subscribe[A <: ReplicatedData](key: Key[A], subscriber: ActorRef) extends ReplicatorMessage
@ -349,6 +349,10 @@ object Replicator {
def dataValue: A = data def dataValue: A = data
} }
final case class Deleted[A <: ReplicatedData](key: Key[A]) extends NoSerializationVerificationNeeded {
override def toString: String = s"Deleted [$key]"
}
object Update { object Update {
/** /**
@ -458,20 +462,40 @@ object Replicator {
* It will eventually be disseminated to other replicas, unless the local replica * It will eventually be disseminated to other replicas, unless the local replica
* crashes before it has been able to communicate with other replicas. * crashes before it has been able to communicate with other replicas.
*/ */
final case class StoreFailure[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends UpdateFailure[A] with DeleteResponse[A] final case class StoreFailure[A <: ReplicatedData](key: Key[A], request: Option[Any])
extends UpdateFailure[A] with DeleteResponse[A] {
/** Java API */
override def getRequest: Optional[Any] = Optional.ofNullable(request.orNull)
}
/** /**
* Send this message to the local `Replicator` to delete a data value for the * Send this message to the local `Replicator` to delete a data value for the
* given `key`. The `Replicator` will reply with one of the [[DeleteResponse]] messages. * given `key`. The `Replicator` will reply with one of the [[DeleteResponse]] messages.
*
* The optional `request` context is included in the reply messages. This is a convenient
* way to pass contextual information (e.g. original sender) without having to use `ask`
* or maintain local correlation data structures.
*/ */
final case class Delete[A <: ReplicatedData](key: Key[A], consistency: WriteConsistency) extends Command[A] final case class Delete[A <: ReplicatedData](key: Key[A], consistency: WriteConsistency, request: Option[Any] = None)
extends Command[A] with NoSerializationVerificationNeeded {
sealed trait DeleteResponse[A <: ReplicatedData] { def this(key: Key[A], consistency: WriteConsistency) = this(key, consistency, None)
def key: Key[A]
def this(key: Key[A], consistency: WriteConsistency, request: Optional[Any]) =
this(key, consistency, Option(request.orElse(null)))
} }
final case class DeleteSuccess[A <: ReplicatedData](key: Key[A]) extends DeleteResponse[A]
final case class ReplicationDeleteFailure[A <: ReplicatedData](key: Key[A]) extends DeleteResponse[A] sealed trait DeleteResponse[A <: ReplicatedData] extends NoSerializationVerificationNeeded {
final case class DataDeleted[A <: ReplicatedData](key: Key[A]) def key: Key[A]
def request: Option[Any]
/** Java API*/
def getRequest: Optional[Any] = Optional.ofNullable(request.orNull)
}
final case class DeleteSuccess[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends DeleteResponse[A]
final case class ReplicationDeleteFailure[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends DeleteResponse[A]
final case class DataDeleted[A <: ReplicatedData](key: Key[A], request: Option[Any])
extends RuntimeException with NoStackTrace with DeleteResponse[A] { extends RuntimeException with NoStackTrace with DeleteResponse[A] {
override def toString: String = s"DataDeleted [$key]" override def toString: String = s"DataDeleted [$key]"
} }
@ -752,7 +776,11 @@ object Replicator {
* A deleted key cannot be reused again, but it is still recommended to delete unused * A deleted key cannot be reused again, but it is still recommended to delete unused
* data entries because that reduces the replication overhead when new nodes join the cluster. * data entries because that reduces the replication overhead when new nodes join the cluster.
* Subsequent `Delete`, `Update` and `Get` requests will be replied with [[Replicator.DataDeleted]]. * Subsequent `Delete`, `Update` and `Get` requests will be replied with [[Replicator.DataDeleted]].
* Subscribers will receive [[Replicator.DataDeleted]]. * Subscribers will receive [[Replicator.Deleted]].
*
* In the `Delete` message you can pass an optional request context in the same way as for the
* `Update` message, described above. For example the original sender can be passed and replied
* to after receiving and transforming `DeleteSuccess`.
* *
* == CRDT Garbage == * == CRDT Garbage ==
* *
@ -951,7 +979,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
case LeaderChanged(leader) receiveLeaderChanged(leader, None) case LeaderChanged(leader) receiveLeaderChanged(leader, None)
case RoleLeaderChanged(role, leader) receiveLeaderChanged(leader, Some(role)) case RoleLeaderChanged(role, leader) receiveLeaderChanged(leader, Some(role))
case GetKeyIds receiveGetKeyIds() case GetKeyIds receiveGetKeyIds()
case Delete(key, consistency) receiveDelete(key, consistency) case Delete(key, consistency, req) receiveDelete(key, consistency, req)
case RemovedNodePruningTick receiveRemovedNodePruningTick() case RemovedNodePruningTick receiveRemovedNodePruningTick()
case GetReplicaCount receiveGetReplicaCount() case GetReplicaCount receiveGetReplicaCount()
} }
@ -961,7 +989,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
log.debug("Received Get for key [{}], local data [{}]", key, localValue) log.debug("Received Get for key [{}], local data [{}]", key, localValue)
if (isLocalGet(consistency)) { if (isLocalGet(consistency)) {
val reply = localValue match { val reply = localValue match {
case Some(DataEnvelope(DeletedData, _)) DataDeleted(key) case Some(DataEnvelope(DeletedData, _)) DataDeleted(key, req)
case Some(DataEnvelope(data, _)) GetSuccess(key, req)(data) case Some(DataEnvelope(data, _)) GetSuccess(key, req)(data)
case None NotFound(key, req) case None NotFound(key, req)
} }
@ -989,7 +1017,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val localValue = getData(key.id) val localValue = getData(key.id)
Try { Try {
localValue match { localValue match {
case Some(DataEnvelope(DeletedData, _)) throw new DataDeleted(key) case Some(DataEnvelope(DeletedData, _)) throw new DataDeleted(key, req)
case Some(envelope @ DataEnvelope(existing, _)) case Some(envelope @ DataEnvelope(existing, _))
existing.merge(modify(Some(existing)).asInstanceOf[existing.T]) existing.merge(modify(Some(existing)).asInstanceOf[existing.T])
case None modify(None) case None modify(None)
@ -1082,27 +1110,27 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
sender() ! GetKeyIdsResult(keys) sender() ! GetKeyIdsResult(keys)
} }
def receiveDelete(key: KeyR, consistency: WriteConsistency): Unit = { def receiveDelete(key: KeyR, consistency: WriteConsistency, req: Option[Any]): Unit = {
getData(key.id) match { getData(key.id) match {
case Some(DataEnvelope(DeletedData, _)) case Some(DataEnvelope(DeletedData, _))
// already deleted // already deleted
sender() ! DataDeleted(key) sender() ! DataDeleted(key, req)
case _ case _
setData(key.id, DeletedEnvelope) setData(key.id, DeletedEnvelope)
val durable = isDurable(key.id) val durable = isDurable(key.id)
if (isLocalUpdate(consistency)) { if (isLocalUpdate(consistency)) {
if (durable) if (durable)
durableStore ! Store(key.id, DeletedData, durableStore ! Store(key.id, DeletedData,
Some(StoreReply(DeleteSuccess(key), StoreFailure(key, None), sender()))) Some(StoreReply(DeleteSuccess(key, req), StoreFailure(key, req), sender())))
else else
sender() ! DeleteSuccess(key) sender() ! DeleteSuccess(key, req)
} else { } else {
val writeAggregator = val writeAggregator =
context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, None, nodes, unreachable, sender(), durable) context.actorOf(WriteAggregator.props(key, DeletedEnvelope, consistency, req, nodes, unreachable, sender(), durable)
.withDispatcher(context.props.dispatcher)) .withDispatcher(context.props.dispatcher))
if (durable) { if (durable) {
durableStore ! Store(key.id, DeletedData, durableStore ! Store(key.id, DeletedData,
Some(StoreReply(DeleteSuccess(key), StoreFailure(key, None), writeAggregator))) Some(StoreReply(DeleteSuccess(key, req), StoreFailure(key, req), writeAggregator)))
} }
} }
} }
@ -1147,7 +1175,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val key = subscriptionKeys(keyId) val key = subscriptionKeys(keyId)
getData(keyId) match { getData(keyId) match {
case Some(envelope) case Some(envelope)
val msg = if (envelope.data == DeletedData) DataDeleted(key) else Changed(key)(envelope.data) val msg = if (envelope.data == DeletedData) Deleted(key) else Changed(key)(envelope.data)
subs.foreach { _ ! msg } subs.foreach { _ ! msg }
case None case None
} }
@ -1600,9 +1628,9 @@ private[akka] class WriteAggregator(
val isTimeoutOrNotEnoughNodes = isTimeout || notEnoughNodes || gotWriteNackFrom.isEmpty val isTimeoutOrNotEnoughNodes = isTimeout || notEnoughNodes || gotWriteNackFrom.isEmpty
val replyMsg = val replyMsg =
if (isSuccess && isDelete) DeleteSuccess(key) if (isSuccess && isDelete) DeleteSuccess(key, req)
else if (isSuccess) UpdateSuccess(key, req) else if (isSuccess) UpdateSuccess(key, req)
else if (isTimeoutOrNotEnoughNodes && isDelete) ReplicationDeleteFailure(key) else if (isTimeoutOrNotEnoughNodes && isDelete) ReplicationDeleteFailure(key, req)
else if (isTimeoutOrNotEnoughNodes) UpdateTimeout(key, req) else if (isTimeoutOrNotEnoughNodes) UpdateTimeout(key, req)
else StoreFailure(key, req) else StoreFailure(key, req)
@ -1702,7 +1730,7 @@ private[akka] class ReadAggregator(
def waitReadRepairAck(envelope: Replicator.Internal.DataEnvelope): Receive = { def waitReadRepairAck(envelope: Replicator.Internal.DataEnvelope): Receive = {
case ReadRepairAck case ReadRepairAck
val replyMsg = val replyMsg =
if (envelope.data == DeletedData) DataDeleted(key) if (envelope.data == DeletedData) DataDeleted(key, req)
else GetSuccess(key, req)(envelope.data) else GetSuccess(key, req)(envelope.data)
replyTo.tell(replyMsg, context.parent) replyTo.tell(replyMsg, context.parent)
context.stop(self) context.stop(self)

View file

@ -82,7 +82,7 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
within(5.seconds) { within(5.seconds) {
awaitAssert { awaitAssert {
replicator ! Get(key, ReadLocal) replicator ! Get(key, ReadLocal)
expectMsg(DataDeleted(key)) expectMsg(DataDeleted(key, None))
} }
} }
@ -141,7 +141,7 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ + 50) replicator ! Update(KeyX, GCounter(), WriteTo(2, timeout))(_ + 50)
expectMsg(UpdateSuccess(KeyX, None)) expectMsg(UpdateSuccess(KeyX, None))
replicator ! Delete(KeyX, WriteLocal) replicator ! Delete(KeyX, WriteLocal)
expectMsg(DeleteSuccess(KeyX)) expectMsg(DeleteSuccess(KeyX, None))
} }
enterBarrier("initial-updates-done") enterBarrier("initial-updates-done")

View file

@ -136,17 +136,17 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 9) replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 9)
expectMsg(UpdateSuccess(KeyX, None)) expectMsg(UpdateSuccess(KeyX, None))
changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9) changedProbe.expectMsg(Changed(KeyX)(c9)).dataValue should be(c9)
replicator ! Delete(KeyX, WriteLocal) replicator ! Delete(KeyX, WriteLocal, Some(777))
expectMsg(DeleteSuccess(KeyX)) expectMsg(DeleteSuccess(KeyX, Some(777)))
changedProbe.expectMsg(DataDeleted(KeyX)) changedProbe.expectMsg(Deleted(KeyX))
replicator ! Get(KeyX, ReadLocal) replicator ! Get(KeyX, ReadLocal, Some(789))
expectMsg(DataDeleted(KeyX)) expectMsg(DataDeleted(KeyX, Some(789)))
replicator ! Get(KeyX, readAll) replicator ! Get(KeyX, readAll, Some(456))
expectMsg(DataDeleted(KeyX)) expectMsg(DataDeleted(KeyX, Some(456)))
replicator ! Update(KeyX, GCounter(), WriteLocal)(_ + 1) replicator ! Update(KeyX, GCounter(), WriteLocal, Some(123))(_ + 1)
expectMsg(DataDeleted(KeyX)) expectMsg(DataDeleted(KeyX, Some(123)))
replicator ! Delete(KeyX, WriteLocal) replicator ! Delete(KeyX, WriteLocal, Some(555))
expectMsg(DataDeleted(KeyX)) expectMsg(DataDeleted(KeyX, Some(555)))
replicator ! GetKeyIds replicator ! GetKeyIds
expectMsg(GetKeyIdsResult(Set("A"))) expectMsg(GetKeyIdsResult(Set("A")))
@ -288,8 +288,8 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
expectMsg(UpdateSuccess(KeyC, None)) expectMsg(UpdateSuccess(KeyC, None))
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31) changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)
replicator ! Delete(KeyY, WriteLocal) replicator ! Delete(KeyY, WriteLocal, Some(777))
expectMsg(DeleteSuccess(KeyY)) expectMsg(DeleteSuccess(KeyY, Some(777)))
replicator ! Get(KeyZ, readMajority) replicator ! Get(KeyZ, readMajority)
expectMsgPF() { case g @ GetSuccess(KeyZ, _) g.get(KeyZ).value } should be(30) expectMsgPF() { case g @ GetSuccess(KeyZ, _) g.get(KeyZ).value } should be(30)
@ -304,8 +304,8 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) } val c = expectMsgPF() { case g @ GetSuccess(KeyC, _) g.get(KeyC) }
c.value should be(31) c.value should be(31)
replicator ! Get(KeyY, ReadLocal) replicator ! Get(KeyY, ReadLocal, Some(777))
expectMsg(DataDeleted(KeyY)) expectMsg(DataDeleted(KeyY, Some(777)))
} }
} }
changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31) changedProbe.expectMsgPF() { case c @ Changed(KeyC) c.get(KeyC).value } should be(31)

View file

@ -184,3 +184,9 @@ Java developers should use `<>` instead of `[]`, e.g: `PNCounterMap<String>`.
**NOTE: Even though the interface is not compatible between 2.4 and 2.5, the binary protocol over the wire is (as long **NOTE: Even though the interface is not compatible between 2.4 and 2.5, the binary protocol over the wire is (as long
as you use String as key type). This means that 2.4 nodes can synchronize with 2.5 nodes.** as you use String as key type). This means that 2.4 nodes can synchronize with 2.5 nodes.**
Subscribers
-----------
When an entity is removed subscribers will not receive ``Replicator.DataDeleted`` any more.
They will receive ``Replicator.Deleted`` instead.

View file

@ -244,7 +244,11 @@ to all nodes.
A deleted key cannot be reused again, but it is still recommended to delete unused A deleted key cannot be reused again, but it is still recommended to delete unused
data entries because that reduces the replication overhead when new nodes join the cluster. data entries because that reduces the replication overhead when new nodes join the cluster.
Subsequent ``Delete``, ``Update`` and ``Get`` requests will be replied with ``Replicator.DataDeleted``. Subsequent ``Delete``, ``Update`` and ``Get`` requests will be replied with ``Replicator.DataDeleted``.
Subscribers will receive ``Replicator.DataDeleted``. Subscribers will receive ``Replicator.Deleted``.
In the `Delete` message you can pass an optional request context in the same way as for the
`Update` message, described above. For example the original sender can be passed and replied
to after receiving and transforming `DeleteSuccess`.
.. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#delete .. includecode:: code/docs/ddata/DistributedDataDocSpec.scala#delete

View file

@ -163,8 +163,24 @@ object MiMa extends AutoPlugin {
FilterAnyProblemStartingWith("akka.cluster.ddata.ORMap"), FilterAnyProblemStartingWith("akka.cluster.ddata.ORMap"),
FilterAnyProblemStartingWith("akka.cluster.ddata.LWWMap"), FilterAnyProblemStartingWith("akka.cluster.ddata.LWWMap"),
FilterAnyProblemStartingWith("akka.cluster.ddata.PNCounterMap"), FilterAnyProblemStartingWith("akka.cluster.ddata.PNCounterMap"),
FilterAnyProblemStartingWith("akka.cluster.ddata.ORMultiMap") FilterAnyProblemStartingWith("akka.cluster.ddata.ORMultiMap"),
// #20140 durable distributed data
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#ReplicationDeleteFailure.apply"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DeleteSuccess.apply"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.Replicator#DeleteResponse.getRequest"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.Replicator#DeleteResponse.request"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.cluster.ddata.Replicator#Command.request"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator.receiveDelete"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#ReplicationDeleteFailure.copy"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#ReplicationDeleteFailure.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DeleteSuccess.copy"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DeleteSuccess.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#Delete.apply"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DataDeleted.apply"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DataDeleted.copy"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#DataDeleted.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.ddata.Replicator#Delete.copy")
) )
Map( Map(