Merge pull request #27511 from akka/wip-27371-DataDeleted-patriknw

API changes related to DataDeleted, #27371
This commit is contained in:
Patrik Nordwall 2019-09-11 16:28:17 +02:00 committed by GitHub
commit 89eab3541c
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
11 changed files with 228 additions and 97 deletions

View file

@ -540,6 +540,11 @@ object Replicator {
extends GetResponse[A]
with ReplicatorMessage
/**
* The [[Get]] request couldn't be performed because the entry has been deleted.
*/
final case class GetDataDeleted[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends GetResponse[A]
/**
* Register a subscriber that will be notified with a [[Changed]] message
* when the value of the given `key` is changed. Current value is also
@ -563,12 +568,21 @@ object Replicator {
*/
final case class Unsubscribe[A <: ReplicatedData](key: Key[A], subscriber: ActorRef) extends ReplicatorMessage
/**
* @see [[Replicator.Subscribe]]
*/
sealed trait SubscribeResponse[A <: ReplicatedData] extends NoSerializationVerificationNeeded {
def key: Key[A]
}
/**
* The data value is retrieved with [[#get]] using the typed key.
*
* @see [[Replicator.Subscribe]]
*/
final case class Changed[A <: ReplicatedData](key: Key[A])(data: A) extends ReplicatorMessage {
final case class Changed[A <: ReplicatedData](key: Key[A])(data: A)
extends SubscribeResponse[A]
with ReplicatorMessage {
/**
* The data value, with correct type.
@ -585,9 +599,10 @@ object Replicator {
def dataValue: A = data
}
final case class Deleted[A <: ReplicatedData](key: Key[A]) extends NoSerializationVerificationNeeded {
override def toString: String = s"Deleted [$key]"
}
/**
* @see [[Replicator.Subscribe]]
*/
final case class Deleted[A <: ReplicatedData](key: Key[A]) extends SubscribeResponse[A]
object Update {
@ -691,6 +706,11 @@ object Replicator {
*/
final case class UpdateTimeout[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends UpdateFailure[A]
/**
* The [[Update]] couldn't be performed because the entry has been deleted.
*/
final case class UpdateDataDeleted[A <: ReplicatedData](key: Key[A], request: Option[Any]) extends UpdateResponse[A]
/**
* If the `modify` function of the [[Update]] throws an exception the reply message
* will be this `ModifyFailure` message. The original exception is included as `cause`.
@ -1172,7 +1192,8 @@ object Replicator {
*
* 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.
* Subsequent `Delete`, `Update` and `Get` requests will be replied with [[Replicator.DataDeleted]].
* Subsequent `Delete`, `Update` and `Get` requests will be replied with [[Replicator.DataDeleted]],
* [[Replicator.UpdateDataDeleted]] and [[Replicator.GetDataDeleted]] respectively.
* Subscribers will receive [[Replicator.Deleted]].
*
* In the `Delete` message you can pass an optional request context in the same way as for the
@ -1515,10 +1536,10 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def receiveGet(key: KeyR, consistency: ReadConsistency, req: Option[Any]): Unit = {
val localValue = getData(key.id)
log.debug("Received Get for key [{}]", key)
log.debug("Received Get for key [{}].", key)
if (isLocalGet(consistency)) {
val reply = localValue match {
case Some(DataEnvelope(DeletedData, _, _)) => DataDeleted(key, req)
case Some(DataEnvelope(DeletedData, _, _)) => GetDataDeleted(key, req)
case Some(DataEnvelope(data, _, _)) => GetSuccess(key, req)(data)
case None => NotFound(key, req)
}
@ -1559,7 +1580,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
Try {
localValue match {
case Some(DataEnvelope(DeletedData, _, _)) => throw new DataDeleted(key, req)
case Some(envelope @ DataEnvelope(DeletedData, _, _)) =>
(envelope, None)
case Some(envelope @ DataEnvelope(existing, _, _)) =>
modify(Some(existing)) match {
case d: DeltaReplicatedData if deltaCrdtEnabled =>
@ -1575,8 +1597,11 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
}
}
} match {
case Success((DataEnvelope(DeletedData, _, _), _)) =>
log.debug("Received Update for deleted key [{}].", key)
replyTo ! UpdateDataDeleted(key, req)
case Success((envelope, delta)) =>
log.debug("Received Update for key [{}]", key)
log.debug("Received Update for key [{}].", key)
// handle the delta
delta match {
@ -1628,9 +1653,6 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
Some(StoreReply(UpdateSuccess(key, req), StoreFailure(key, req), writeAggregator)))
}
}
case Failure(e: DataDeleted[_]) =>
log.debug("Received Update for deleted key [{}]", key)
replyTo ! e
case Failure(e) =>
log.debug("Received Update for key [{}], failed: {}", key, e.getMessage)
replyTo ! ModifyFailure(key, "Update failed: " + e.getMessage, e, req)
@ -1859,7 +1881,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
val isDebugEnabled = log.isDebugEnabled
if (isDebugEnabled)
log.debug(
"Received DeltaPropagation from [{}], containing [{}]",
"Received DeltaPropagation from [{}], containing [{}].",
fromNode.address,
deltas.collect { case (key, Delta(_, fromSeqNr, toSeqNr)) => s"$key $fromSeqNr-$toSeqNr" }.mkString(", "))
@ -1960,7 +1982,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def receiveStatus(otherDigests: Map[KeyId, Digest], chunk: Int, totChunks: Int, fromSystemUid: Option[Long]): Unit = {
if (log.isDebugEnabled)
log.debug(
"Received gossip status from [{}], chunk [{}] of [{}] containing [{}]",
"Received gossip status from [{}], chunk [{}] of [{}] containing [{}].",
replyTo.path.address,
(chunk + 1),
totChunks,
@ -2008,7 +2030,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def receiveGossip(updatedData: Map[KeyId, DataEnvelope], sendBack: Boolean, fromSystemUid: Option[Long]): Unit = {
if (log.isDebugEnabled)
log.debug("Received gossip from [{}], containing [{}]", replyTo.path.address, updatedData.keys.mkString(", "))
log.debug("Received gossip from [{}], containing [{}].", replyTo.path.address, updatedData.keys.mkString(", "))
var replyData = Map.empty[KeyId, DataEnvelope]
updatedData.foreach {
case (key, envelope) =>
@ -2545,7 +2567,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def waitReadRepairAck(envelope: Replicator.Internal.DataEnvelope): Receive = {
case ReadRepairAck =>
val replyMsg =
if (envelope.data == DeletedData) DataDeleted(key, req)
if (envelope.data == DeletedData) GetDataDeleted(key, req)
else GetSuccess(key, req)(envelope.data)
replyTo.tell(replyMsg, context.parent)
context.stop(self)

View file

@ -86,7 +86,7 @@ class ReplicatorChaosSpec extends MultiNodeSpec(ReplicatorChaosSpec) with STMult
within(5.seconds) {
awaitAssert {
replicator ! Get(key, ReadLocal)
expectMsg(DataDeleted(key, None))
expectMsg(GetDataDeleted(key, None))
}
}

View file

@ -4,14 +4,17 @@
package akka.cluster.ddata
import scala.concurrent.Await
import scala.concurrent.duration._
import akka.pattern.ask
import akka.cluster.Cluster
import akka.remote.testconductor.RoleName
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
object ReplicatorSpec extends MultiNodeConfig {
@ -74,6 +77,8 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
enterBarrier("after-" + afterCounter)
}
private implicit val askTimeout: Timeout = 5.seconds
def join(from: RoleName, to: RoleName): Unit = {
runOn(from) {
cluster.join(node(to).address)
@ -144,13 +149,24 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
expectMsg(DeleteSuccess(KeyX, Some(777)))
changedProbe.expectMsg(Deleted(KeyX))
replicator ! Get(KeyX, ReadLocal, Some(789))
expectMsg(DataDeleted(KeyX, Some(789)))
expectMsg(GetDataDeleted(KeyX, Some(789)))
replicator ! Get(KeyX, readAll, Some(456))
expectMsg(DataDeleted(KeyX, Some(456)))
expectMsg(GetDataDeleted(KeyX, Some(456)))
// verify ask-mapTo for GetDataDeleted, issue #27371
Await
.result((replicator ? Get(KeyX, ReadLocal, None)).mapTo[GetResponse[GCounter]], askTimeout.duration)
.getClass should be(classOf[GetDataDeleted[GCounter]])
replicator ! Update(KeyX, GCounter(), WriteLocal, Some(123))(_ :+ 1)
expectMsg(DataDeleted(KeyX, Some(123)))
expectMsg(UpdateDataDeleted(KeyX, Some(123)))
replicator ! Delete(KeyX, WriteLocal, Some(555))
expectMsg(DataDeleted(KeyX, Some(555)))
// verify ask-mapTo for UpdateDataDeleted, issue #27371
Await
.result(
(replicator ? Update(KeyX, GCounter(), WriteLocal, Some(123))(_ :+ 1)).mapTo[UpdateResponse[GCounter]],
askTimeout.duration)
.getClass should be(classOf[UpdateDataDeleted[GCounter]])
replicator ! GetKeyIds
expectMsg(GetKeyIdsResult(Set("A")))
@ -309,7 +325,7 @@ class ReplicatorSpec extends MultiNodeSpec(ReplicatorSpec) with STMultiNodeSpec
c.value should be(31)
replicator ! Get(KeyY, ReadLocal, Some(777))
expectMsg(DataDeleted(KeyY, Some(777)))
expectMsg(GetDataDeleted(KeyY, Some(777)))
}
}
changedProbe.expectMsgPF() { case c @ Changed(KeyC) => c.get(KeyC).value } should be(31)