akka-distributed-data compiler warnings #26088
This commit is contained in:
parent
4a04eddf7d
commit
d7f12d3568
24 changed files with 138 additions and 72 deletions
|
|
@ -482,7 +482,7 @@ final class ORMap[A, B <: ReplicatedData] private[akka] (
|
|||
case ORMap.DeltaGroup(ops) =>
|
||||
ops.foreach {
|
||||
processDelta.orElse {
|
||||
case ORMap.DeltaGroup(args) =>
|
||||
case ORMap.DeltaGroup(_) =>
|
||||
throw new IllegalStateException("Cannot nest DeltaGroups")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -181,7 +181,7 @@ final class ORMultiMap[A, B] private[akka] (
|
|||
*/
|
||||
@InternalApi private[akka] def put(node: UniqueAddress, key: A, value: Set[B]): ORMultiMap[A, B] = {
|
||||
val newUnderlying = underlying.updated(node, key, ORSet.empty[B], valueDeltas = withValueDeltas) { existing =>
|
||||
value.foldLeft(existing.clear(node)) { (s, element) =>
|
||||
value.foldLeft(existing.clear()) { (s, element) =>
|
||||
s.add(node, element)
|
||||
}
|
||||
}
|
||||
|
|
@ -216,7 +216,7 @@ final class ORMultiMap[A, B] private[akka] (
|
|||
@InternalApi private[akka] def remove(node: UniqueAddress, key: A): ORMultiMap[A, B] = {
|
||||
if (withValueDeltas) {
|
||||
val u = underlying.updated(node, key, ORSet.empty[B], valueDeltas = true) { existing =>
|
||||
existing.clear(node)
|
||||
existing.clear()
|
||||
}
|
||||
new ORMultiMap(u.removeKey(node, key), withValueDeltas)
|
||||
} else {
|
||||
|
|
|
|||
|
|
@ -6,10 +6,9 @@ package akka.cluster.ddata
|
|||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.UniqueAddress
|
||||
import akka.util.HashCode
|
||||
import akka.util.{ unused, HashCode }
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
object ORSet {
|
||||
|
|
@ -392,15 +391,15 @@ final class ORSet[A] private[akka] (
|
|||
* This has the same result as using [[#remove]] for each
|
||||
* element, but it is more efficient.
|
||||
*/
|
||||
def clear(node: SelfUniqueAddress): ORSet[A] = clear(node.uniqueAddress)
|
||||
def clear(@unused node: SelfUniqueAddress): ORSet[A] = clear()
|
||||
|
||||
@deprecated("Use `remove` that takes a `SelfUniqueAddress` parameter instead.", since = "2.5.20")
|
||||
def clear(node: Cluster): ORSet[A] = clear(node.selfUniqueAddress)
|
||||
def clear(@unused node: Cluster): ORSet[A] = clear()
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@InternalApi private[akka] def clear(node: UniqueAddress): ORSet[A] = {
|
||||
@InternalApi private[akka] def clear(): ORSet[A] = {
|
||||
val newFullState = new ORSet[A](elementsMap = Map.empty, vvector)
|
||||
val clearOp = ORSet.FullStateDeltaOp(newFullState)
|
||||
val newDelta = delta match {
|
||||
|
|
@ -461,7 +460,7 @@ final class ORSet[A] private[akka] (
|
|||
case (acc, op: ORSet.AddDeltaOp[A]) => acc.dryMerge(op.underlying, addDeltaOp = true)
|
||||
case (acc, op: ORSet.RemoveDeltaOp[A]) => acc.mergeRemoveDelta(op)
|
||||
case (acc, op: ORSet.FullStateDeltaOp[A]) => acc.dryMerge(op.underlying, addDeltaOp = false)
|
||||
case (acc, op: ORSet.DeltaGroup[A]) =>
|
||||
case (_, _: ORSet.DeltaGroup[A]) =>
|
||||
throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested")
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import akka.actor.Address
|
|||
import akka.cluster.Member
|
||||
import akka.cluster.UniqueAddress
|
||||
import akka.annotation.InternalApi
|
||||
import akka.util.unused
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -21,6 +22,7 @@ import akka.annotation.InternalApi
|
|||
}
|
||||
final case class PruningPerformed(obsoleteTime: Long) extends PruningState {
|
||||
def isObsolete(currentTime: Long): Boolean = obsoleteTime <= currentTime
|
||||
def addSeen(@unused node: Address): PruningState = this
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -44,5 +46,5 @@ import akka.annotation.InternalApi
|
|||
this
|
||||
}
|
||||
|
||||
def addSeen(node: Address): PruningState = this
|
||||
def addSeen(node: Address): PruningState
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1569,7 +1569,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
getData(key) match {
|
||||
case someEnvelope @ Some(envelope) if envelope eq writeEnvelope => someEnvelope
|
||||
case Some(DataEnvelope(DeletedData, _, _)) => Some(DeletedEnvelope) // already deleted
|
||||
case Some(envelope @ DataEnvelope(existing, _, _)) =>
|
||||
case Some(envelope @ DataEnvelope(existingData @ _, _, _)) =>
|
||||
try {
|
||||
// DataEnvelope will mergeDelta when needed
|
||||
val merged = envelope.merge(writeEnvelope).addSeen(selfAddress)
|
||||
|
|
@ -1918,8 +1918,8 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def hasSubscriber(subscriber: ActorRef): Boolean =
|
||||
(subscribers.exists { case (k, s) => s.contains(subscriber) }) ||
|
||||
(newSubscribers.exists { case (k, s) => s.contains(subscriber) })
|
||||
subscribers.exists { case (_, s) => s.contains(subscriber) } ||
|
||||
newSubscribers.exists { case (_, s) => s.contains(subscriber) }
|
||||
|
||||
def receiveTerminated(ref: ActorRef): Unit = {
|
||||
if (ref == durableStore) {
|
||||
|
|
@ -2006,7 +2006,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
val knownNodes = nodes.union(weaklyUpNodes).union(removedNodes.keySet.map(_.address))
|
||||
val newRemovedNodes =
|
||||
dataEntries.foldLeft(Set.empty[UniqueAddress]) {
|
||||
case (acc, (_, (envelope @ DataEnvelope(data: RemovedNodePruning, _, _), _))) =>
|
||||
case (acc, (_, (DataEnvelope(data: RemovedNodePruning, _, _), _))) =>
|
||||
acc.union(data.modifiedByNodes.filterNot(n => n == selfUniqueAddress || knownNodes(n.address)))
|
||||
case (acc, _) =>
|
||||
acc
|
||||
|
|
@ -2037,7 +2037,7 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
if (envelope.needPruningFrom(removed)) {
|
||||
envelope.data match {
|
||||
case dataWithRemovedNodePruning: RemovedNodePruning =>
|
||||
case _: RemovedNodePruning =>
|
||||
envelope.pruning.get(removed) match {
|
||||
case None => init()
|
||||
case Some(PruningInitialized(owner, _)) if owner != selfUniqueAddress => init()
|
||||
|
|
@ -2256,19 +2256,17 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
case _: Replicator.UpdateSuccess[_] =>
|
||||
gotLocalStoreReply = true
|
||||
if (isDone) reply(isTimeout = false)
|
||||
case f: Replicator.StoreFailure[_] =>
|
||||
case _: Replicator.StoreFailure[_] =>
|
||||
gotLocalStoreReply = true
|
||||
gotWriteNackFrom += selfUniqueAddress.address
|
||||
if (isDone) reply(isTimeout = false)
|
||||
|
||||
case SendToSecondary =>
|
||||
deltaMsg match {
|
||||
case None =>
|
||||
case Some(d) =>
|
||||
// Deltas must be applied in order and we can't keep track of ordering of
|
||||
// simultaneous updates so there is a chance that the delta could not be applied.
|
||||
// Try again with the full state to the primary nodes that have not acked.
|
||||
primaryNodes.toSet.intersect(remaining).foreach { replica(_) ! writeMsg }
|
||||
if (deltaMsg.isDefined) {
|
||||
// Deltas must be applied in order and we can't keep track of ordering of
|
||||
// simultaneous updates so there is a chance that the delta could not be applied.
|
||||
// Try again with the full state to the primary nodes that have not acked.
|
||||
primaryNodes.toSet.intersect(remaining).foreach { replica(_) ! writeMsg }
|
||||
}
|
||||
secondaryNodes.foreach { replica(_) ! writeMsg }
|
||||
case ReceiveTimeout =>
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ private object ReplicatedDataSerializer {
|
|||
abstract class KeyComparator[A <: GeneratedMessage] extends Comparator[A] {
|
||||
|
||||
/**
|
||||
* Get the key from the entry. The key may be a String, Integer, Long, or Any
|
||||
* Get the key from the entry. The key may be a String, Int, Long, or OtherMessage
|
||||
* @param entry The protobuf entry used with Map types
|
||||
* @return The Key
|
||||
*/
|
||||
|
|
@ -54,6 +54,9 @@ private object ReplicatedDataSerializer {
|
|||
case (k1: Long, k2) => -1
|
||||
case (k1, k2: Long) => 1
|
||||
case (k1: OtherMessage, k2: OtherMessage) => OtherMessageComparator.compare(k1, k2)
|
||||
case (k1, k2) =>
|
||||
throw new IllegalStateException(
|
||||
s"Invalid keys (${k1.getClass}, ${k2.getClass}): must be of type String, Int, Long or OtherMessage")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -541,7 +544,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
b.addEntries(createEntry(rd.ORSetDeltaOp.Remove, u))
|
||||
case ORSet.FullStateDeltaOp(u) =>
|
||||
b.addEntries(createEntry(rd.ORSetDeltaOp.Full, u))
|
||||
case ORSet.DeltaGroup(u) =>
|
||||
case ORSet.DeltaGroup(_) =>
|
||||
throw new IllegalArgumentException("ORSet.DeltaGroup should not be nested")
|
||||
}
|
||||
b.build()
|
||||
|
|
@ -899,7 +902,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
case ORMap.UpdateDeltaOp(op, m, zt) =>
|
||||
b.addEntries(
|
||||
createEntry(rd.ORMapDeltaOp.ORMapUpdate, op.asInstanceOf[ORSet.AddDeltaOp[_]].underlying, m, zt.value))
|
||||
case ORMap.DeltaGroup(u) =>
|
||||
case ORMap.DeltaGroup(_) =>
|
||||
throw new IllegalArgumentException("ORMap.DeltaGroup should not be nested")
|
||||
}
|
||||
b.build()
|
||||
|
|
|
|||
|
|
@ -261,7 +261,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
private def statusToProto(status: Status): dm.Status = {
|
||||
val b = dm.Status.newBuilder()
|
||||
b.setChunk(status.chunk).setTotChunks(status.totChunks)
|
||||
val entries = status.digests.foreach {
|
||||
status.digests.foreach {
|
||||
case (key, digest) =>
|
||||
b.addEntries(dm.Status.Entry.newBuilder().setKey(key).setDigest(ByteString.copyFrom(digest.toArray)))
|
||||
}
|
||||
|
|
@ -278,7 +278,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
|
||||
private def gossipToProto(gossip: Gossip): dm.Gossip = {
|
||||
val b = dm.Gossip.newBuilder().setSendBack(gossip.sendBack)
|
||||
val entries = gossip.updatedData.foreach {
|
||||
gossip.updatedData.foreach {
|
||||
case (key, data) =>
|
||||
b.addEntries(dm.Gossip.Entry.newBuilder().setKey(key).setEnvelope(dataEnvelopeToProto(data)))
|
||||
}
|
||||
|
|
@ -296,7 +296,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
val b = dm.DeltaPropagation.newBuilder().setFromNode(uniqueAddressToProto(deltaPropagation.fromNode))
|
||||
if (deltaPropagation.reply)
|
||||
b.setReply(deltaPropagation.reply)
|
||||
val entries = deltaPropagation.deltas.foreach {
|
||||
deltaPropagation.deltas.foreach {
|
||||
case (key, Delta(data, fromSeqNr, toSeqNr)) =>
|
||||
val b2 = dm.DeltaPropagation.Entry
|
||||
.newBuilder()
|
||||
|
|
@ -331,11 +331,14 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
case _: ReadAll => -1
|
||||
}
|
||||
|
||||
val timoutInMillis = get.consistency.timeout.toMillis
|
||||
require(timoutInMillis <= 0XFFFFFFFFL, "Timeouts must fit in a 32-bit unsigned int")
|
||||
|
||||
val b = dm.Get
|
||||
.newBuilder()
|
||||
.setKey(otherMessageToProto(get.key))
|
||||
.setConsistency(consistencyValue)
|
||||
.setTimeout(get.consistency.timeout.toMillis.toInt)
|
||||
.setTimeout(timoutInMillis.toInt)
|
||||
|
||||
get.request.foreach(o => b.setRequest(otherMessageToProto(o)))
|
||||
b.build()
|
||||
|
|
@ -345,7 +348,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
val get = dm.Get.parseFrom(bytes)
|
||||
val key = otherMessageFromProto(get.getKey).asInstanceOf[KeyR]
|
||||
val request = if (get.hasRequest()) Some(otherMessageFromProto(get.getRequest)) else None
|
||||
val timeout = Duration(get.getTimeout, TimeUnit.MILLISECONDS)
|
||||
// 32-bit unsigned protobuf integers are mapped to
|
||||
// 32-bit signed Java ints, using the leftmost bit as sign.
|
||||
val timeout =
|
||||
if (get.getTimeout < 0) Duration(Int.MaxValue.toLong + (get.getTimeout - Int.MaxValue), TimeUnit.MILLISECONDS)
|
||||
else Duration(get.getTimeout.toLong, TimeUnit.MILLISECONDS)
|
||||
val consistency = get.getConsistency match {
|
||||
case 0 => ReadMajority(timeout)
|
||||
case -1 => ReadAll(timeout)
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue