chore: Use Scala Option Converter (#2203)

* chore: Use Scala Option Converter
---------

Co-authored-by: He-Pin <hepin1989@gmail.com>
This commit is contained in:
PJ Fanning 2025-09-19 16:34:29 +01:00 committed by GitHub
parent 9b5e641246
commit 04f5fcd1b7
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
47 changed files with 67 additions and 293 deletions

View file

@ -531,7 +531,7 @@ Scala has proven the most viable way to do it, as long as you keep the following
1. Use `org.apache.pekko.util.FutureConverters` to translate `Future`s to `CompletionStage`s (or vice versa).
1. Use `org.apache.pekko.util.OptionConverters` to translate `Option`s to Java `Optional`s (or vice versa).
1. Use `scala.jdk.javaapi.OptionConverters` to translate `Option`s to Java `Optional`s (or vice versa).
1. Use `org.apache.pekko.util.FunctionConverters` to translate Scala Functions to Java Functions (or vice versa).

View file

@ -20,7 +20,7 @@ import org.slf4j.event.Level
import org.apache.pekko
import pekko.annotation.InternalApi
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.OptionVal
/**

View file

@ -20,7 +20,7 @@ import org.slf4j.event.Level
import org.apache.pekko
import pekko.util.ccompat.JavaConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object LoggingEvent {

View file

@ -22,13 +22,13 @@ import java.util.*;
import java.util.concurrent.CompletableFuture;
import scala.Option;
import scala.concurrent.Future;
import scala.jdk.javaapi.OptionConverters;
/**
* These tests are here to ensure that methods from {@link org.apache.pekko.util.FutureConverters},
* {@link org.apache.pekko.util.JavaDurationConverters} and {@link
* org.apache.pekko.util.OptionConverters} for use within Java can be compiled from with Java
* sources. This is because methods marked with the Scala 3 inline keyword cannot be called from
* within Java (see https://github.com/lampepfl/dotty/issues/19346)
* {@link org.apache.pekko.util.JavaDurationConverters} for use within Java can be compiled from
* with Java sources. This is because methods marked with the Scala 3 inline keyword cannot be
* called from within Java (see https://github.com/lampepfl/dotty/issues/19346)
*/
public class JavaConverterScala3InlineTest {
public void compileTest() {

View file

@ -1,76 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.pekko.util
import org.apache.pekko
import pekko.util.OptionConverters._
import java.util._
import scala.annotation.nowarn
/**
* These tests are here to ensure that methods from [[org.apache.pekko.util.FutureConverters]], [[org.apache.pekko.util.OptionConverters]]
* and [[org.apache.pekko.util.FunctionConverters]] that used within Pekko ecosystem but not Pekko core properly cross compile on Scala 2.12
* and Scala 2.13+.
*
* Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib
*/
@nowarn("msg=deprecated")
object Scala212CompatTest {
// .toJavaPrimitive tests
val javaDoubleOptional: java.util.Optional[Double] = java.util.Optional.of(1.0)
val scalaDoubleOption: Option[Double] = Some(1.0)
val doubleOptionalToJavaPrimitive: OptionalDouble = javaDoubleOptional.toJavaPrimitive
val doubleOptionToJavaPrimitive: OptionalDouble = scalaDoubleOption.toJavaPrimitive
val javaIntOptional: java.util.Optional[Int] = java.util.Optional.of(1)
val scalaIntOption: Option[Int] = Some(1)
val intOptionalToJavaPrimitive: OptionalInt = javaIntOptional.toJavaPrimitive
val intOptionToJavaPrimitive: OptionalInt = scalaIntOption.toJavaPrimitive
val javaLongOptional: java.util.Optional[Long] = java.util.Optional.of(1L)
val scalaLongOption: Option[Long] = Some(1L)
val longOptionalToJavaPrimitive: OptionalLong = javaLongOptional.toJavaPrimitive
val longOptionToJavaPrimitive: OptionalLong = scalaLongOption.toJavaPrimitive
// from java optional primitive
val javaOptionalDouble: java.util.OptionalDouble = java.util.OptionalDouble.of(1.0)
val optionalDoubleToScala: Option[Double] = javaOptionalDouble.toScala
val optionalDoubleToJavaGeneric: Optional[Double] = javaOptionalDouble.toJavaGeneric
val javaOptionalInt: java.util.OptionalInt = java.util.OptionalInt.of(1)
val optionalIntToScala: Option[Int] = javaOptionalInt.toScala
val optionalIntToJavaGeneric: Optional[Int] = javaOptionalInt.toJavaGeneric
val javaOptionalLong: java.util.OptionalLong = java.util.OptionalLong.of(1L)
val optionalLongToScala: Option[Long] = javaOptionalLong.toScala
val optionalLongToJavaGeneric: Optional[Long] = javaOptionalLong.toJavaGeneric
// OptionConverters toScala and toJava
OptionConverters.toJava(OptionConverters.toScala(java.util.Optional.of("")))
OptionConverters.toJava(OptionConverters.toScala(java.util.OptionalDouble.of(1.0)))
OptionConverters.toJava(OptionConverters.toScala(java.util.OptionalInt.of(1)))
OptionConverters.toJava(OptionConverters.toScala(java.util.OptionalLong.of(1L)))
// Iterable.single
val queue = scala.collection.immutable.Queue.empty[ByteString]
queue.enqueue(Iterable.single(ByteString.empty))
}

View file

@ -33,7 +33,7 @@ import pekko.annotation.InternalApi
import pekko.util.Helpers.toRootLowerCase
import pekko.util.Helpers.Requiring
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Point-to-point reliable delivery between a single producer actor sending messages and a single consumer

View file

@ -30,7 +30,7 @@ import pekko.actor.typed.receptionist.ServiceKey
import pekko.actor.typed.scaladsl.Behaviors
import pekko.annotation.ApiMayChange
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Work pulling is a pattern where several worker actors pull tasks in their own pace from

View file

@ -21,3 +21,15 @@ ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.japi.Option$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.japi.Option$None$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.japi.Option$Some")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.japi.Option$Some$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOption")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOption$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptional")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptional$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalDouble")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalDouble$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalInt")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalInt$")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalLong")
ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.OptionConverters$RichOptionalLong$")

View file

@ -1,85 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.pekko.util
import org.apache.pekko.annotation.InternalStableApi
import java.util._
import scala.jdk.OptionShape
/**
* INTERNAL API
*
* Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib
*/
@InternalStableApi
private[pekko] object OptionConverters {
@inline final def toScala[A](o: Optional[A]): Option[A] = scala.jdk.javaapi.OptionConverters.toScala(o)
@inline def toScala(o: OptionalDouble): Option[java.lang.Double] = scala.jdk.javaapi.OptionConverters.toScala(o)
@inline def toScala(o: OptionalInt): Option[java.lang.Integer] = scala.jdk.javaapi.OptionConverters.toScala(o)
@inline def toScala(o: OptionalLong): Option[java.lang.Long] = scala.jdk.javaapi.OptionConverters.toScala(o)
@inline final def toJava[A](o: Option[A]): Optional[A] = scala.jdk.javaapi.OptionConverters.toJava(o)
implicit final class RichOptional[A](private val o: java.util.Optional[A]) extends AnyVal {
@inline def toScala: Option[A] = scala.jdk.OptionConverters.RichOptional(o).toScala
@inline def toJavaPrimitive[O](implicit shape: OptionShape[A, O]): O =
scala.jdk.OptionConverters.RichOptional(o).toJavaPrimitive
}
implicit final class RichOption[A](private val o: Option[A]) extends AnyVal {
@inline def toJava: Optional[A] = scala.jdk.OptionConverters.RichOption(o).toJava
@inline def toJavaPrimitive[O](implicit shape: OptionShape[A, O]): O =
scala.jdk.OptionConverters.RichOption(o).toJavaPrimitive
}
implicit class RichOptionalDouble(private val o: OptionalDouble) extends AnyVal {
/** Convert a Java `OptionalDouble` to a Scala `Option` */
@inline def toScala: Option[Double] = scala.jdk.OptionConverters.RichOptionalDouble(o).toScala
/** Convert a Java `OptionalDouble` to a generic Java `Optional` */
@inline def toJavaGeneric: Optional[Double] = scala.jdk.OptionConverters.RichOptionalDouble(o).toJavaGeneric
}
/** Provides conversions from `OptionalInt` to Scala `Option` and the generic `Optional` */
implicit class RichOptionalInt(private val o: OptionalInt) extends AnyVal {
/** Convert a Java `OptionalInt` to a Scala `Option` */
@inline def toScala: Option[Int] = scala.jdk.OptionConverters.RichOptionalInt(o).toScala
/** Convert a Java `OptionalInt` to a generic Java `Optional` */
@inline def toJavaGeneric: Optional[Int] = scala.jdk.OptionConverters.RichOptionalInt(o).toJavaGeneric
}
/** Provides conversions from `OptionalLong` to Scala `Option` and the generic `Optional` */
implicit class RichOptionalLong(private val o: OptionalLong) extends AnyVal {
/** Convert a Java `OptionalLong` to a Scala `Option` */
@inline def toScala: Option[Long] = scala.jdk.OptionConverters.RichOptionalLong(o).toScala
/** Convert a Java `OptionalLong` to a generic Java `Optional` */
@inline def toJavaGeneric: Optional[Long] = scala.jdk.OptionConverters.RichOptionalLong(o).toJavaGeneric
}
}

View file

@ -1,79 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* license agreements; and to You under the Apache License, version 2.0:
*
* https://www.apache.org/licenses/LICENSE-2.0
*
* This file is part of the Apache Pekko project, derived from Akka.
*/
package org.apache.pekko.util
import org.apache.pekko.annotation.InternalStableApi
import java.util._
import scala.jdk.OptionShape
/**
* INTERNAL API
*
* Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib.
*/
@InternalStableApi
private[pekko] object OptionConverters {
// Ideally these methods should have the Scala 3 inline keyword but then Java sources are
// unable to call these methods, see https://github.com/lampepfl/dotty/issues/19346
final def toScala[A](o: Optional[A]): Option[A] = scala.jdk.javaapi.OptionConverters.toScala(o)
def toScala(o: OptionalDouble): Option[java.lang.Double] = scala.jdk.javaapi.OptionConverters.toScala(o)
def toScala(o: OptionalInt): Option[java.lang.Integer] = scala.jdk.javaapi.OptionConverters.toScala(o)
def toScala(o: OptionalLong): Option[java.lang.Long] = scala.jdk.javaapi.OptionConverters.toScala(o)
final def toJava[A](o: Option[A]): Optional[A] = scala.jdk.javaapi.OptionConverters.toJava(o)
implicit final class RichOptional[A](private val o: java.util.Optional[A]) extends AnyVal {
inline def toScala: Option[A] = scala.jdk.OptionConverters.RichOptional(o).toScala
inline def toJavaPrimitive[O](implicit shape: OptionShape[A, O]): O =
scala.jdk.OptionConverters.RichOptional(o).toJavaPrimitive
}
implicit final class RichOption[A](private val o: Option[A]) extends AnyVal {
inline def toJava: Optional[A] = scala.jdk.OptionConverters.RichOption(o).toJava
inline def toJavaPrimitive[O](implicit shape: OptionShape[A, O]): O =
scala.jdk.OptionConverters.RichOption(o).toJavaPrimitive
}
implicit class RichOptionalDouble(private val o: OptionalDouble) extends AnyVal {
/** Convert a Java `OptionalDouble` to a Scala `Option` */
inline def toScala: Option[Double] = scala.jdk.OptionConverters.RichOptionalDouble(o).toScala
/** Convert a Java `OptionalDouble` to a generic Java `Optional` */
inline def toJavaGeneric: Optional[Double] = scala.jdk.OptionConverters.RichOptionalDouble(o).toJavaGeneric
}
/** Provides conversions from `OptionalInt` to Scala `Option` and the generic `Optional` */
implicit class RichOptionalInt(private val o: OptionalInt) extends AnyVal {
/** Convert a Java `OptionalInt` to a Scala `Option` */
inline def toScala: Option[Int] = scala.jdk.OptionConverters.RichOptionalInt(o).toScala
/** Convert a Java `OptionalInt` to a generic Java `Optional` */
inline def toJavaGeneric: Optional[Int] = scala.jdk.OptionConverters.RichOptionalInt(o).toJavaGeneric
}
/** Provides conversions from `OptionalLong` to Scala `Option` and the generic `Optional` */
implicit class RichOptionalLong(private val o: OptionalLong) extends AnyVal {
/** Convert a Java `OptionalLong` to a Scala `Option` */
inline def toScala: Option[Long] = scala.jdk.OptionConverters.RichOptionalLong(o).toScala
/** Convert a Java `OptionalLong` to a generic Java `Optional` */
inline def toJavaGeneric: Optional[Long] = scala.jdk.OptionConverters.RichOptionalLong(o).toJavaGeneric
}
}

View file

@ -283,7 +283,7 @@ abstract class AbstractActor extends Actor {
@throws(classOf[Exception])
@nowarn("msg=deprecated")
override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
preRestart(reason, message.toJava)
}
@ -295,7 +295,7 @@ abstract class AbstractActor extends Actor {
*/
@throws(classOf[Exception])
def preRestart(reason: Throwable, message: Optional[Any]): Unit = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
super.preRestart(reason, message.toScala)
}

View file

@ -100,7 +100,7 @@ final case class ActorIdentity(correlationId: Any, ref: Option[ActorRef]) {
* not defined if no actor matched the request.
*/
def getActorRef: Optional[ActorRef] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
ref.toJava
}
}

View file

@ -41,7 +41,7 @@ import pekko.japi.Util.immutableSeq
import pekko.serialization.SerializationExtension
import pekko.util._
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.Helpers.toRootLowerCase
import pekko.util.ccompat.JavaConverters._

View file

@ -12,6 +12,7 @@
*/
package org.apache.pekko.actor
import java.net.MalformedURLException
import java.net.URI
import java.net.URISyntaxException
@ -22,7 +23,7 @@ import scala.collection.immutable
import org.apache.pekko
import pekko.annotation.InternalApi
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* The address specifies the physical location under which an Actor can be

View file

@ -34,7 +34,7 @@ import pekko.Done
import pekko.annotation.InternalApi
import pekko.event.Logging
import pekko.pattern.after
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.OptionVal
import pekko.util.FutureConverters._

View file

@ -20,7 +20,7 @@ import scala.reflect.ClassTag
import org.apache.pekko
import pekko.annotation.InternalApi
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Marker supertype for a setup part that can be put inside [[pekko.actor.setup.ActorSystemSetup]], if a specific concrete setup

View file

@ -816,7 +816,7 @@ object Patterns {
delayFunction: IntFunction[Optional[java.time.Duration]],
scheduler: Scheduler,
context: ExecutionContext): CompletionStage[T] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
require(attempt != null, "Parameter attempt should not be null.")
scalaRetry(
() => attempt.call().asScala,
@ -857,7 +857,7 @@ object Patterns {
delayFunction: IntFunction[Optional[java.time.Duration]],
scheduler: Scheduler,
context: ExecutionContext): CompletionStage[T] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
require(attempt != null, "Parameter attempt should not be null.")
scalaRetry(
() => attempt.call().asScala,

View file

@ -33,7 +33,7 @@ import pekko.annotation.ApiMayChange
import pekko.cluster.sharding.typed.ShardingEnvelope
import pekko.cluster.sharding.typed.delivery.internal.ShardingProducerControllerImpl
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Reliable delivery between a producer actor sending messages to sharded consumer

View file

@ -147,7 +147,7 @@ import pekko.util.JavaDurationConverters._
// javadsl impl
override def init[M, E](entity: javadsl.Entity[M, E]): ActorRef[E] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
init(
new scaladsl.Entity(
createBehavior = (ctx: EntityContext[M]) =>

View file

@ -40,7 +40,7 @@ import pekko.cluster.sharding.typed.scaladsl.StartEntity
import pekko.cluster.typed.Cluster
import pekko.cluster.typed.SelfUp
import pekko.cluster.typed.Subscribe
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.PrettyDuration
/**

View file

@ -32,7 +32,7 @@ import pekko.cluster.sharding.ShardCoordinator.ShardAllocationStrategy
import pekko.cluster.sharding.typed.internal.EntityTypeKeyImpl
import pekko.japi.function.{ Function => JFunction }
import pekko.pattern.StatusReply
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
@FunctionalInterface
trait EntityFactory[M] {

View file

@ -26,7 +26,7 @@ import pekko.coordination.lease.LeaseSettings
import pekko.coordination.lease.javadsl.{ Lease => JavaLease }
import pekko.coordination.lease.scaladsl.{ Lease => ScalaLease }
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* INTERNAL API

View file

@ -25,7 +25,7 @@ import scala.concurrent.duration.FiniteDuration
import org.apache.pekko
import pekko.actor.{ DeadLetterSuppression, NoSerializationVerificationNeeded }
import pekko.util.HashCode
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object ServiceDiscovery {

View file

@ -17,7 +17,7 @@ import java.util.Optional
import org.apache.pekko
import pekko.cluster.UniqueAddress
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Interface for implementing a state based convergent

View file

@ -64,7 +64,7 @@ final class EventEnvelope(
* Java API
*/
def getEventMetaData(): Optional[Any] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
eventMetadata.toJava
}

View file

@ -93,7 +93,7 @@ final class EventEnvelope[Event](
* Java API
*/
def getOptionalEvent(): Optional[Event] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
eventOption.toJava
}
@ -101,7 +101,7 @@ final class EventEnvelope[Event](
* Java API
*/
def getEventMetaData(): Optional[AnyRef] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
eventMetadata.map(_.asInstanceOf[AnyRef]).toJava
}

View file

@ -226,7 +226,7 @@ final class EventSourcedBehaviorTestKit[Command, Event, State](
private val _persistenceTestKit = new PersistenceTestKit(delegate.persistenceTestKit)
private val _snapshotTestKit = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
delegate.snapshotTestKit.map(new SnapshotTestKit(_)).toJava
}

View file

@ -27,7 +27,7 @@ import pekko.persistence.state.javadsl.GetObjectResult
import pekko.persistence.testkit.state.scaladsl.{ PersistenceTestKitDurableStateStore => SStore }
import pekko.stream.javadsl.Source
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object PersistenceTestKitDurableStateStore {
val Identifier = pekko.persistence.testkit.state.scaladsl.PersistenceTestKitDurableStateStore.Identifier

View file

@ -387,7 +387,7 @@ private[pekko] final case class PublishedEventImpl(
replicatedMetaData: Option[ReplicatedPublishedEventMetaData])
extends PublishedEvent
with InternalProtocol {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
def tags: Set[String] = payload match {
case t: Tagged => t.tags

View file

@ -27,7 +27,7 @@ import pekko.annotation.InternalApi
import pekko.persistence.typed._
import pekko.persistence.typed.EventAdapter
import pekko.persistence.typed.internal._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.unused
abstract class EventSourcedBehavior[Command, Event, State] private[pekko] (

View file

@ -20,7 +20,7 @@ import org.apache.pekko
import pekko.japi.function.Function3
import pekko.persistence.typed.SnapshotAdapter
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Helper functions for migration from PersistentFSM to Persistence Typed

View file

@ -30,7 +30,7 @@ abstract class SnapshotStore extends SSnapshotStore with SnapshotStorePlugin {
override final def loadAsync(
persistenceId: String,
criteria: SnapshotSelectionCriteria): Future[Option[SelectedSnapshot]] = {
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
doLoadAsync(persistenceId, criteria).asScala.map(_.toScala)(ExecutionContext.parasitic)
}

View file

@ -18,7 +18,7 @@ import java.util.concurrent.CompletionStage
import org.apache.pekko
import pekko.persistence.state.scaladsl.{ GetObjectResult => SGetObjectResult }
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* API for reading durable state objects with payload `A`.

View file

@ -17,7 +17,7 @@ import scala.concurrent.Future
import org.apache.pekko
import pekko.persistence.state.javadsl.{ GetObjectResult => JGetObjectResult }
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* API for reading durable state objects with payload `A`.

View file

@ -18,6 +18,7 @@ import java.util.concurrent.ConcurrentHashMap
import scala.annotation.nowarn
import scala.collection.immutable
import scala.util.{ Failure, Success }
import com.fasterxml.jackson.annotation.{ JsonAutoDetect, JsonCreator, PropertyAccessor }
import com.fasterxml.jackson.core.{
JsonFactory,
@ -56,7 +57,7 @@ import pekko.actor.setup.Setup
import pekko.annotation.InternalStableApi
import pekko.event.{ Logging, LoggingAdapter }
import pekko.util.unused
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object JacksonObjectMapperProvider extends ExtensionId[JacksonObjectMapperProvider] with ExtensionIdProvider {
override def get(system: ActorSystem): JacksonObjectMapperProvider = super.get(system)

View file

@ -32,7 +32,7 @@ import pekko.stream.impl.TraversalBuilder
import pekko.util.{ ByteString, OptionVal }
import pekko.util.JavaDurationConverters._
import pekko.util.LineNumbers
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* Holds attributes which can be used to alter [[pekko.stream.scaladsl.Flow]] / [[pekko.stream.javadsl.Flow]]

View file

@ -37,7 +37,7 @@ import pekko.stream.impl.fusing.{ StatefulMapConcat, ZipWithIndexJava }
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.Timeout
import pekko.util.unused
import org.reactivestreams.Processor
@ -4342,7 +4342,7 @@ final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Gr
harvest: function.Function[Agg, Emit],
emitOnTimer: Optional[Pair[function.Predicate[Agg], java.time.Duration]])
: javadsl.Flow[In, Emit, Mat] = {
import org.apache.pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
asScala
.aggregateWithBoundary(() => allocate.create())(
aggregate = (agg, out) => aggregate.apply(agg, out).toScala,

View file

@ -26,7 +26,7 @@ import pekko.stream._
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.ccompat.JavaConverters._
object FlowWithContext {

View file

@ -23,7 +23,7 @@ import pekko.Done
import scala.concurrent.ExecutionContext
import pekko.stream.QueueOfferResult
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* This trait allows to have a queue as a data source for some stream.

View file

@ -20,7 +20,7 @@ import pekko.annotation.ApiMayChange
import pekko.japi.Pair
import pekko.stream.scaladsl
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object RetryFlow {

View file

@ -32,7 +32,7 @@ import pekko.stream.impl.LinearTraversalBuilder
import pekko.stream.scaladsl.SinkToCompletionStage
import pekko.util.ConstantFun.scalaAnyToUnit
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import org.reactivestreams.{ Publisher, Subscriber }

View file

@ -36,7 +36,7 @@ import pekko.stream.impl.fusing.{ ArraySource, StatefulMapConcat, ZipWithIndexJa
import pekko.util.{ unused, _ }
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.ccompat.JavaConverters._
import org.reactivestreams.{ Publisher, Subscriber }
@ -4782,7 +4782,7 @@ final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[
aggregate: function.Function2[Agg, Out, Pair[Agg, Boolean]],
harvest: function.Function[Agg, Emit],
emitOnTimer: Optional[Pair[function.Predicate[Agg], java.time.Duration]]): javadsl.Source[Emit, Mat] = {
import org.apache.pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
asScala
.aggregateWithBoundary(() => allocate.create())(
aggregate = (agg, out) => aggregate.apply(agg, out).toScala,

View file

@ -28,7 +28,7 @@ import pekko.stream._
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.ccompat.JavaConverters._
object SourceWithContext {

View file

@ -30,7 +30,7 @@ import pekko.stream.impl.fusing.{ StatefulMapConcat, ZipWithIndexJava }
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.ccompat.JavaConverters._
object SubFlow {
@ -2824,7 +2824,7 @@ final class SubFlow[In, Out, Mat](
harvest: function.Function[Agg, Emit],
emitOnTimer: Optional[Pair[function.Predicate[Agg], java.time.Duration]])
: javadsl.SubFlow[In, Emit, Mat] = {
import org.apache.pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
new SubFlow(
asScala.aggregateWithBoundary(() => allocate.create())(
aggregate = (agg, out) => aggregate.apply(agg, out).toScala,

View file

@ -30,7 +30,7 @@ import pekko.stream.impl.fusing.{ StatefulMapConcat, ZipWithIndexJava }
import pekko.util.ConstantFun
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
import pekko.util.ccompat.JavaConverters._
/**
@ -2798,7 +2798,7 @@ final class SubSource[Out, Mat](
harvest: function.Function[Agg, Emit],
emitOnTimer: Optional[Pair[function.Predicate[Agg], java.time.Duration]])
: javadsl.SubSource[Emit, Mat] = {
import org.apache.pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
new SubSource(
asScala.aggregateWithBoundary(() => allocate.create())(
aggregate = (agg, out) => aggregate.apply(agg, out).toScala,

View file

@ -41,7 +41,7 @@ import pekko.stream.scaladsl
import pekko.util.ByteString
import pekko.util.FutureConverters._
import pekko.util.JavaDurationConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
object Tcp extends ExtensionId[Tcp] with ExtensionIdProvider {

View file

@ -24,7 +24,7 @@ import pekko.annotation.InternalApi
import scala.concurrent.ExecutionContext
import pekko.stream.QueueOfferResult
import pekko.util.FutureConverters._
import pekko.util.OptionConverters._
import scala.jdk.OptionConverters._
/**
* This trait allows to have a queue as a data source for some stream.