Merge pull request #27511 from akka/wip-27371-DataDeleted-patriknw
API changes related to DataDeleted, #27371
This commit is contained in:
commit
89eab3541c
11 changed files with 228 additions and 97 deletions
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue