diff --git a/actor/src/main/scala/org/apache/pekko/actor/LightArrayRevolverScheduler.scala b/actor/src/main/scala/org/apache/pekko/actor/LightArrayRevolverScheduler.scala index b012cf979d2..0e46fa02efe 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/LightArrayRevolverScheduler.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/LightArrayRevolverScheduler.scala @@ -356,7 +356,8 @@ class LightArrayRevolverScheduler(config: Config, log: LoggingAdapter, threadFac object LightArrayRevolverScheduler { @nowarn("msg=deprecated") - private[this] val taskOffset = unsafe.objectFieldOffset(classOf[TaskHolder].getDeclaredField("task")) + private[this] val taskOffset = + unsafe.objectFieldOffset(classOf[TaskHolder].getDeclaredField("task")): @nowarn("cat=deprecation") private class TaskQueue extends AbstractNodeQueue[TaskHolder] @@ -375,7 +376,8 @@ object LightArrayRevolverScheduler { private final def extractTask(replaceWith: Runnable): Runnable = task match { case t @ (ExecutedTask | CancelledTask) => t - case x => if (unsafe.compareAndSwapObject(this, taskOffset, x, replaceWith)) x else extractTask(replaceWith) + case x => if (unsafe.compareAndSwapObject(this, taskOffset, x, replaceWith): @nowarn("cat=deprecation")) x + else extractTask(replaceWith) } private[pekko] final def executeTask(): Boolean = extractTask(ExecutedTask) match { diff --git a/actor/src/main/scala/org/apache/pekko/actor/RepointableActorRef.scala b/actor/src/main/scala/org/apache/pekko/actor/RepointableActorRef.scala index bcd8eb8d0a1..e012fa78ab5 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/RepointableActorRef.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/RepointableActorRef.scala @@ -66,17 +66,21 @@ private[pekko] class RepointableActorRef( _lookupDoNotCallMeDirectly } - def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell] - def lookup = Unsafe.instance.getObjectVolatile(this, lookupOffset).asInstanceOf[Cell] + def underlying: Cell = + Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell]: @nowarn("cat=deprecation") + def lookup = Unsafe.instance.getObjectVolatile(this, lookupOffset).asInstanceOf[Cell]: @nowarn("cat=deprecation") - @tailrec final def swapCell(next: Cell): Cell = { + @tailrec + final def swapCell(next: Cell): Cell = { val old = underlying - if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next)) old else swapCell(next) + if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next): @nowarn("cat=deprecation")) old + else swapCell(next) } @tailrec final def swapLookup(next: Cell): Cell = { val old = lookup - if (Unsafe.instance.compareAndSwapObject(this, lookupOffset, old, next)) old else swapLookup(next) + if (Unsafe.instance.compareAndSwapObject(this, lookupOffset, old, next): @nowarn("cat=deprecation")) old + else swapLookup(next) } /** diff --git a/actor/src/main/scala/org/apache/pekko/actor/dungeon/Children.scala b/actor/src/main/scala/org/apache/pekko/actor/dungeon/Children.scala index e302ffbc13a..d7d1f5f3628 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/dungeon/Children.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/dungeon/Children.scala @@ -38,7 +38,8 @@ private[pekko] trait Children { this: ActorCell => private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer def childrenRefs: ChildrenContainer = - Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer] + Unsafe.instance.getObjectVolatile(this, AbstractActorCell.childrenOffset).asInstanceOf[ChildrenContainer]: @nowarn( + "cat=deprecation") final def children: immutable.Iterable[ActorRef] = childrenRefs.children @nowarn("msg=deprecated") @@ -65,7 +66,8 @@ private[pekko] trait Children { this: ActorCell => @nowarn @volatile private var _functionRefsDoNotCallMeDirectly = Map.empty[String, FunctionRef] private def functionRefs: Map[String, FunctionRef] = - Unsafe.instance.getObjectVolatile(this, AbstractActorCell.functionRefsOffset).asInstanceOf[Map[String, FunctionRef]] + Unsafe.instance.getObjectVolatile(this, AbstractActorCell.functionRefsOffset).asInstanceOf[Map[String, + FunctionRef]]: @nowarn("cat=deprecation") private[pekko] def getFunctionRefOrNobody(name: String, uid: Int = ActorCell.undefinedUid): InternalActorRef = functionRefs.getOrElse(name, Children.GetNobody()) match { @@ -84,7 +86,8 @@ private[pekko] trait Children { this: ActorCell => @tailrec def rec(): Unit = { val old = functionRefs val added = old.updated(childPath.name, ref) - if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, added)) rec() + if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, added): @nowarn( + "cat=deprecation")) rec() } rec() @@ -99,7 +102,8 @@ private[pekko] trait Children { this: ActorCell => if (!old.contains(name)) false else { val removed = old - name - if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, removed)) rec() + if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.functionRefsOffset, old, removed): @nowarn( + "cat=deprecation")) rec() else { ref.stop() true @@ -112,17 +116,17 @@ private[pekko] trait Children { this: ActorCell => protected def stopFunctionRefs(): Unit = { val refs = Unsafe.instance .getAndSetObject(this, AbstractActorCell.functionRefsOffset, Map.empty) - .asInstanceOf[Map[String, FunctionRef]] + .asInstanceOf[Map[String, FunctionRef]]: @nowarn("cat=deprecation") refs.valuesIterator.foreach(_.stop()) } @nowarn @volatile private var _nextNameDoNotCallMeDirectly = 0L final protected def randomName(sb: java.lang.StringBuilder): String = { - val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1) + val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1): @nowarn("cat=deprecation") Helpers.base64(num, sb) } final protected def randomName(): String = { - val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1) + val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1): @nowarn("cat=deprecation") Helpers.base64(num) } @@ -151,7 +155,8 @@ private[pekko] trait Children { this: ActorCell => * low level CAS helpers */ private final def swapChildrenRefs(oldChildren: ChildrenContainer, newChildren: ChildrenContainer): Boolean = - Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.childrenOffset, oldChildren, newChildren) + Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.childrenOffset, oldChildren, newChildren): @nowarn( + "cat=deprecation") @tailrec final def reserveChild(name: String): Boolean = { val c = childrenRefs @@ -184,7 +189,8 @@ private[pekko] trait Children { this: ActorCell => } final protected def setTerminated(): Unit = - Unsafe.instance.putObjectVolatile(this, AbstractActorCell.childrenOffset, TerminatedChildrenContainer) + Unsafe.instance.putObjectVolatile(this, AbstractActorCell.childrenOffset, TerminatedChildrenContainer): @nowarn( + "cat=deprecation") /* * ActorCell-internal API diff --git a/actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala b/actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala index 1ce9ee1bc21..1abc8311e63 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/dungeon/Dispatch.scala @@ -53,11 +53,14 @@ private[pekko] trait Dispatch { this: ActorCell => } final def mailbox: Mailbox = - Unsafe.instance.getObjectVolatile(this, AbstractActorCell.mailboxOffset).asInstanceOf[Mailbox] + Unsafe.instance.getObjectVolatile(this, AbstractActorCell.mailboxOffset).asInstanceOf[Mailbox]: @nowarn( + "cat=deprecation") - @tailrec final def swapMailbox(newMailbox: Mailbox): Mailbox = { + @tailrec + final def swapMailbox(newMailbox: Mailbox): Mailbox = { val oldMailbox = mailbox - if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.mailboxOffset, oldMailbox, newMailbox)) + if (!Unsafe.instance.compareAndSwapObject(this, AbstractActorCell.mailboxOffset, oldMailbox, newMailbox): @nowarn( + "cat=deprecation")) swapMailbox(newMailbox) else oldMailbox } diff --git a/actor/src/main/scala/org/apache/pekko/dispatch/AbstractDispatcher.scala b/actor/src/main/scala/org/apache/pekko/dispatch/AbstractDispatcher.scala index f2d6aa8d56e..40c7b4a772c 100644 --- a/actor/src/main/scala/org/apache/pekko/dispatch/AbstractDispatcher.scala +++ b/actor/src/main/scala/org/apache/pekko/dispatch/AbstractDispatcher.scala @@ -124,7 +124,7 @@ abstract class MessageDispatcher(val configurator: MessageDispatcherConfigurator } private final def addInhabitants(add: Long): Long = { - val old = Unsafe.instance.getAndAddLong(this, inhabitantsOffset, add) + val old = Unsafe.instance.getAndAddLong(this, inhabitantsOffset, add): @nowarn("cat=deprecation") val ret = old + add if (ret < 0) { // We haven't succeeded in decreasing the inhabitants yet but the simple fact that we're trying to @@ -136,11 +136,12 @@ abstract class MessageDispatcher(val configurator: MessageDispatcherConfigurator ret } - final def inhabitants: Long = Unsafe.instance.getLongVolatile(this, inhabitantsOffset) + final def inhabitants: Long = Unsafe.instance.getLongVolatile(this, inhabitantsOffset): @nowarn("cat=deprecation") - private final def shutdownSchedule: Int = Unsafe.instance.getIntVolatile(this, shutdownScheduleOffset) + private final def shutdownSchedule: Int = + Unsafe.instance.getIntVolatile(this, shutdownScheduleOffset): @nowarn("cat=deprecation") private final def updateShutdownSchedule(expect: Int, update: Int): Boolean = - Unsafe.instance.compareAndSwapInt(this, shutdownScheduleOffset, expect, update) + Unsafe.instance.compareAndSwapInt(this, shutdownScheduleOffset, expect, update): @nowarn("cat=deprecation") /** * Creates and returns a mailbox for the given actor. diff --git a/actor/src/main/scala/org/apache/pekko/dispatch/Mailbox.scala b/actor/src/main/scala/org/apache/pekko/dispatch/Mailbox.scala index b9df8ee00d7..61ed379edba 100644 --- a/actor/src/main/scala/org/apache/pekko/dispatch/Mailbox.scala +++ b/actor/src/main/scala/org/apache/pekko/dispatch/Mailbox.scala @@ -17,13 +17,10 @@ import java.util.{ Comparator, Deque, PriorityQueue, Queue } import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.locks.ReentrantLock - -import scala.annotation.tailrec +import scala.annotation.{ nowarn, tailrec } import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.util.control.NonFatal - import com.typesafe.config.Config - import org.apache.pekko import pekko.actor.{ ActorCell, ActorRef, ActorSystem, DeadLetter, InternalActorRef } import pekko.annotation.InternalStableApi @@ -121,7 +118,8 @@ private[pekko] abstract class Mailbox(val messageQueue: MessageQueue) @volatile protected var _systemQueueDoNotCallMeDirectly: SystemMessage = _ // null by default - final def currentStatus: Mailbox.Status = Unsafe.instance.getIntVolatile(this, AbstractMailbox.mailboxStatusOffset) + final def currentStatus: Mailbox.Status = + Unsafe.instance.getIntVolatile(this, AbstractMailbox.mailboxStatusOffset): @nowarn("cat=deprecation") final def shouldProcessMessage: Boolean = (currentStatus & shouldNotProcessMask) == 0 @@ -134,10 +132,11 @@ private[pekko] abstract class Mailbox(val messageQueue: MessageQueue) final def isScheduled: Boolean = (currentStatus & Scheduled) != 0 protected final def updateStatus(oldStatus: Status, newStatus: Status): Boolean = - Unsafe.instance.compareAndSwapInt(this, AbstractMailbox.mailboxStatusOffset, oldStatus, newStatus) + Unsafe.instance.compareAndSwapInt(this, AbstractMailbox.mailboxStatusOffset, oldStatus, newStatus): @nowarn( + "cat=deprecation") protected final def setStatus(newStatus: Status): Unit = - Unsafe.instance.putIntVolatile(this, AbstractMailbox.mailboxStatusOffset, newStatus) + Unsafe.instance.putIntVolatile(this, AbstractMailbox.mailboxStatusOffset, newStatus): @nowarn("cat=deprecation") /** * Reduce the suspend count by one. Caller does not need to worry about whether @@ -210,14 +209,16 @@ private[pekko] abstract class Mailbox(val messageQueue: MessageQueue) // Note: contrary how it looks, there is no allocation here, as SystemMessageList is a value class and as such // it just exists as a typed view during compile-time. The actual return type is still SystemMessage. new LatestFirstSystemMessageList( - Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[SystemMessage]) + Unsafe.instance.getObjectVolatile(this, AbstractMailbox.systemMessageOffset).asInstanceOf[ + SystemMessage]): @nowarn("cat=deprecation") protected final def systemQueuePut(_old: LatestFirstSystemMessageList, _new: LatestFirstSystemMessageList): Boolean = (_old.head eq _new.head) || - // Note: calling .head is not actually existing on the bytecode level as the parameters _old and _new - // are SystemMessage instances hidden during compile time behind the SystemMessageList value class. - // Without calling .head the parameters would be boxed in SystemMessageList wrapper. - Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old.head, _new.head) + // Note: calling .head is not actually existing on the bytecode level as the parameters _old and _new + // are SystemMessage instances hidden during compile time behind the SystemMessageList value class. + // Without calling .head the parameters would be boxed in SystemMessageList wrapper. + Unsafe.instance.compareAndSwapObject(this, AbstractMailbox.systemMessageOffset, _old.head, _new.head): @nowarn( + "cat=deprecation") final def canBeScheduledForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = currentStatus match { diff --git a/actor/src/main/scala/org/apache/pekko/pattern/AskSupport.scala b/actor/src/main/scala/org/apache/pekko/pattern/AskSupport.scala index 18f1ccc7008..c110e3d67ad 100644 --- a/actor/src/main/scala/org/apache/pekko/pattern/AskSupport.scala +++ b/actor/src/main/scala/org/apache/pekko/pattern/AskSupport.scala @@ -550,10 +550,10 @@ private[pekko] final class PromiseActorRef( } private[this] def watchedBy: Set[ActorRef] = - Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]] + Unsafe.instance.getObjectVolatile(this, watchedByOffset).asInstanceOf[Set[ActorRef]]: @nowarn("cat=deprecation") private[this] def updateWatchedBy(oldWatchedBy: Set[ActorRef], newWatchedBy: Set[ActorRef]): Boolean = - Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy) + Unsafe.instance.compareAndSwapObject(this, watchedByOffset, oldWatchedBy, newWatchedBy): @nowarn("cat=deprecation") @tailrec // Returns false if the Promise is already completed private[this] final def addWatcher(watcher: ActorRef): Boolean = watchedBy match { @@ -573,12 +573,13 @@ private[pekko] final class PromiseActorRef( case other => if (!updateWatchedBy(other, null)) clearWatchers() else other } - private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset) + private[this] def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset): @nowarn("cat=deprecation") private[this] def updateState(oldState: AnyRef, newState: AnyRef): Boolean = - Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState) + Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState): @nowarn("cat=deprecation") - private[this] def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState) + private[this] def setState(newState: AnyRef): Unit = + Unsafe.instance.putObjectVolatile(this, stateOffset, newState): @nowarn("cat=deprecation") override def getParent: InternalActorRef = provider.tempContainer diff --git a/actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala b/actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala index 38bf58f2739..00dcbeb2609 100644 --- a/actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala +++ b/actor/src/main/scala/org/apache/pekko/pattern/CircuitBreaker.scala @@ -318,7 +318,8 @@ class CircuitBreaker( * @return Whether the previous state matched correctly */ private[this] def swapState(oldState: State, newState: State): Boolean = - Unsafe.instance.compareAndSwapObject(this, AbstractCircuitBreaker.stateOffset, oldState, newState) + Unsafe.instance.compareAndSwapObject(this, AbstractCircuitBreaker.stateOffset, oldState, newState): @nowarn( + "cat=deprecation") /** * Helper method for accessing underlying state via Unsafe @@ -326,7 +327,8 @@ class CircuitBreaker( * @return Reference to current state */ private[this] def currentState: State = - Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.stateOffset).asInstanceOf[State] + Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.stateOffset).asInstanceOf[State]: @nowarn( + "cat=deprecation") /** * Helper method for updating the underlying resetTimeout via Unsafe @@ -336,13 +338,14 @@ class CircuitBreaker( this, AbstractCircuitBreaker.resetTimeoutOffset, oldResetTimeout, - newResetTimeout) + newResetTimeout): @nowarn("cat=deprecation") /** * Helper method for accessing to the underlying resetTimeout via Unsafe */ private[this] def currentResetTimeout: FiniteDuration = - Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.resetTimeoutOffset).asInstanceOf[FiniteDuration] + Unsafe.instance.getObjectVolatile(this, AbstractCircuitBreaker.resetTimeoutOffset).asInstanceOf[ + FiniteDuration]: @nowarn("cat=deprecation") /** * Wraps invocations of asynchronous calls that need to be protected. diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala index b97fef185cf..76ba57050b8 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/Association.scala @@ -271,13 +271,15 @@ private[remote] class Association( * @return Whether the previous state matched correctly */ private[artery] def swapState(oldState: AssociationState, newState: AssociationState): Boolean = - Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState) + Unsafe.instance.compareAndSwapObject(this, AbstractAssociation.sharedStateOffset, oldState, newState): @nowarn( + "cat=deprecation") /** * @return Reference to current shared state */ def associationState: AssociationState = - Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[AssociationState] + Unsafe.instance.getObjectVolatile(this, AbstractAssociation.sharedStateOffset).asInstanceOf[ + AssociationState]: @nowarn("cat=deprecation") def setControlIdleKillSwitch(killSwitch: OptionVal[SharedKillSwitch]): Unit = { val current = associationState