Updated DistributedData Delete API to include optional request context.
#20140
This commit is contained in:
parent
4a9c753710
commit
65cba329d0
6 changed files with 95 additions and 41 deletions
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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.
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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(
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue