From f2b677dfa0d22fffd98184ca6b3b0e7488ea568c Mon Sep 17 00:00:00 2001 From: PJ Fanning Date: Sat, 13 Sep 2025 10:03:10 +0100 Subject: [PATCH] remove scala 2.12 support (#1986) * remove scala 2.12 support * move around some source because we don't need as many source dirs * move more files * remove annotations * remove annotations * Create remove-scala-2.12-support.excludes * remove some scala-2.12 refs * remove compat.PartialFunction * Update remove-scala-2.12-support.excludes --- .github/workflows/dependency-graph.yml | 2 +- .github/workflows/nightly-builds.yml | 2 +- CONTRIBUTING.md | 2 +- LICENSE | 3 +- .../SameThreadExecutionContextSpec.scala | 8 +- .../ByteStringBuilderScala213PlusSpec.scala | 0 .../pekko/util/Scala212CompatTest.scala | 2 - .../internal/receptionist/Platform.scala | 26 - .../remove-scala-2.12-support.excludes | 21 + .../org/apache/pekko/compat/Future.scala | 57 - .../apache/pekko/compat/PartialFunction.scala | 35 - .../internal/SameThreadExecutionContext.scala | 39 - .../dispatch/internal/ScalaBatchable.scala | 32 - .../org/apache/pekko/util/ByteIterator.scala | 696 -------- .../org/apache/pekko/util/ByteString.scala | 1456 ----------------- .../pekko/util/FunctionConverters.scala | 562 ------- .../apache/pekko/util/FutureConverters.scala | 43 - .../apache/pekko/util/OptionConverters.scala | 91 -- .../pekko/util/ccompat/CompatImpl.scala | 31 - .../util/ccompat/ccompatUsedUntil213.scala | 27 - .../apache/pekko/util/ccompat/package.scala | 154 -- .../util/ccompat/ccompatUsedUntil213.scala | 27 - .../apache/pekko/compat/PartialFunction.scala | 31 - .../pekko/util/JavaDurationConverters.scala | 0 .../apache/pekko/compat/PartialFunction.scala | 31 - .../org/apache/pekko/actor/AbstractFSM.scala | 2 +- .../apache/pekko/actor/ActorSelection.scala | 2 - .../apache/pekko/actor/FaultHandling.scala | 2 - .../org/apache/pekko/compat/Future.scala | 0 .../main/scala/org/apache/pekko/io/Dns.scala | 2 - .../main/scala/org/apache/pekko/io/Udp.scala | 2 - .../org/apache/pekko/io/UdpConnected.scala | 2 - .../org/apache/pekko/io/dns/DnsSettings.scala | 2 - .../org/apache/pekko/routing/MurmurHash.scala | 3 - .../pekko/routing/RoutedActorCell.scala | 2 - .../pekko/serialization/Serialization.scala | 2 - .../pekko/util/FunctionConverters.scala | 0 .../pekko/util/SubclassifiedIndex.scala | 3 - .../apache/pekko/util/ccompat/package.scala | 0 .../metrics/protobuf/MessageSerializer.scala | 2 - ...sterShardingRememberEntitiesPerfSpec.scala | 2 - ...sterShardingPreparingForShutdownSpec.scala | 2 - .../typed/ReplicatedShardingSpec.scala | 2 - .../typed/scaladsl/ClusterShardingSpec.scala | 2 - .../ClusterShardingMessageSerializer.scala | 2 - .../ClusterShardCoordinatorDowning2Spec.scala | 2 - .../ClusterShardCoordinatorDowningSpec.scala | 2 - .../sharding/ClusterShardingFailureSpec.scala | 2 - .../sharding/ClusterShardingLeavingSpec.scala | 2 - .../ClusterShardingMinMembersSpec.scala | 2 - .../ClusterShardingRememberEntitiesSpec.scala | 2 - .../sharding/MultiDcClusterShardingSpec.scala | 2 - .../MultiNodeClusterShardingSpec.scala | 2 - .../CoordinatedShutdownShardingSpec.scala | 2 - .../pekko/cluster/client/ClusterClient.scala | 2 - .../DistributedPubSubMessageSerializer.scala | 2 - .../ClusterSingletonManagerDownedSpec.scala | 2 - .../ClusterSingletonLeavingSpeedSpec.scala | 2 - .../ClusterSingletonRestart2Spec.scala | 2 - .../ClusterSingletonRestartSpec.scala | 2 - .../ClusterReceptionistProtocol.scala | 27 - .../apache/pekko/cluster/ClusterEvent.scala | 2 - .../pekko/cluster/ClusterHeartbeat.scala | 2 - .../cluster/CrossDcClusterHeartbeat.scala | 2 - .../cluster/JoinConfigCompatChecker.scala | 2 - .../pekko/cluster/MembershipState.scala | 2 - .../apache/pekko/cluster/Reachability.scala | 2 - .../protobuf/ClusterMessageSerializer.scala | 2 - .../pekko/cluster/ClusterShutdownSpec.scala | 2 - .../cluster/MinMembersBeforeUpSpec.scala | 2 - .../MultiDcHeartbeatTakingOverSpec.scala | 2 - .../pekko/cluster/MultiNodeClusterSpec.scala | 2 - .../pekko/cluster/NodeMembershipSpec.scala | 2 - .../cluster/RestartFirstSeedNodeSpec.scala | 2 - .../pekko/cluster/RestartNode2Spec.scala | 2 - .../pekko/cluster/RestartNode3Spec.scala | 2 - .../pekko/cluster/RestartNodeSpec.scala | 2 - .../UnreachableNodeJoinsAgainSpec.scala | 2 - .../ddata/DeltaPropagationSelector.scala | 2 - .../pekko/cluster/ddata/Replicator.scala | 2 - .../protobuf/ReplicatedDataSerializer.scala | 2 - .../ReplicatorMessageSerializer.scala | 2 - .../ddata/protobuf/SerializationSupport.scala | 2 - .../cluster/ddata/DurablePruningSpec.scala | 2 - .../src/main/paradox/general/configuration.md | 2 +- legal/pekko-actor-jar-license.txt | 2 +- .../pekko/remote/testconductor/Player.scala | 2 - .../pekko/remote/testkit/MultiNodeSpec.scala | 2 - .../persistence/AtLeastOnceDelivery.scala | 2 - .../persistence/journal/EventAdapters.scala | 2 - .../journal/japi/AsyncWriteJournal.scala | 2 - .../serialization/MessageSerializer.scala | 2 - .../snapshot/local/LocalSnapshotStore.scala | 2 - project/Dependencies.scala | 3 +- .../org/apache/pekko/remote/Remoting.scala | 3 - .../pekko/remote/artery/Association.scala | 2 - .../remote/artery/RemoteInstrument.scala | 2 - .../aeron/ArteryAeronUdpTransport.scala | 2 - .../artery/tcp/ArteryTcpTransport.scala | 2 - .../remote/artery/tcp/SSLEngineProvider.scala | 2 - .../DaemonMsgCreateSerializer.scala | 2 - .../MessageContainerSerializer.scala | 2 - .../transport/netty/NettySSLSupport.scala | 2 - .../apache/pekko/remote/DaemonicSpec.scala | 2 - .../pekko/stream/testkit/StreamTestKit.scala | 2 - .../compression/CompressionTestingTools.scala | 2 - .../scaladsl/FlowStatefulMapConcatSpec.scala | 2 - .../pekko/stream/javadsl/CollectionUtil.scala | 39 - .../pekko/stream/impl/fusing/MapConcat.scala | 2 - .../apache/pekko/stream/impl/fusing/Ops.scala | 2 - .../apache/pekko/stream/javadsl/Flow.scala | 2 - .../apache/pekko/stream/javadsl/Source.scala | 2 - .../StatefulMapConcatAccumulator.scala | 2 - .../apache/pekko/stream/javadsl/SubFlow.scala | 2 - .../pekko/stream/javadsl/SubSource.scala | 2 - .../apache/pekko/stream/scaladsl/Flow.scala | 2 - .../stream/scaladsl/FlowWithContextOps.scala | 2 - .../StatefulMapConcatAccumulator.scala | 2 - .../pekko/testkit/TestEventListener.scala | 2 - .../org/apache/pekko/testkit/package.scala | 2 - 120 files changed, 31 insertions(+), 3597 deletions(-) rename actor-tests/src/test/{scala-2.13+ => scala}/org/apache/pekko/util/ByteStringBuilderScala213PlusSpec.scala (100%) delete mode 100644 actor-typed/src/main/scala-2.12/org/apache/pekko/actor/typed/internal/receptionist/Platform.scala create mode 100644 actor/src/main/mima-filters/2.0.x.backwards.excludes/remove-scala-2.12-support.excludes delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/compat/Future.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/compat/PartialFunction.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/SameThreadExecutionContext.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/ScalaBatchable.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/ByteIterator.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/ByteString.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/FunctionConverters.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/FutureConverters.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/OptionConverters.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/CompatImpl.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala delete mode 100644 actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala delete mode 100644 actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala delete mode 100644 actor/src/main/scala-2.13/org/apache/pekko/compat/PartialFunction.scala rename actor/src/main/{scala-2 => scala-2.13}/org/apache/pekko/util/JavaDurationConverters.scala (100%) delete mode 100644 actor/src/main/scala-3/org/apache/pekko/compat/PartialFunction.scala rename actor/src/main/{scala-2.13+ => scala}/org/apache/pekko/compat/Future.scala (100%) rename actor/src/main/{scala-2.13+ => scala}/org/apache/pekko/util/FunctionConverters.scala (100%) rename actor/src/main/{scala-2.13+ => scala}/org/apache/pekko/util/ccompat/package.scala (100%) delete mode 100644 cluster-typed/src/main/scala-2.12/org/apache/pekko/cluster/typed/internal/receptionist/ClusterReceptionistProtocol.scala delete mode 100644 stream/src/main/scala-2.12/org/apache/pekko/stream/javadsl/CollectionUtil.scala diff --git a/.github/workflows/dependency-graph.yml b/.github/workflows/dependency-graph.yml index 01304054e7..435e82302b 100644 --- a/.github/workflows/dependency-graph.yml +++ b/.github/workflows/dependency-graph.yml @@ -39,4 +39,4 @@ jobs: - uses: scalacenter/sbt-dependency-submission@64084844d2b0a9b6c3765f33acde2fbe3f5ae7d3 # v3.1.0 with: configs-ignore: provided optional test TestJdk9 compile-internal runtime-internal pr-validation multi-jvm scala-tool scala-doc-tool - modules-ignore: pekko-bench-jmh_2.12 pekko-docs_2.12 pekko-bench-jmh_2.13 pekko-docs_2.13 pekko-bench-jmh_3 pekko-docs_3 + modules-ignore: pekko-bench-jmh_2.13 pekko-docs_2.13 pekko-bench-jmh_3 pekko-docs_3 diff --git a/.github/workflows/nightly-builds.yml b/.github/workflows/nightly-builds.yml index f01ceca363..46034d5607 100644 --- a/.github/workflows/nightly-builds.yml +++ b/.github/workflows/nightly-builds.yml @@ -114,7 +114,7 @@ jobs: # No need to specify the full Scala version. Only the Scala # binary version is required and Pekko build will set the right # full version from it. - scalaVersion: ["2.12.x", "2.13.x", "3.3.x"] + scalaVersion: ["2.13.x", "3.3.x"] javaVersion: [17, 21] env: DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 4d375ce2f9..6b93cd8d2c 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -236,7 +236,7 @@ Pekko uses [MiMa](https://github.com/lightbend/mima) to validate the binary comp PR fails due to binary compatibility issues, you may see an error like this: ``` -[info] stream: found 1 potential binary incompatibilities while checking against org.apache.pekko:pekko-stream_2.12:2.4.2 (filtered 222) +[info] stream: found 1 potential binary incompatibilities while checking against org.apache.pekko:pekko-stream_2.13:2.4.2 (filtered 222) [error] * method foldAsync(java.lang.Object,scala.Function2)org.apache.pekko.stream.scaladsl.FlowOps in trait org.apache.pekko.stream.scaladsl.FlowOps is present only in current version [error] filter with: ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.pekko.stream.scaladsl.FlowOps.foldAsync") ``` diff --git a/LICENSE b/LICENSE index be35cdd3ab..76701cf517 100644 --- a/LICENSE +++ b/LICENSE @@ -212,7 +212,7 @@ Copyright (c) 2003-2011, LAMP/EPFL pekko-actor contains code from scala-collection-compat in the `org.apache.pekko.util.ccompat` package which has released under an Apache 2.0 license. -- actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala +- actor/src/main/scala/org/apache/pekko/util/ccompat/package.scala Scala (https://www.scala-lang.org) @@ -222,7 +222,6 @@ Copyright EPFL and Lightbend, Inc. pekko-actor contains code from scala-library in the `org.apache.pekko.util.ccompat` package and in `org.apache.pekko.util.Helpers.scala` which was released under an Apache 2.0 license. -- actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala - actor/src/main/scala/org/apache/pekko/util/Helpers.scala Scala (https://www.scala-lang.org) diff --git a/actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala index e6ead3715e..740afaea67 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/dispatch/SameThreadExecutionContextSpec.scala @@ -29,12 +29,8 @@ class SameThreadExecutionContextSpec extends PekkoSpec with Matchers { "return a Scala specific version" in { val ec = SameThreadExecutionContext() - if (util.Properties.versionNumberString.startsWith("2.12")) { - ec.getClass.getName should startWith("org.apache.pekko.dispatch.internal.SameThreadExecutionContext") - } else { - // in 2.13 and higher parasitic is available - ec.getClass.getName should ===("scala.concurrent.ExecutionContext$parasitic$") - } + // in Scala 2.13 and higher parasitic is available + ec.getClass.getName should ===("scala.concurrent.ExecutionContext$parasitic$") } "should run follow up future operations in the same dispatcher" in { diff --git a/actor-tests/src/test/scala-2.13+/org/apache/pekko/util/ByteStringBuilderScala213PlusSpec.scala b/actor-tests/src/test/scala/org/apache/pekko/util/ByteStringBuilderScala213PlusSpec.scala similarity index 100% rename from actor-tests/src/test/scala-2.13+/org/apache/pekko/util/ByteStringBuilderScala213PlusSpec.scala rename to actor-tests/src/test/scala/org/apache/pekko/util/ByteStringBuilderScala213PlusSpec.scala diff --git a/actor-tests/src/test/scala/org/apache/pekko/util/Scala212CompatTest.scala b/actor-tests/src/test/scala/org/apache/pekko/util/Scala212CompatTest.scala index 6f7af67721..2380197b22 100644 --- a/actor-tests/src/test/scala/org/apache/pekko/util/Scala212CompatTest.scala +++ b/actor-tests/src/test/scala/org/apache/pekko/util/Scala212CompatTest.scala @@ -18,7 +18,6 @@ package org.apache.pekko.util import org.apache.pekko -import pekko.util.ccompat._ import pekko.util.OptionConverters._ import java.util._ @@ -32,7 +31,6 @@ import scala.annotation.nowarn * Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib */ -@ccompatUsedUntil213 @nowarn("msg=deprecated") object Scala212CompatTest { diff --git a/actor-typed/src/main/scala-2.12/org/apache/pekko/actor/typed/internal/receptionist/Platform.scala b/actor-typed/src/main/scala-2.12/org/apache/pekko/actor/typed/internal/receptionist/Platform.scala deleted file mode 100644 index bc464be208..0000000000 --- a/actor-typed/src/main/scala-2.12/org/apache/pekko/actor/typed/internal/receptionist/Platform.scala +++ /dev/null @@ -1,26 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.actor.typed.internal.receptionist - -import org.apache.pekko -import pekko.actor.typed.ActorRef -import pekko.annotation.InternalApi - -/** - * INTERNAL API - */ -@InternalApi private[receptionist] object Platform { - type Service[K <: AbstractServiceKey] = ActorRef[K#Protocol] - type Subscriber[K <: AbstractServiceKey] = ActorRef[ReceptionistMessages.Listing[K#Protocol]] -} diff --git a/actor/src/main/mima-filters/2.0.x.backwards.excludes/remove-scala-2.12-support.excludes b/actor/src/main/mima-filters/2.0.x.backwards.excludes/remove-scala-2.12-support.excludes new file mode 100644 index 0000000000..8f0cad9a35 --- /dev/null +++ b/actor/src/main/mima-filters/2.0.x.backwards.excludes/remove-scala-2.12-support.excludes @@ -0,0 +1,21 @@ +# 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. + +# Remove deprecated method in Scheduler +ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.compat.PartialFunction") +ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.compat.PartialFunction$") +ProblemFilters.exclude[MissingClassProblem]("org.apache.pekko.util.ccompat.ccompatUsedUntil213") diff --git a/actor/src/main/scala-2.12/org/apache/pekko/compat/Future.scala b/actor/src/main/scala-2.12/org/apache/pekko/compat/Future.scala deleted file mode 100644 index 281bebb254..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/compat/Future.scala +++ /dev/null @@ -1,57 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.compat - -import scala.collection.immutable -import scala.concurrent.{ ExecutionContext, Future => SFuture } - -import scala.annotation.nowarn - -import org.apache.pekko -import pekko.annotation.InternalApi -import pekko.util.ccompat._ - -/** - * INTERNAL API - * - * Compatibility wrapper for `scala.concurrent.Future` to be able to compile the same code - * against Scala 2.12, 2.13 - * - * Remove these classes as soon as support for Scala 2.12 is dropped! - */ -@nowarn @InternalApi private[pekko] object Future { - def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.fold[T, R](futures)(zero)(op)(executor) - - def fold[T, R](futures: immutable.Iterable[SFuture[T]])(zero: R)(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.foldLeft[T, R](futures)(zero)(op)(executor) - - def reduce[T, R >: T](futures: IterableOnce[SFuture[T]])(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.reduce[T, R](futures)(op)(executor) - - def reduce[T, R >: T](futures: immutable.Iterable[SFuture[T]])(op: (R, T) => R)( - implicit executor: ExecutionContext): SFuture[R] = - SFuture.reduceLeft[T, R](futures)(op)(executor) - - def find[T](futures: IterableOnce[SFuture[T]])(p: T => Boolean)( - implicit executor: ExecutionContext): SFuture[Option[T]] = - SFuture.find[T](futures)(p)(executor) - - def find[T](futures: immutable.Iterable[SFuture[T]])(p: T => Boolean)( - implicit executor: ExecutionContext): SFuture[Option[T]] = - SFuture.find[T](futures)(p)(executor) -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/compat/PartialFunction.scala b/actor/src/main/scala-2.12/org/apache/pekko/compat/PartialFunction.scala deleted file mode 100644 index ead020fb13..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/compat/PartialFunction.scala +++ /dev/null @@ -1,35 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.compat - -import scala.annotation.nowarn - -import org.apache.pekko.annotation.InternalApi - -/** - * INTERNAL API - * - * Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code - * against Scala 2.12, 2.13, 3.0 - * - * Remove these classes as soon as support for Scala 2.12 is dropped! - */ -@InternalApi private[pekko] object PartialFunction { - - def fromFunction[A, B](f: (A) => B): scala.PartialFunction[A, B] = { - @nowarn val pf = scala.PartialFunction(f) - pf - } - -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/SameThreadExecutionContext.scala b/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/SameThreadExecutionContext.scala deleted file mode 100644 index 4f6b99be43..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/SameThreadExecutionContext.scala +++ /dev/null @@ -1,39 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.dispatch.internal - -import scala.concurrent.ExecutionContext - -import org.apache.pekko -import pekko.annotation.InternalApi -import pekko.dispatch.BatchingExecutor - -/** - * Factory to create same thread ec. Not intended to be called from any other site than to create [[pekko.dispatch.ExecutionContexts#parasitic]] - * - * INTERNAL API - */ -@InternalApi -private[dispatch] object SameThreadExecutionContext { - - private val sameThread = new ExecutionContext with BatchingExecutor { - override protected def unbatchedExecute(runnable: Runnable): Unit = runnable.run() - override protected def resubmitOnBlock: Boolean = false // No point since we execute on same thread - override def reportFailure(t: Throwable): Unit = - throw new IllegalStateException("exception in sameThreadExecutionContext", t) - } - - def apply(): ExecutionContext = sameThread - -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/ScalaBatchable.scala b/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/ScalaBatchable.scala deleted file mode 100644 index 373fa2f654..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/dispatch/internal/ScalaBatchable.scala +++ /dev/null @@ -1,32 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.dispatch.internal - -import org.apache.pekko -import pekko.annotation.InternalApi - -/** - * INTERNAL API - */ -@InternalApi -private[pekko] object ScalaBatchable { - - // see Scala 2.13 source tree for explanation - def isBatchable(runnable: Runnable): Boolean = runnable match { - case b: pekko.dispatch.Batchable => b.isBatchable - case _: scala.concurrent.OnCompleteRunnable => true - case _ => false - } - -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/ByteIterator.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/ByteIterator.scala deleted file mode 100644 index 03c536d555..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/ByteIterator.scala +++ /dev/null @@ -1,696 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.util - -import java.nio.{ ByteBuffer, ByteOrder } - -import scala.annotation.tailrec -import scala.collection.LinearSeq -import scala.collection.mutable.ListBuffer -import scala.reflect.ClassTag - -import org.apache.pekko.util.Collections.EmptyImmutableSeq - -object ByteIterator { - object ByteArrayIterator { - - protected[pekko] def apply(array: Array[Byte]): ByteArrayIterator = - new ByteArrayIterator(array, 0, array.length) - - protected[pekko] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator = - new ByteArrayIterator(array, from, until) - - val empty: ByteArrayIterator = apply(Array.emptyByteArray) - } - - class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int) - extends ByteIterator { - iterator => - - final def len: Int = until - from - - final def hasNext: Boolean = from < until - - final def head: Byte = array(from) - - final def next(): Byte = { - if (!hasNext) EmptyImmutableSeq.iterator.next() - else { - val i = from; from = from + 1; array(i) - } - } - - def clear(): Unit = { this.array = Array.emptyByteArray; from = 0; until = from } - - final override def length: Int = { val l = len; clear(); l } - - final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match { - case that: ByteIterator => - if (that.isEmpty) this - else if (this.isEmpty) that - else - that match { - case that: ByteArrayIterator => - if ((this.array eq that.array) && (this.until == that.from)) { - this.until = that.until - that.clear() - this - } else { - val result = MultiByteArrayIterator(List(this, that)) - this.clear() - result - } - case that: MultiByteArrayIterator => this ++: that - } - case _ => super.++(that) - } - - final override def clone: ByteArrayIterator = new ByteArrayIterator(array, from, until) - - final override def take(n: Int): this.type = { - if (n < len) until = { if (n > 0) from + n else from } - this - } - - final override def drop(n: Int): this.type = { - if (n > 0) from = { if (n < len) from + n else until } - this - } - - final override def takeWhile(p: Byte => Boolean): this.type = { - val prev = from - dropWhile(p) - until = from; from = prev - this - } - - final override def dropWhile(p: Byte => Boolean): this.type = { - var stop = false - while (!stop && hasNext) { - if (p(array(from))) { - from = from + 1 - } else { - stop = true - } - } - this - } - - final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = { - val n = 0 max ((xs.length - start) min this.len min len) - Array.copy(this.array, from, xs, start, n) - this.drop(n) - } - - final override def toByteString: ByteString = { - val result = - if ((from == 0) && (until == array.length)) ByteString.ByteString1C(array) - else ByteString.ByteString1(array, from, len) - clear() - result - } - - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = { - if (n <= this.len) { - System.arraycopy(this.array, this.from, xs, offset, n) - this.drop(n) - } else EmptyImmutableSeq.iterator.next() - } - - private def wrappedByteBuffer: ByteBuffer = ByteBuffer.wrap(array, from, len).asReadOnlyBuffer - - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asShortBuffer.get(xs, offset, n); drop(2 * n) - } - - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asIntBuffer.get(xs, offset, n); drop(4 * n) - } - - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asLongBuffer.get(xs, offset, n); drop(8 * n) - } - - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asFloatBuffer.get(xs, offset, n); drop(4 * n) - } - - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = { - wrappedByteBuffer.order(byteOrder).asDoubleBuffer.get(xs, offset, n); drop(8 * n) - } - - def copyToBuffer(buffer: ByteBuffer): Int = { - val copyLength = math.min(buffer.remaining, len) - if (copyLength > 0) { - buffer.put(array, from, copyLength) - drop(copyLength) - } - copyLength - } - - def asInputStream: java.io.InputStream = new java.io.InputStream { - override def available: Int = iterator.len - - def read: Int = if (hasNext) next().toInt & 0xFF else -1 - - override def read(b: Array[Byte], off: Int, len: Int): Int = { - if ((off < 0) || (len < 0) || (off + len > b.length)) throw new IndexOutOfBoundsException - if (len == 0) 0 - else if (!isEmpty) { - val nRead = math.min(available, len) - copyToArray(b, off, nRead) - nRead - } else -1 - } - - override def skip(n: Long): Long = { - val nSkip = math.min(iterator.len, n.toInt) - iterator.drop(nSkip) - nSkip - } - } - } - - object MultiByteArrayIterator { - protected val clearedList: List[ByteArrayIterator] = List(ByteArrayIterator.empty) - - val empty: MultiByteArrayIterator = new MultiByteArrayIterator(Nil) - - protected[pekko] def apply(iterators: LinearSeq[ByteArrayIterator]): MultiByteArrayIterator = - new MultiByteArrayIterator(iterators) - } - - class MultiByteArrayIterator private (private var iterators: LinearSeq[ByteArrayIterator]) extends ByteIterator { - // After normalization: - // * iterators.isEmpty == false - // * (!iterator.head.isEmpty || iterators.tail.isEmpty) == true - private def normalize(): this.type = { - @tailrec def norm(xs: LinearSeq[ByteArrayIterator]): LinearSeq[ByteArrayIterator] = { - if (xs.isEmpty) MultiByteArrayIterator.clearedList - else if (xs.head.isEmpty) norm(xs.tail) - else xs - } - iterators = norm(iterators) - this - } - normalize() - - private def current: ByteArrayIterator = iterators.head - private def dropCurrent(): Unit = { iterators = iterators.tail } - final def clear(): Unit = { iterators = MultiByteArrayIterator.empty.iterators } - - final def hasNext: Boolean = current.hasNext - - final def head: Byte = current.head - - final def next(): Byte = { - val result = current.next() - normalize() - result - } - - final override def len: Int = iterators.foldLeft(0) { _ + _.len } - - final override def length: Int = { - val result = len - clear() - result - } - - private[pekko] def ++:(that: ByteArrayIterator): this.type = { - iterators = that +: iterators - this - } - - final override def ++(that: TraversableOnce[Byte]): ByteIterator = that match { - case that: ByteIterator => - if (that.isEmpty) this - else if (this.isEmpty) that - else { - that match { - case that: ByteArrayIterator => - iterators = this.iterators :+ that - that.clear() - this - case that: MultiByteArrayIterator => - iterators = this.iterators ++ that.iterators - that.clear() - this - } - } - case _ => super.++(that) - } - - final override def clone: MultiByteArrayIterator = { - val clonedIterators: List[ByteArrayIterator] = iterators.map(_.clone)(collection.breakOut) - new MultiByteArrayIterator(clonedIterators) - } - - /** For performance sensitive code, call take() directly on ByteString (it's optimised there) */ - final override def take(n: Int): this.type = { - var rest = n - val builder = new ListBuffer[ByteArrayIterator] - while ((rest > 0) && !iterators.isEmpty) { - current.take(rest) - if (current.hasNext) { - rest -= current.len - builder += current - } - iterators = iterators.tail - } - iterators = builder.result - normalize() - } - - /** For performance sensitive code, call drop() directly on ByteString (it's optimised there) */ - final override def drop(n: Int): this.type = - if ((n > 0) && !isEmpty) { - val nCurrent = math.min(n, current.len) - current.drop(n) - val rest = n - nCurrent - assert(current.isEmpty || (rest == 0)) - normalize() - drop(rest) - } else this - - final override def takeWhile(p: Byte => Boolean): this.type = { - var stop = false - val builder = new ListBuffer[ByteArrayIterator] - while (!stop && iterators.nonEmpty) { - val lastLen = current.len - current.takeWhile(p) - if (current.hasNext) builder += current - if (current.len < lastLen) stop = true - dropCurrent() - } - iterators = builder.result - normalize() - } - - @tailrec final override def dropWhile(p: Byte => Boolean): this.type = - if (!isEmpty) { - current.dropWhile(p) - val dropMore = current.isEmpty - normalize() - if (dropMore) dropWhile(p) else this - } else this - - final override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = { - var pos = start - var rest = len - while ((rest > 0) && !iterators.isEmpty && pos < xs.length) { - val n = 0 max ((xs.length - pos) min current.len min rest) - current.copyToArray(xs, pos, n) - pos += n - rest -= n - if (current.isEmpty) { - dropCurrent() - } - } - normalize() - } - - override def foreach[@specialized U](f: Byte => U): Unit = { - iterators.foreach { _.foreach(f) } - clear() - } - - final override def toByteString: ByteString = { - if (iterators.tail.isEmpty) iterators.head.toByteString - else { - val result = iterators.foldLeft(ByteString.empty) { _ ++ _.toByteString } - clear() - result - } - } - - @tailrec protected final def getToArray[A](xs: Array[A], offset: Int, n: Int, elemSize: Int)(getSingle: => A)( - getMult: (Array[A], Int, Int) => Unit): this.type = - if (n <= 0) this - else { - if (isEmpty) EmptyImmutableSeq.iterator.next() - val nDone = if (current.len >= elemSize) { - val nCurrent = math.min(n, current.len / elemSize) - getMult(xs, offset, nCurrent) - nCurrent - } else { - xs(offset) = getSingle - 1 - } - normalize() - getToArray(xs, offset + nDone, n - nDone, elemSize)(getSingle)(getMult) - } - - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type = - getToArray(xs, offset, n, 1) { getByte } { current.getBytes(_, _, _) } - - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 2) { getShort(byteOrder) } { current.getShorts(_, _, _)(byteOrder) } - - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 4) { getInt(byteOrder) } { current.getInts(_, _, _)(byteOrder) } - - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getLong(byteOrder) } { current.getLongs(_, _, _)(byteOrder) } - - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getFloat(byteOrder) } { current.getFloats(_, _, _)(byteOrder) } - - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type = - getToArray(xs, offset, n, 8) { getDouble(byteOrder) } { current.getDoubles(_, _, _)(byteOrder) } - - /** For performance sensitive code, call copyToBuffer() directly on ByteString (it's optimised there) */ - override def copyToBuffer(buffer: ByteBuffer): Int = { - // the fold here is better than indexing into the LinearSeq - val n = iterators.foldLeft(0) { _ + _.copyToBuffer(buffer) } - normalize() - n - } - - def asInputStream: java.io.InputStream = new java.io.InputStream { - override def available: Int = current.len - - def read: Int = if (hasNext) next().toInt & 0xFF else -1 - - override def read(b: Array[Byte], off: Int, len: Int): Int = { - val nRead = current.asInputStream.read(b, off, len) - normalize() - nRead - } - - override def skip(n: Long): Long = { - @tailrec def skipImpl(n: Long, skipped: Long): Long = - if (n > 0) { - if (!isEmpty) { - val m = current.asInputStream.skip(n) - normalize() - val newN = n - m - val newSkipped = skipped + m - if (newN > 0) skipImpl(newN, newSkipped) - else newSkipped - } else 0 - } else 0 - - skipImpl(n, 0) - } - } - } -} - -/** - * An iterator over a ByteString. - */ -abstract class ByteIterator extends BufferedIterator[Byte] { - def len: Int - - def head: Byte - - def next(): Byte - - protected def clear(): Unit - - def ++(that: TraversableOnce[Byte]): ByteIterator = - if (that.isEmpty) this else ByteIterator.ByteArrayIterator(that.toArray) - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def clone: ByteIterator = - throw new UnsupportedOperationException("Method clone is not implemented in ByteIterator") - - override def duplicate: (ByteIterator, ByteIterator) = (this, clone) - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def take(n: Int): this.type = - throw new UnsupportedOperationException("Method take is not implemented in ByteIterator") - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def drop(n: Int): this.type = - throw new UnsupportedOperationException("Method drop is not implemented in ByteIterator") - - override def slice(from: Int, until: Int): this.type = { - if (from > 0) drop(from).take(until - from) - else take(until) - } - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def takeWhile(p: Byte => Boolean): this.type = - throw new UnsupportedOperationException("Method takeWhile is not implemented in ByteIterator") - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // the parent class. - override def dropWhile(p: Byte => Boolean): this.type = - throw new UnsupportedOperationException("Method dropWhile is not implemented in ByteIterator") - - override def span(p: Byte => Boolean): (ByteIterator, ByteIterator) = { - val that = clone - this.takeWhile(p) - that.drop(this.len) - (this, that) - } - - override def indexWhere(p: Byte => Boolean): Int = indexWhere(p, 0) - override def indexWhere(p: Byte => Boolean, from: Int): Int = { - var index = 0 - while (index < from && hasNext) { - next() - index += 1 - } - var found = false - while (!found && hasNext) if (p(next())) { - found = true - } else { - index += 1 - } - if (found) index else -1 - } - - def indexOf(elem: Byte): Int = indexOf(elem, 0) - def indexOf(elem: Byte, from: Int): Int = indexWhere(_ == elem, from) - - override def indexOf[B >: Byte](elem: B): Int = indexOf[B](elem, 0) - override def indexOf[B >: Byte](elem: B, from: Int): Int = indexWhere(_ == elem, from) - - def toByteString: ByteString - - override def toSeq: ByteString = toByteString - - override def foreach[@specialized U](f: Byte => U): Unit = - while (hasNext) f(next()) - - override def foldLeft[@specialized B](z: B)(op: (B, Byte) => B): B = { - var acc = z - foreach { byte => - acc = op(acc, byte) - } - acc - } - - override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = { - val target = new Array[B](len) - copyToArray(target) - target - } - - /** - * Get a single Byte from this iterator. Identical to next(). - */ - def getByte: Byte = next() - - /** - * Get a single Short from this iterator. - */ - def getShort(implicit byteOrder: ByteOrder): Short = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next() & 0xFF) << 8 | (next() & 0xFF) << 0).toShort - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next() & 0xFF) << 0 | (next() & 0xFF) << 8).toShort - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a single Int from this iterator. - */ - def getInt(implicit byteOrder: ByteOrder): Int = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next() & 0xFF) << 24 - | (next() & 0xFF) << 16 - | (next() & 0xFF) << 8 - | (next() & 0xFF) << 0) - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next() & 0xFF) << 0 - | (next() & 0xFF) << 8 - | (next() & 0xFF) << 16 - | (next() & 0xFF) << 24) - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a single Long from this iterator. - */ - def getLong(implicit byteOrder: ByteOrder): Long = { - if (byteOrder == ByteOrder.BIG_ENDIAN) - ((next().toLong & 0xFF) << 56 - | (next().toLong & 0xFF) << 48 - | (next().toLong & 0xFF) << 40 - | (next().toLong & 0xFF) << 32 - | (next().toLong & 0xFF) << 24 - | (next().toLong & 0xFF) << 16 - | (next().toLong & 0xFF) << 8 - | (next().toLong & 0xFF) << 0) - else if (byteOrder == ByteOrder.LITTLE_ENDIAN) - ((next().toLong & 0xFF) << 0 - | (next().toLong & 0xFF) << 8 - | (next().toLong & 0xFF) << 16 - | (next().toLong & 0xFF) << 24 - | (next().toLong & 0xFF) << 32 - | (next().toLong & 0xFF) << 40 - | (next().toLong & 0xFF) << 48 - | (next().toLong & 0xFF) << 56) - else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Get a Long from this iterator where only the least significant `n` - * bytes were encoded. - */ - def getLongPart(n: Int)(implicit byteOrder: ByteOrder): Long = { - if (byteOrder == ByteOrder.BIG_ENDIAN) { - var x = 0L - (1 to n).foreach(_ => x = (x << 8) | (next() & 0xFF)) - x - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - var x = 0L - (0 until n).foreach(i => x |= (next() & 0xFF) << 8 * i) - x - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - def getFloat(implicit byteOrder: ByteOrder): Float = - java.lang.Float.intBitsToFloat(getInt(byteOrder)) - - def getDouble(implicit byteOrder: ByteOrder): Double = - java.lang.Double.longBitsToDouble(getLong(byteOrder)) - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < xs.length. - */ - def getBytes(xs: Array[Byte]): this.type = getBytes(xs, 0, xs.length) - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if length < n or if (xs.length - offset) < n. - */ - def getBytes(xs: Array[Byte], offset: Int, n: Int): this.type - - /** - * Get a specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < n. - */ - def getBytes(n: Int): Array[Byte] = { - val bytes = new Array[Byte](n) - getBytes(bytes, 0, n) - bytes - } - - /** - * Get a ByteString with specific number of Bytes from this iterator. In contrast to - * copyToArray, this method will fail if this.len < n. - */ - def getByteString(n: Int): ByteString = { - val bs = clone.take(n).toByteString - drop(n) - bs - } - - /** - * Get a number of Shorts from this iterator. - */ - def getShorts(xs: Array[Short])(implicit byteOrder: ByteOrder): this.type = - getShorts(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Shorts from this iterator. - */ - def getShorts(xs: Array[Short], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Ints from this iterator. - */ - def getInts(xs: Array[Int])(implicit byteOrder: ByteOrder): this.type = - getInts(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Ints from this iterator. - */ - def getInts(xs: Array[Int], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Longs from this iterator. - */ - def getLongs(xs: Array[Long])(implicit byteOrder: ByteOrder): this.type = - getLongs(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Longs from this iterator. - */ - def getLongs(xs: Array[Long], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Floats from this iterator. - */ - def getFloats(xs: Array[Float])(implicit byteOrder: ByteOrder): this.type = - getFloats(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Floats from this iterator. - */ - def getFloats(xs: Array[Float], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Get a number of Doubles from this iterator. - */ - def getDoubles(xs: Array[Double])(implicit byteOrder: ByteOrder): this.type = - getDoubles(xs, 0, xs.length)(byteOrder) - - /** - * Get a number of Doubles from this iterator. - */ - def getDoubles(xs: Array[Double], offset: Int, n: Int)(implicit byteOrder: ByteOrder): this.type - - /** - * Copy as many bytes as possible to a ByteBuffer, starting from it's - * current position. This method will not overflow the buffer. - * - * @param buffer a ByteBuffer to copy bytes to - * @return the number of bytes actually copied - */ - /** For performance sensitive code, call take() directly on ByteString (it's optimised there) */ - def copyToBuffer(buffer: ByteBuffer): Int - - /** - * Directly wraps this ByteIterator in an InputStream without copying. - * Read and skip operations on the stream will advance the iterator - * accordingly. - */ - def asInputStream: java.io.InputStream -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/ByteString.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/ByteString.scala deleted file mode 100644 index 05b60a2cad..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/ByteString.scala +++ /dev/null @@ -1,1456 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.util - -import java.io.{ ByteArrayInputStream, InputStream, ObjectInputStream, ObjectOutputStream, SequenceInputStream } -import java.lang.{ Iterable => JIterable } -import java.nio.{ ByteBuffer, ByteOrder } -import java.nio.charset.{ Charset, StandardCharsets } -import java.util.Base64 - -import scala.annotation.{ tailrec, varargs } -import scala.collection.IndexedSeqOptimized -import scala.collection.JavaConverters._ -import scala.collection.generic.CanBuildFrom -import scala.collection.immutable -import scala.collection.immutable.{ IndexedSeq, VectorBuilder } -import scala.collection.mutable.{ Builder, WrappedArray } -import scala.reflect.ClassTag - -object ByteString { - - /** - * Creates a new ByteString by copying a byte array. - */ - def apply(bytes: Array[Byte]): ByteString = CompactByteString(bytes) - - /** - * Creates a new ByteString by copying bytes. - */ - def apply(bytes: Byte*): ByteString = CompactByteString(bytes: _*) - - /** - * Creates a new ByteString by iterating over bytes. - */ - def apply(bytes: Iterator[Byte]): ByteString = CompactByteString(bytes) - - /** - * Creates a new ByteString by converting from integral numbers to bytes. - */ - def apply[T](bytes: T*)(implicit num: Integral[T]): ByteString = - CompactByteString(bytes: _*)(num) - - /** - * Creates a new ByteString by copying bytes from a ByteBuffer. - */ - def apply(bytes: ByteBuffer): ByteString = CompactByteString(bytes) - - /** - * Creates a new ByteString by encoding a String as UTF-8. - */ - def apply(string: String): ByteString = apply(string, StandardCharsets.UTF_8) - - /** - * Creates a new ByteString by encoding a String with a charset. - */ - def apply(string: String, charset: String): ByteString = CompactByteString(string, charset) - - /** - * Creates a new ByteString by encoding a String with a charset. - */ - def apply(string: String, charset: Charset): ByteString = CompactByteString(string, charset) - - /** - * Creates a new ByteString by copying a byte array. - */ - def fromArray(array: Array[Byte]): ByteString = apply(array) - - /** - * Unsafe API: Use only in situations you are completely confident that this is what - * you need, and that you understand the implications documented below. - * - * Creates a ByteString without copying the passed in byte array, unlike other factory - * methods defined on ByteString. This method of creating a ByteString saves one array - * copy and allocation and therefore can lead to better performance, however it also means - * that one MUST NOT modify the passed in array, or unexpected immutable data structure - * contract-breaking behavior will manifest itself. - * - * This API is intended for users who have obtained an byte array from some other API, and - * want wrap it into an ByteArray, and from there on only use that reference (the ByteString) - * to operate on the wrapped data. For all other intents and purposes, please use the usual - * apply and create methods - which provide the immutability guarantees by copying the array. - */ - def fromArrayUnsafe(array: Array[Byte]): ByteString = ByteString1C(array) - - /** - * Creates a new ByteString by copying length bytes starting at offset from - * an Array. - */ - def fromArray(array: Array[Byte], offset: Int, length: Int): ByteString = - CompactByteString.fromArray(array, offset, length) - - /** - * Unsafe API: Use only in situations you are completely confident that this is what - * you need, and that you understand the implications documented below. - * - * Creates a ByteString without copying the passed in byte array, unlike other factory - * methods defined on ByteString. This method of creating a ByteString saves one array - * copy and allocation and therefore can lead to better performance, however it also means - * that one MUST NOT modify the passed in array, or unexpected immutable data structure - * contract-breaking behavior will manifest itself. - * - * This API is intended for users who have obtained an byte array from some other API, and - * want wrap it into an ByteArray, and from there on only use that reference (the ByteString) - * to operate on the wrapped data. For all other intents and purposes, please use the usual - * apply and create methods - which provide the immutability guarantees by copying the array. - */ - def fromArrayUnsafe(array: Array[Byte], offset: Int, length: Int): ByteString = ByteString1(array, offset, length) - - /** - * JAVA API - * Creates a new ByteString by copying an int array by converting from integral numbers to bytes. - */ - @varargs - def fromInts(array: Int*): ByteString = - apply(array: _*)(scala.math.Numeric.IntIsIntegral) - - /** - * Creates a new ByteString which will contain the UTF-8 representation of the given String - */ - def fromString(string: String): ByteString = apply(string) - - /** - * Creates a new ByteString which will contain the representation of the given String in the given charset - */ - def fromString(string: String, charset: String): ByteString = apply(string, charset) - - /** - * Creates a new ByteString which will contain the representation of the given String in the given charset - */ - def fromString(string: String, charset: Charset): ByteString = apply(string, charset) - - /** - * Standard "UTF-8" charset - */ - val UTF_8: String = StandardCharsets.UTF_8.name() - - /** - * Creates a new ByteString by copying bytes out of a ByteBuffer. - */ - def fromByteBuffer(buffer: ByteBuffer): ByteString = apply(buffer) - - val empty: ByteString = CompactByteString(Array.empty[Byte]) - - /** Java API */ - val emptyByteString: ByteString = empty - - def newBuilder: ByteStringBuilder = new ByteStringBuilder - - /** Java API */ - def createBuilder: ByteStringBuilder = new ByteStringBuilder - - implicit val canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] = - new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] { - def apply(ignore: TraversableOnce[Byte]): ByteStringBuilder = newBuilder - def apply(): ByteStringBuilder = newBuilder - } - - private[pekko] object ByteString1C extends Companion { - def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes(StandardCharsets.UTF_8)) - def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) - val SerializationIdentity = 1.toByte - - def readFromInputStream(is: ObjectInputStream): ByteString1C = { - val length = is.readInt() - val arr = new Array[Byte](length) - is.readFully(arr, 0, length) - ByteString1C(arr) - } - } - - /** - * A compact (unsliced) and unfragmented ByteString, implementation of ByteString1C. - */ - @SerialVersionUID(3956956327691936932L) - final class ByteString1C private (private val bytes: Array[Byte]) extends CompactByteString { - def apply(idx: Int): Byte = bytes(idx) - - override def length: Int = bytes.length - - // Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead - override def iterator: ByteIterator.ByteArrayIterator = ByteIterator.ByteArrayIterator(bytes, 0, bytes.length) - - /** INTERNAL API */ - private[pekko] def toByteString1: ByteString1 = ByteString1(bytes, 0, bytes.length) - - /** INTERNAL API */ - private[pekko] def byteStringCompanion = ByteString1C - - override def asByteBuffer: ByteBuffer = toByteString1.asByteBuffer - - override def asByteBuffers: scala.collection.immutable.Iterable[ByteBuffer] = List(asByteBuffer) - - override def decodeString(charset: String): String = - if (isEmpty) "" else new String(bytes, charset) - - override def decodeString(charset: Charset): String = - if (isEmpty) "" else new String(bytes, charset) - - override def decodeBase64: ByteString = - if (isEmpty) this else ByteString1C(Base64.getDecoder.decode(bytes)) - - override def encodeBase64: ByteString = - if (isEmpty) this else ByteString1C(Base64.getEncoder.encode(bytes)) - - override def ++(that: ByteString): ByteString = { - if (that.isEmpty) this - else if (this.isEmpty) that - else toByteString1 ++ that - } - - override def take(n: Int): ByteString = - if (n <= 0) ByteString.empty - else if (n >= length) this - else toByteString1.take(n) - - override def dropRight(n: Int): ByteString = - if (n <= 0) this - else toByteString1.dropRight(n) - - override def drop(n: Int): ByteString = - if (n <= 0) this - else toByteString1.drop(n) - - override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0) - override def indexOf[B >: Byte](elem: B, from: Int): Int = { - if (from >= length) -1 - else { - var found = -1 - var i = math.max(from, 0) - while (i < length && found == -1) { - if (bytes(i) == elem) found = i - i += 1 - } - found - } - } - - override def indexOf(elem: Byte): Int = indexOf(elem, 0) - override def indexOf(elem: Byte, from: Int): Int = { - if (from >= length) -1 - else { - var found = -1 - var i = math.max(from, 0) - while (i < length && found == -1) { - if (bytes(i) == elem) found = i - i += 1 - } - found - } - } - - override def slice(from: Int, until: Int): ByteString = - if (from <= 0 && until >= length) this - else if (from >= length || until <= 0 || from >= until) ByteString.empty - else toByteString1.slice(from, until) - - private[pekko] override def writeToOutputStream(os: ObjectOutputStream): Unit = - toByteString1.writeToOutputStream(os) - - override def copyToBuffer(buffer: ByteBuffer): Int = - writeToBuffer(buffer, offset = 0) - - /** INTERNAL API: Specialized for internal use, writing multiple ByteString1C into the same ByteBuffer. */ - private[pekko] def writeToBuffer(buffer: ByteBuffer, offset: Int): Int = { - val copyLength = Math.min(buffer.remaining, offset + length) - if (copyLength > 0) { - buffer.put(bytes, offset, copyLength) - } - copyLength - } - - /** INTERNAL API: Specialized for internal use, appending ByteString1C to a ByteStringBuilder. */ - private[pekko] def appendToBuilder(buffer: ByteStringBuilder) = { - buffer.putByteArrayUnsafe(bytes) - } - - override def toArrayUnsafe(): Array[Byte] = bytes - - override def asInputStream: InputStream = new ByteArrayInputStream(bytes) - } - - /** INTERNAL API: ByteString backed by exactly one array, with start / end markers */ - private[pekko] object ByteString1 extends Companion { - val empty: ByteString1 = new ByteString1(Array.empty[Byte]) - def fromString(s: String): ByteString1 = apply(s.getBytes(StandardCharsets.UTF_8)) - def apply(bytes: Array[Byte]): ByteString1 = apply(bytes, 0, bytes.length) - def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 = - if (length == 0) empty - else new ByteString1(bytes, Math.max(0, startIndex), Math.max(0, length)) - - val SerializationIdentity = 0.toByte - - def readFromInputStream(is: ObjectInputStream): ByteString1 = - ByteString1C.readFromInputStream(is).toByteString1 - - } - - /** - * An unfragmented ByteString. - */ - final class ByteString1 private (private val bytes: Array[Byte], private val startIndex: Int, val length: Int) - extends ByteString - with Serializable { - - private def this(bytes: Array[Byte]) = this(bytes, 0, bytes.length) - - def apply(idx: Int): Byte = bytes(checkRangeConvert(idx)) - - // Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead - override def iterator: ByteIterator.ByteArrayIterator = - ByteIterator.ByteArrayIterator(bytes, startIndex, startIndex + length) - - private def checkRangeConvert(index: Int): Int = { - if (0 <= index && length > index) - index + startIndex - else - throw new IndexOutOfBoundsException(index.toString) - } - - private[pekko] def writeToOutputStream(os: ObjectOutputStream): Unit = { - os.writeInt(length) - os.write(bytes, startIndex, length) - } - - def isCompact: Boolean = length == bytes.length - - private[pekko] def byteStringCompanion = ByteString1 - - override def dropRight(n: Int): ByteString = - dropRight1(n) - - /** INTERNAL API */ - private[pekko] def dropRight1(n: Int): ByteString1 = - if (n <= 0) this - else if (length - n <= 0) ByteString1.empty - else new ByteString1(bytes, startIndex, length - n) - - override def drop(n: Int): ByteString = - if (n <= 0) this else drop1(n) - - /** INTERNAL API */ - private[pekko] def drop1(n: Int): ByteString1 = { - val nextStartIndex = startIndex + n - if (nextStartIndex >= bytes.length) ByteString1.empty - else ByteString1(bytes, nextStartIndex, length - n) - } - - override def take(n: Int): ByteString = - if (n <= 0) ByteString.empty else take1(n) - - private[pekko] def take1(n: Int): ByteString1 = - if (n >= length) this - else ByteString1(bytes, startIndex, n) - - override def slice(from: Int, until: Int): ByteString = - drop(from).take(until - Math.max(0, from)) - - override def copyToBuffer(buffer: ByteBuffer): Int = - writeToBuffer(buffer) - - /** INTERNAL API: Specialized for internal use, writing multiple ByteString1C into the same ByteBuffer. */ - private[pekko] def writeToBuffer(buffer: ByteBuffer): Int = { - val copyLength = Math.min(buffer.remaining, length) - if (copyLength > 0) { - buffer.put(bytes, startIndex, copyLength) - } - copyLength - } - - def compact: CompactByteString = - if (isCompact) ByteString1C(bytes) else ByteString1C(toArray) - - def asByteBuffer: ByteBuffer = { - val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer - if (buffer.remaining < bytes.length) buffer.slice - else buffer - } - - def asByteBuffers: scala.collection.immutable.Iterable[ByteBuffer] = List(asByteBuffer) - - override def decodeString(charset: String): String = - if (isEmpty) "" - else new String(bytes, startIndex, length, charset) - - override def decodeString(charset: Charset): String = // avoids Charset.forName lookup in String internals - if (isEmpty) "" - else new String(bytes, startIndex, length, charset) - - override def decodeBase64: ByteString = - if (isEmpty) this - else if (isCompact) ByteString1C(Base64.getDecoder.decode(bytes)) - else { - val dst = Base64.getDecoder.decode(ByteBuffer.wrap(bytes, startIndex, length)) - if (dst.hasArray) { - if (dst.array.length == dst.remaining) ByteString1C(dst.array) - else ByteString1(dst.array, dst.arrayOffset + dst.position(), dst.remaining) - } else CompactByteString(dst) - } - - override def encodeBase64: ByteString = - if (isEmpty) this - else if (isCompact) ByteString1C(Base64.getEncoder.encode(bytes)) - else { - val dst = Base64.getEncoder.encode(ByteBuffer.wrap(bytes, startIndex, length)) - if (dst.hasArray) { - if (dst.array.length == dst.remaining) ByteString1C(dst.array) - else ByteString1(dst.array, dst.arrayOffset + dst.position(), dst.remaining) - } else CompactByteString(dst) - } - - def ++(that: ByteString): ByteString = { - if (that.isEmpty) this - else if (this.isEmpty) that - else - that match { - case b: ByteString1C => ByteStrings(this, b.toByteString1) - case b: ByteString1 => - if ((bytes eq b.bytes) && (startIndex + length == b.startIndex)) - new ByteString1(bytes, startIndex, length + b.length) - else ByteStrings(this, b) - case bs: ByteStrings => ByteStrings(this, bs) - } - } - - override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0) - override def indexOf[B >: Byte](elem: B, from: Int): Int = { - if (from >= length) -1 - else { - var found = -1 - var i = math.max(from, 0) - while (i < length && found == -1) { - if (bytes(startIndex + i) == elem) found = i - i += 1 - } - found - } - } - - override def indexOf(elem: Byte): Int = indexOf(elem, 0) - override def indexOf(elem: Byte, from: Int): Int = { - if (from >= length) -1 - else { - var found = -1 - var i = math.max(from, 0) - while (i < length && found == -1) { - if (bytes(startIndex + i) == elem) found = i - i += 1 - } - found - } - } - - protected def writeReplace(): AnyRef = new SerializationProxy(this) - - override def toArrayUnsafe(): Array[Byte] = { - if (startIndex == 0 && length == bytes.length) bytes - else toArray - } - - override def asInputStream: InputStream = - new ByteArrayInputStream(bytes, startIndex, length) - } - - private[pekko] object ByteStrings extends Companion { - def apply(bytestrings: Vector[ByteString1]): ByteString = - new ByteStrings(bytestrings, bytestrings.foldLeft(0)(_ + _.length)) - - def apply(bytestrings: Vector[ByteString1], length: Int): ByteString = new ByteStrings(bytestrings, length) - - def apply(b1: ByteString1, b2: ByteString1): ByteString = compare(b1, b2) match { - case 3 => new ByteStrings(Vector(b1, b2), b1.length + b2.length) - case 2 => b2 - case 1 => b1 - case 0 => ByteString.empty - } - - def apply(b: ByteString1, bs: ByteStrings): ByteString = compare(b, bs) match { - case 3 => new ByteStrings(b +: bs.bytestrings, bs.length + b.length) - case 2 => bs - case 1 => b - case 0 => ByteString.empty - } - - def apply(bs: ByteStrings, b: ByteString1): ByteString = compare(bs, b) match { - case 3 => new ByteStrings(bs.bytestrings :+ b, bs.length + b.length) - case 2 => b - case 1 => bs - case 0 => ByteString.empty - } - - def apply(bs1: ByteStrings, bs2: ByteStrings): ByteString = compare(bs1, bs2) match { - case 3 => new ByteStrings(bs1.bytestrings ++ bs2.bytestrings, bs1.length + bs2.length) - case 2 => bs2 - case 1 => bs1 - case 0 => ByteString.empty - } - - // 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty - def compare(b1: ByteString, b2: ByteString): Int = - if (b1.isEmpty) - if (b2.isEmpty) 0 else 2 - else if (b2.isEmpty) 1 - else 3 - - val SerializationIdentity = 2.toByte - - def readFromInputStream(is: ObjectInputStream): ByteStrings = { - val nByteStrings = is.readInt() - - val builder = new VectorBuilder[ByteString1] - var length = 0 - - builder.sizeHint(nByteStrings) - - var i = 0 - while (i < nByteStrings) { - val bs = ByteString1.readFromInputStream(is) - builder += bs - length += bs.length - i += 1 - } - - new ByteStrings(builder.result(), length) - } - } - - /** - * A ByteString with 2 or more fragments. - */ - final class ByteStrings private (private[pekko] val bytestrings: Vector[ByteString1], val length: Int) - extends ByteString - with Serializable { - if (bytestrings.isEmpty) throw new IllegalArgumentException("bytestrings must not be empty") - if (bytestrings.head.isEmpty) throw new IllegalArgumentException("bytestrings.head must not be empty") - - def apply(idx: Int): Byte = { - if (0 <= idx && idx < length) { - var pos = 0 - var seen = 0 - var frag = bytestrings(pos) - while (idx >= seen + frag.length) { - seen += frag.length - pos += 1 - frag = bytestrings(pos) - } - frag(idx - seen) - } else throw new IndexOutOfBoundsException(idx.toString) - } - - /** Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead */ - override def iterator: ByteIterator.MultiByteArrayIterator = - ByteIterator.MultiByteArrayIterator(bytestrings.toStream.map { _.iterator }) - - def ++(that: ByteString): ByteString = { - if (that.isEmpty) this - else if (this.isEmpty) that - else - that match { - case b: ByteString1C => ByteStrings(this, b.toByteString1) - case b: ByteString1 => ByteStrings(this, b) - case bs: ByteStrings => ByteStrings(this, bs) - } - } - - private[pekko] def byteStringCompanion = ByteStrings - - def isCompact: Boolean = if (bytestrings.length == 1) bytestrings.head.isCompact else false - - override def copyToBuffer(buffer: ByteBuffer): Int = { - @tailrec def copyItToTheBuffer(buffer: ByteBuffer, i: Int, written: Int): Int = - if (i < bytestrings.length) copyItToTheBuffer(buffer, i + 1, written + bytestrings(i).writeToBuffer(buffer)) - else written - - copyItToTheBuffer(buffer, 0, 0) - } - - def compact: CompactByteString = { - if (isCompact) bytestrings.head.compact - else { - val ar = new Array[Byte](length) - var pos = 0 - bytestrings.foreach { b => - b.copyToArray(ar, pos, b.length) - pos += b.length - } - ByteString1C(ar) - } - } - - def asByteBuffer: ByteBuffer = compact.asByteBuffer - - def asByteBuffers: scala.collection.immutable.Iterable[ByteBuffer] = bytestrings.map { _.asByteBuffer } - - override def asInputStream: InputStream = - new SequenceInputStream(bytestrings.iterator.map(_.asInputStream).asJavaEnumeration) - - def decodeString(charset: String): String = compact.decodeString(charset) - - def decodeString(charset: Charset): String = compact.decodeString(charset) - - override def decodeBase64: ByteString = compact.decodeBase64 - - override def encodeBase64: ByteString = compact.encodeBase64 - - private[pekko] def writeToOutputStream(os: ObjectOutputStream): Unit = { - os.writeInt(bytestrings.length) - bytestrings.foreach(_.writeToOutputStream(os)) - } - - override def take(n: Int): ByteString = - if (n <= 0) ByteString.empty - else if (n >= length) this - else take0(n) - - private[pekko] def take0(n: Int): ByteString = { - @tailrec def go(last: Int, restToTake: Int): (Int, Int) = { - val bs = bytestrings(last) - if (bs.length > restToTake) (last, restToTake) - else go(last + 1, restToTake - bs.length) - } - - val (last, restToTake) = go(0, n) - - if (last == 0) bytestrings(last).take(restToTake) - else if (restToTake == 0) new ByteStrings(bytestrings.take(last), n) - else new ByteStrings(bytestrings.take(last) :+ bytestrings(last).take1(restToTake), n) - } - - override def dropRight(n: Int): ByteString = - if (0 < n && n < length) dropRight0(n) - else if (n >= length) ByteString.empty - else this - - private def dropRight0(n: Int): ByteString = { - val byteStringsSize = bytestrings.length - @tailrec def dropRightWithFullDropsAndRemainig(fullDrops: Int, remainingToDrop: Int): ByteString = { - val bs = bytestrings(byteStringsSize - fullDrops - 1) - if (bs.length > remainingToDrop) { - if (fullDrops == byteStringsSize - 1) - bytestrings(0).dropRight(remainingToDrop) - else if (remainingToDrop == 0) - new ByteStrings(bytestrings.dropRight(fullDrops), length - n) - else - new ByteStrings( - bytestrings.dropRight(fullDrops + 1) :+ bytestrings(byteStringsSize - fullDrops - 1) - .dropRight1(remainingToDrop), - length - n) - } else { - dropRightWithFullDropsAndRemainig(fullDrops + 1, remainingToDrop - bs.length) - } - } - - dropRightWithFullDropsAndRemainig(0, n) - } - - override def slice(from: Int, until: Int): ByteString = - if (from <= 0 && until >= length) this - else if (from > length || until <= from) ByteString.empty - else drop(from).dropRight(length - until) - - override def drop(n: Int): ByteString = - if (n <= 0) this - else if (n >= length) ByteString.empty - else drop0(n) - - private def drop0(n: Int): ByteString = { - // impl note: could be optimised a bit by using VectorIterator instead, - // however then we're forced to call .toVector which halfs performance - // We can work around that, as there's a Scala private method "remainingVector" which is fast, - // but let's not go into calling private APIs here just yet. - @tailrec def findSplit(fullDrops: Int, remainingToDrop: Int): (Int, Int) = { - val bs = bytestrings(fullDrops) - if (bs.length > remainingToDrop) (fullDrops, remainingToDrop) - else findSplit(fullDrops + 1, remainingToDrop - bs.length) - } - - val (fullDrops, remainingToDrop) = findSplit(0, n) - - if (remainingToDrop == 0) - new ByteStrings(bytestrings.drop(fullDrops), length - n) - else if (fullDrops == bytestrings.length - 1) - bytestrings(fullDrops).drop(remainingToDrop) - else - new ByteStrings(bytestrings(fullDrops).drop1(remainingToDrop) +: bytestrings.drop(fullDrops + 1), length - n) - } - - override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0) - override def indexOf[B >: Byte](elem: B, from: Int): Int = { - if (from >= length) -1 - else { - val byteStringsSize = bytestrings.size - - @tailrec - def find(bsIdx: Int, relativeIndex: Int, bytesPassed: Int): Int = { - if (bsIdx >= byteStringsSize) -1 - else { - val bs = bytestrings(bsIdx) - - if (bs.length <= relativeIndex) { - find(bsIdx + 1, relativeIndex - bs.length, bytesPassed + bs.length) - } else { - val subIndexOf = bs.indexOf(elem, relativeIndex) - if (subIndexOf < 0) { - find(bsIdx + 1, relativeIndex - bs.length, bytesPassed + bs.length) - } else subIndexOf + bytesPassed - } - } - } - - find(0, math.max(from, 0), 0) - } - } - - override def indexOf(elem: Byte): Int = indexOf(elem, 0) - override def indexOf(elem: Byte, from: Int): Int = { - if (from >= length) -1 - else { - val byteStringsSize = bytestrings.size - - @tailrec - def find(bsIdx: Int, relativeIndex: Int, bytesPassed: Int): Int = { - if (bsIdx >= byteStringsSize) -1 - else { - val bs = bytestrings(bsIdx) - - if (bs.length <= relativeIndex) { - find(bsIdx + 1, relativeIndex - bs.length, bytesPassed + bs.length) - } else { - val subIndexOf = bs.indexOf(elem, relativeIndex) - if (subIndexOf < 0) { - find(bsIdx + 1, relativeIndex - bs.length, bytesPassed + bs.length) - } else subIndexOf + bytesPassed - } - } - } - - find(0, math.max(from, 0), 0) - } - } - - protected def writeReplace(): AnyRef = new SerializationProxy(this) - } - - @SerialVersionUID(1L) - private class SerializationProxy(@transient private var orig: ByteString) extends Serializable { - private def writeObject(out: ObjectOutputStream): Unit = { - out.writeByte(orig.byteStringCompanion.SerializationIdentity) - orig.writeToOutputStream(out) - } - - private def readObject(in: ObjectInputStream): Unit = { - val serializationId = in.readByte() - - orig = Companion(from = serializationId).readFromInputStream(in) - } - - private def readResolve(): AnyRef = orig - } - - private[pekko] object Companion { - private val companionMap = Seq(ByteString1, ByteString1C, ByteStrings) - .map(x => x.SerializationIdentity -> x) - .toMap - .withDefault(x => throw new IllegalArgumentException("Invalid serialization id " + x)) - - def apply(from: Byte): Companion = companionMap(from) - } - - private[pekko] sealed trait Companion { - def SerializationIdentity: Byte - def readFromInputStream(is: ObjectInputStream): ByteString - } -} - -/** - * A rope-like immutable data structure containing bytes. - * The goal of this structure is to reduce copying of arrays - * when concatenating and slicing sequences of bytes, - * and also providing a thread safe way of working with bytes. - * - * TODO: Add performance characteristics - */ -sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimized[Byte, ByteString] { - def apply(idx: Int): Byte - private[pekko] def byteStringCompanion: ByteString.Companion - // override so that toString will also be `ByteString(...)` for the concrete subclasses - // of ByteString which changed for Scala 2.12, see https://github.com/akka/akka/issues/21774 - override final def stringPrefix: String = "ByteString" - - override protected[this] def newBuilder: ByteStringBuilder = ByteString.newBuilder - - override def isEmpty: Boolean = length == 0 - - // *must* be overridden by derived classes. This construction is necessary - // to specialize the return type, as the method is already implemented in - // a parent trait. - // - // Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead - override def iterator: ByteIterator = - throw new UnsupportedOperationException("Method iterator is not implemented in ByteString") - - override def head: Byte = apply(0) - override def tail: ByteString = drop(1) - override def last: Byte = apply(length - 1) - override def init: ByteString = dropRight(1) - - // *must* be overridden by derived classes. - override def take(n: Int): ByteString = - throw new UnsupportedOperationException("Method take is not implemented in ByteString") - override def takeRight(n: Int): ByteString = slice(length - n, length) - - // these methods are optimized in derived classes utilising the maximum knowlage about data layout available to them: - // *must* be overridden by derived classes. - override def slice(from: Int, until: Int): ByteString = - throw new UnsupportedOperationException("Method slice is not implemented in ByteString") - - // *must* be overridden by derived classes. - override def drop(n: Int): ByteString = - throw new UnsupportedOperationException("Method drop is not implemented in ByteString") - - // *must* be overridden by derived classes. - override def dropRight(n: Int): ByteString = - throw new UnsupportedOperationException("Method dropRight is not implemented in ByteString") - - override def takeWhile(p: Byte => Boolean): ByteString = iterator.takeWhile(p).toByteString - override def dropWhile(p: Byte => Boolean): ByteString = iterator.dropWhile(p).toByteString - override def span(p: Byte => Boolean): (ByteString, ByteString) = { - val (a, b) = iterator.span(p); (a.toByteString, b.toByteString) - } - - override def splitAt(n: Int): (ByteString, ByteString) = (take(n), drop(n)) - - override def indexWhere(p: Byte => Boolean): Int = iterator.indexWhere(p) - - // optimized in subclasses - override def indexOf[B >: Byte](elem: B): Int = indexOf[B](elem, 0) - - // optimized version of indexOf for bytes, implemented in subclasses - /** - * Finds index of first occurrence of some byte in this ByteString after or at some start index. - * - * Similar to indexOf, but it avoids boxing if the value is already a byte. - * - * @param elem the element value to search for. - * @param from the start index - * @return the index `>= from` of the first element of this ByteString that is equal (as determined by `==`) - * to `elem`, or `-1`, if none exists. - * @since 1.1.0 - */ - def indexOf(elem: Byte, from: Int): Int = indexOf[Byte](elem, from) - - /** - * Finds index of first occurrence of some byte in this ByteString. - * - * Similar to indexOf, but it avoids boxing if the value is already a byte. - * - * @param elem the element value to search for. - * @return the index `>= from` of the first element of this ByteString that is equal (as determined by `==`) - * to `elem`, or `-1`, if none exists. - * @since 1.1.0 - */ - def indexOf(elem: Byte): Int = indexOf(elem, 0) - - override def grouped(size: Int): Iterator[ByteString] = { - if (size <= 0) { - throw new IllegalArgumentException(s"size=$size must be positive") - } - - Iterator.iterate(this)(_.drop(size)).takeWhile(_.nonEmpty).map(_.take(size)) - } - - override def toString(): String = { - val maxSize = 100 - if (size > maxSize) - take(maxSize).toString + s"... and [${size - maxSize}] more" - else - super.toString - } - - /** - * Java API: copy this ByteString into a fresh byte array - * - * @return this ByteString copied into a byte array - */ - protected[ByteString] def toArray: Array[Byte] = toArray[Byte] - - override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = iterator.toArray - override def copyToArray[B >: Byte](xs: Array[B], start: Int, len: Int): Unit = - iterator.copyToArray(xs, start, len) - - /** - * Unsafe API: Use only in situations you are completely confident that this is what - * you need, and that you understand the implications documented below. - * - * If the ByteString is backed by a single array it is returned without any copy. If it is backed by a rope - * of multiple ByteString instances a new array will be allocated and the contents will be copied - * into it before returning it. - * - * This method of exposing the bytes of a ByteString can save one array - * copy and allocation in the happy path scenario which can lead to better performance, - * however it also means that one MUST NOT modify the returned array, or unexpected - * immutable data structure contract-breaking behavior will manifest itself. - * - * This API is intended for users who need to pass the byte array to some other API, which will - * only read the bytes and never mutate then. For all other intents and purposes, please use the usual - * toArray method - which provide the immutability guarantees by copying the backing array. - */ - def toArrayUnsafe(): Array[Byte] = toArray - - /** - * Return the bytes in this ByteString as an InputStream. - * - * @return the bytes in this ByteString accessible as an InputStream - * @see [[asByteBuffer]] - * @since 1.1.0 - */ - def asInputStream: InputStream - - override def foreach[@specialized U](f: Byte => U): Unit = iterator.foreach(f) - - private[pekko] def writeToOutputStream(os: ObjectOutputStream): Unit - - /** - * Efficiently concatenate another ByteString. - */ - def ++(that: ByteString): ByteString - - /** - * Java API: efficiently concatenate another ByteString. - */ - def concat(that: ByteString): ByteString = this ++ that - - /** - * Copy as many bytes as possible to a ByteBuffer, starting from it's - * current position. This method will not overflow the buffer. - * - * @param buffer a ByteBuffer to copy bytes to - * @return the number of bytes actually copied - */ - // *must* be overridden by derived classes. - def copyToBuffer(buffer: ByteBuffer): Int = - throw new UnsupportedOperationException( - s"Method copyToBuffer is not implemented in ByteString, failed for buffer $buffer") - - /** - * Create a new ByteString with all contents compacted into a single, - * full byte array. - * If isCompact returns true, compact is an O(1) operation, but - * might return a different object with an optimized implementation. - */ - def compact: CompactByteString - - /** - * Check whether this ByteString is compact in memory. - * If the ByteString is compact, it might, however, not be represented - * by an object that takes full advantage of that fact. Use compact to - * get such an object. - */ - def isCompact: Boolean - - /** - * Returns a read-only ByteBuffer that directly wraps this ByteString - * if it is not fragmented. - */ - def asByteBuffer: ByteBuffer - - /** - * Scala API: Returns an immutable Iterable of read-only ByteBuffers that directly wraps this ByteStrings - * all fragments. Will always have at least one entry. - */ - def asByteBuffers: immutable.Iterable[ByteBuffer] - - /** - * Java API: Returns an Iterable of read-only ByteBuffers that directly wraps this ByteStrings - * all fragments. Will always have at least one entry. - */ - def getByteBuffers(): JIterable[ByteBuffer] = { - asByteBuffers.asJava - } - - /** - * Creates a new ByteBuffer with a copy of all bytes contained in this - * ByteString. - */ - def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray) - - /** - * Decodes this ByteString as a UTF-8 encoded String. - */ - final def utf8String: String = decodeString(StandardCharsets.UTF_8) - - /** - * Decodes this ByteString using a charset to produce a String. - * If you have a [[Charset]] instance available, use `decodeString(charset: java.nio.charset.Charset` instead. - */ - def decodeString(charset: String): String - - /** - * Decodes this ByteString using a charset to produce a String. - * Avoids Charset.forName lookup in String internals, thus is preferable to `decodeString(charset: String)`. - */ - def decodeString(charset: Charset): String - - /** - * Returns a ByteString which is the binary representation of this ByteString - * if this ByteString is Base64-encoded. - */ - def decodeBase64: ByteString - - /** - * Returns a ByteString which is the Base64 representation of this ByteString - */ - def encodeBase64: ByteString - - /** - * map method that will automatically cast Int back into Byte. - */ - final def mapI(f: Byte => Int): ByteString = map(f.andThen(_.toByte)) -} - -object CompactByteString { - - /** - * Creates a new CompactByteString by copying a byte array. - */ - def apply(bytes: Array[Byte]): CompactByteString = - if (bytes.isEmpty) empty else ByteString.ByteString1C(bytes.clone) - - /** - * Creates a new CompactByteString by copying bytes. - */ - def apply(bytes: Byte*): CompactByteString = { - if (bytes.isEmpty) empty - else { - val ar = new Array[Byte](bytes.size) - bytes.copyToArray(ar) - ByteString.ByteString1C(ar) - } - } - - /** - * Creates a new CompactByteString by traversing bytes. - */ - def apply(bytes: Iterator[Byte]): CompactByteString = { - if (bytes.isEmpty) empty - else ByteString.ByteString1C(bytes.toArray) - } - - /** - * Creates a new CompactByteString by converting from integral numbers to bytes. - */ - def apply[T](bytes: T*)(implicit num: Integral[T]): CompactByteString = { - if (bytes.isEmpty) empty - else ByteString.ByteString1C(bytes.map(x => num.toInt(x).toByte)(collection.breakOut)) - } - - /** - * Creates a new CompactByteString by copying bytes from a ByteBuffer. - */ - def apply(bytes: ByteBuffer): CompactByteString = { - if (bytes.remaining < 1) empty - else { - val ar = new Array[Byte](bytes.remaining) - bytes.get(ar) - ByteString.ByteString1C(ar) - } - } - - /** - * Creates a new CompactByteString by encoding a String as UTF-8. - */ - def apply(string: String): CompactByteString = apply(string, StandardCharsets.UTF_8) - - /** - * Creates a new CompactByteString by encoding a String with a charset. - */ - def apply(string: String, charset: String): CompactByteString = - if (string.isEmpty) empty else ByteString.ByteString1C(string.getBytes(charset)) - - /** - * Creates a new CompactByteString by encoding a String with a charset. - */ - def apply(string: String, charset: Charset): CompactByteString = - if (string.isEmpty) empty else ByteString.ByteString1C(string.getBytes(charset)) - - /** - * Creates a new CompactByteString by copying length bytes starting at offset from - * an Array. - */ - def fromArray(array: Array[Byte], offset: Int, length: Int): CompactByteString = { - val copyOffset = Math.max(offset, 0) - val copyLength = Math.max(Math.min(array.length - copyOffset, length), 0) - if (copyLength == 0) empty - else { - val copyArray = new Array[Byte](copyLength) - System.arraycopy(array, copyOffset, copyArray, 0, copyLength) - ByteString.ByteString1C(copyArray) - } - } - - val empty: CompactByteString = ByteString.ByteString1C(Array.empty[Byte]) -} - -/** - * A compact ByteString. - * - * The ByteString is guarantied to be contiguous in memory and to use only - * as much memory as required for its contents. - */ -sealed abstract class CompactByteString extends ByteString with Serializable { - def isCompact: Boolean = true - def compact: this.type = this -} - -/** - * A mutable builder for efficiently creating a [[org.apache.pekko.util.ByteString]]. - * - * The created ByteString is not automatically compacted. - */ -final class ByteStringBuilder extends Builder[Byte, ByteString] { - builder => - - import ByteString.{ ByteString1, ByteString1C, ByteStrings } - private var _length: Int = 0 - private val _builder: VectorBuilder[ByteString1] = new VectorBuilder[ByteString1]() - private var _temp: Array[Byte] = _ - private var _tempLength: Int = 0 - private var _tempCapacity: Int = 0 - - protected def fillArray(len: Int)(fill: (Array[Byte], Int) => Unit): this.type = { - ensureTempSize(_tempLength + len) - fill(_temp, _tempLength) - _tempLength += len - _length += len - this - } - - protected final def fillByteBuffer(len: Int, byteOrder: ByteOrder)(fill: ByteBuffer => Unit): this.type = { - fillArray(len) { - case (array, start) => - val buffer = ByteBuffer.wrap(array, start, len) - buffer.order(byteOrder) - fill(buffer) - } - } - - def length: Int = _length - - override def sizeHint(len: Int): Unit = { - resizeTemp(len - (_length - _tempLength)) - } - - private def clearTemp(): Unit = { - if (_tempLength > 0) { - val arr = new Array[Byte](_tempLength) - System.arraycopy(_temp, 0, arr, 0, _tempLength) - _builder += ByteString1(arr) - _tempLength = 0 - } - } - - private def resizeTemp(size: Int): Unit = { - val newtemp = new Array[Byte](size) - if (_tempLength > 0) System.arraycopy(_temp, 0, newtemp, 0, _tempLength) - _temp = newtemp - _tempCapacity = _temp.length - } - - private def shouldResizeTempFor(size: Int): Boolean = _tempCapacity < size || _tempCapacity == 0 - - private def ensureTempSize(size: Int): Unit = { - if (shouldResizeTempFor(size)) { - var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2 - while (newSize < size) newSize *= 2 - resizeTemp(newSize) - } - } - - def +=(elem: Byte): this.type = { - ensureTempSize(_tempLength + 1) - _temp(_tempLength) = elem - _tempLength += 1 - _length += 1 - this - } - def ++=(bytes: ByteString): this.type = { - if (bytes.nonEmpty) { - clearTemp() - bytes match { - case b: ByteString1C => - _builder += b.toByteString1 - _length += b.length - case b: ByteString1 => - _builder += b - _length += b.length - case bs: ByteStrings => - _builder ++= bs.bytestrings - _length += bs.length - } - } - this - } - override def ++=(xs: TraversableOnce[Byte]): this.type = { - xs match { - case bs: ByteString => ++=(bs) - case xs: WrappedArray.ofByte => - if (xs.nonEmpty) putByteArrayUnsafe(xs.array.clone) - case seq: collection.IndexedSeq[Byte] if shouldResizeTempFor(seq.length) => - if (seq.nonEmpty) { - val copied = new Array[Byte](seq.length) - seq.copyToArray(copied) - - clearTemp() - _builder += ByteString.ByteString1(copied) - _length += seq.length - } - case seq: collection.IndexedSeq[_] => - if (seq.nonEmpty) { - ensureTempSize(_tempLength + xs.size) - xs.copyToArray(_temp, _tempLength) - _tempLength += seq.length - _length += seq.length - } - case _ => - super.++=(xs) - } - this - } - - private[pekko] def putByteArrayUnsafe(xs: Array[Byte]): this.type = { - clearTemp() - _builder += ByteString1(xs) - _length += xs.length - this - } - - /** - * Java API: append a ByteString to this builder. - */ - def append(bs: ByteString): this.type = if (bs.isEmpty) this else this ++= bs - - /** - * Add a single Byte to this builder. - */ - def putByte(x: Byte): this.type = this += x - - /** - * Add a single Short to this builder. - */ - def putShort(x: Int)(implicit byteOrder: ByteOrder): this.type = { - if (byteOrder == ByteOrder.BIG_ENDIAN) { - this += (x >>> 8).toByte - this += (x >>> 0).toByte - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - this += (x >>> 0).toByte - this += (x >>> 8).toByte - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - - /** - * Add a single Int to this builder. - */ - def putInt(x: Int)(implicit byteOrder: ByteOrder): this.type = { - fillArray(4) { (target, offset) => - if (byteOrder == ByteOrder.BIG_ENDIAN) { - target(offset + 0) = (x >>> 24).toByte - target(offset + 1) = (x >>> 16).toByte - target(offset + 2) = (x >>> 8).toByte - target(offset + 3) = (x >>> 0).toByte - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - target(offset + 0) = (x >>> 0).toByte - target(offset + 1) = (x >>> 8).toByte - target(offset + 2) = (x >>> 16).toByte - target(offset + 3) = (x >>> 24).toByte - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - this - } - - /** - * Add a single Long to this builder. - */ - def putLong(x: Long)(implicit byteOrder: ByteOrder): this.type = { - fillArray(8) { (target, offset) => - if (byteOrder == ByteOrder.BIG_ENDIAN) { - target(offset + 0) = (x >>> 56).toByte - target(offset + 1) = (x >>> 48).toByte - target(offset + 2) = (x >>> 40).toByte - target(offset + 3) = (x >>> 32).toByte - target(offset + 4) = (x >>> 24).toByte - target(offset + 5) = (x >>> 16).toByte - target(offset + 6) = (x >>> 8).toByte - target(offset + 7) = (x >>> 0).toByte - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - target(offset + 0) = (x >>> 0).toByte - target(offset + 1) = (x >>> 8).toByte - target(offset + 2) = (x >>> 16).toByte - target(offset + 3) = (x >>> 24).toByte - target(offset + 4) = (x >>> 32).toByte - target(offset + 5) = (x >>> 40).toByte - target(offset + 6) = (x >>> 48).toByte - target(offset + 7) = (x >>> 56).toByte - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - this - } - - /** - * Add the `n` least significant bytes of the given Long to this builder. - */ - def putLongPart(x: Long, n: Int)(implicit byteOrder: ByteOrder): this.type = { - fillArray(n) { (target, offset) => - if (byteOrder == ByteOrder.BIG_ENDIAN) { - val start = n * 8 - 8 - (0 until n).foreach { i => - target(offset + i) = (x >>> start - 8 * i).toByte - } - } else if (byteOrder == ByteOrder.LITTLE_ENDIAN) { - (0 until n).foreach { i => - target(offset + i) = (x >>> 8 * i).toByte - } - } else throw new IllegalArgumentException("Unknown byte order " + byteOrder) - } - } - - /** - * Add a single Float to this builder. - */ - def putFloat(x: Float)(implicit byteOrder: ByteOrder): this.type = - putInt(java.lang.Float.floatToRawIntBits(x))(byteOrder) - - /** - * Add a single Double to this builder. - */ - def putDouble(x: Double)(implicit byteOrder: ByteOrder): this.type = - putLong(java.lang.Double.doubleToRawLongBits(x))(byteOrder) - - /** - * Add a number of Bytes from an array to this builder. - */ - def putBytes(array: Array[Byte]): this.type = - putBytes(array, 0, array.length) - - /** - * Add a number of Bytes from an array to this builder. - */ - def putBytes(array: Array[Byte], start: Int, len: Int): this.type = - fillArray(len) { case (target, targetOffset) => System.arraycopy(array, start, target, targetOffset, len) } - - /** - * Add a number of Shorts from an array to this builder. - */ - def putShorts(array: Array[Short])(implicit byteOrder: ByteOrder): this.type = - putShorts(array, 0, array.length)(byteOrder) - - /** - * Add a number of Shorts from an array to this builder. - */ - def putShorts(array: Array[Short], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type = - fillByteBuffer(len * 2, byteOrder) { _.asShortBuffer.put(array, start, len) } - - /** - * Add a number of Ints from an array to this builder. - */ - def putInts(array: Array[Int])(implicit byteOrder: ByteOrder): this.type = - putInts(array, 0, array.length)(byteOrder) - - /** - * Add a number of Ints from an array to this builder. - */ - def putInts(array: Array[Int], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type = - fillByteBuffer(len * 4, byteOrder) { _.asIntBuffer.put(array, start, len) } - - /** - * Add a number of Longs from an array to this builder. - */ - def putLongs(array: Array[Long])(implicit byteOrder: ByteOrder): this.type = - putLongs(array, 0, array.length)(byteOrder) - - /** - * Add a number of Longs from an array to this builder. - */ - def putLongs(array: Array[Long], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type = - fillByteBuffer(len * 8, byteOrder) { _.asLongBuffer.put(array, start, len) } - - /** - * Add a number of Floats from an array to this builder. - */ - def putFloats(array: Array[Float])(implicit byteOrder: ByteOrder): this.type = - putFloats(array, 0, array.length)(byteOrder) - - /** - * Add a number of Floats from an array to this builder. - */ - def putFloats(array: Array[Float], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type = - fillByteBuffer(len * 4, byteOrder) { _.asFloatBuffer.put(array, start, len) } - - /** - * Add a number of Doubles from an array to this builder. - */ - def putDoubles(array: Array[Double])(implicit byteOrder: ByteOrder): this.type = - putDoubles(array, 0, array.length)(byteOrder) - - /** - * Add a number of Doubles from an array to this builder. - */ - def putDoubles(array: Array[Double], start: Int, len: Int)(implicit byteOrder: ByteOrder): this.type = - fillByteBuffer(len * 8, byteOrder) { _.asDoubleBuffer.put(array, start, len) } - - def clear(): Unit = { - _builder.clear() - _length = 0 - _tempLength = 0 - } - - def result: ByteString = - if (_length == 0) ByteString.empty - else { - clearTemp() - val bytestrings = _builder.result - if (bytestrings.size == 1) - bytestrings.head - else - ByteStrings(bytestrings, _length) - } - - /** - * Directly wraps this ByteStringBuilder in an OutputStream. Write - * operations on the stream are forwarded to the builder. - */ - def asOutputStream: java.io.OutputStream = new java.io.OutputStream { - override def write(b: Int): Unit = builder += b.toByte - override def write(b: Array[Byte], off: Int, len: Int): Unit = { builder.putBytes(b, off, len) } - } - - /** - * Tests whether this ByteStringBuilder is empty. - */ - def isEmpty: Boolean = _length == 0 - - /** - * Tests whether this ByteStringBuilder is not empty. - */ - def nonEmpty: Boolean = _length > 0 -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/FunctionConverters.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/FunctionConverters.scala deleted file mode 100644 index 70c948950d..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/FunctionConverters.scala +++ /dev/null @@ -1,562 +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 scala.compat.java8 - -import scala.language.implicitConversions -import scala.annotation.nowarn - -/** - * INTERNAL API - * - * Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib. - * - * DO NOT edit this file manually, its copied over from scala-java8-compat. More specifically - * scala-java8-compat generates this source from a template, so you have to do +compile in - * scala-java8-compat and check the src_managed folder in target and then add - * `@nowarn("msg=never used")` to functions as neccessary. - */ -@InternalStableApi -private[pekko] object FunctionConverters extends java8.Priority1FunctionConverters { - import java8.functionConverterImpls._ - - @inline def asScalaFromBiConsumer[T, U](jf: java.util.function.BiConsumer[T, U]): scala.Function2[T, U, Unit] = - new FromJavaBiConsumer[T, U](jf) - - @inline def asJavaBiConsumer[T, U](sf: scala.Function2[T, U, Unit]): java.util.function.BiConsumer[T, U] = - new AsJavaBiConsumer[T, U](sf) - - @inline def asScalaFromBiFunction[T, U, R](jf: java.util.function.BiFunction[T, U, R]): scala.Function2[T, U, R] = - new FromJavaBiFunction[T, U, R](jf) - - @inline def asJavaBiFunction[T, U, R](sf: scala.Function2[T, U, R]): java.util.function.BiFunction[T, U, R] = - new AsJavaBiFunction[T, U, R](sf) - - @inline def asScalaFromBiPredicate[T, U](jf: java.util.function.BiPredicate[T, U]): scala.Function2[T, U, Boolean] = - new FromJavaBiPredicate[T, U](jf) - - @inline def asJavaBiPredicate[T, U](sf: scala.Function2[T, U, Boolean]): java.util.function.BiPredicate[T, U] = - new AsJavaBiPredicate[T, U](sf) - - @inline def asScalaFromBinaryOperator[T](jf: java.util.function.BinaryOperator[T]): scala.Function2[T, T, T] = - new FromJavaBinaryOperator[T](jf) - - @inline def asJavaBinaryOperator[T](sf: scala.Function2[T, T, T]): java.util.function.BinaryOperator[T] = - new AsJavaBinaryOperator[T](sf) - - @inline def asScalaFromBooleanSupplier(jf: java.util.function.BooleanSupplier): scala.Function0[Boolean] = - new FromJavaBooleanSupplier(jf) - - @inline def asJavaBooleanSupplier(sf: scala.Function0[Boolean]): java.util.function.BooleanSupplier = - new AsJavaBooleanSupplier(sf) - - @inline def asScalaFromConsumer[T](jf: java.util.function.Consumer[T]): scala.Function1[T, Unit] = - new FromJavaConsumer[T](jf) - - @inline def asJavaConsumer[T](sf: scala.Function1[T, Unit]): java.util.function.Consumer[T] = - new AsJavaConsumer[T](sf) - - @inline def asScalaFromDoubleBinaryOperator( - jf: java.util.function.DoubleBinaryOperator): scala.Function2[Double, Double, Double] = - new FromJavaDoubleBinaryOperator(jf) - - @inline def asJavaDoubleBinaryOperator( - sf: scala.Function2[Double, Double, Double]): java.util.function.DoubleBinaryOperator = - new AsJavaDoubleBinaryOperator(sf) - - @inline def asScalaFromDoubleConsumer(jf: java.util.function.DoubleConsumer): scala.Function1[Double, Unit] = - new FromJavaDoubleConsumer(jf) - - @inline def asJavaDoubleConsumer(sf: scala.Function1[Double, Unit]): java.util.function.DoubleConsumer = - new AsJavaDoubleConsumer(sf) - - @inline def asScalaFromDoubleFunction[R](jf: java.util.function.DoubleFunction[R]): scala.Function1[Double, R] = - new FromJavaDoubleFunction[R](jf) - - @inline def asJavaDoubleFunction[R](sf: scala.Function1[Double, R]): java.util.function.DoubleFunction[R] = - new AsJavaDoubleFunction[R](sf) - - @inline def asScalaFromDoublePredicate(jf: java.util.function.DoublePredicate): scala.Function1[Double, Boolean] = - new FromJavaDoublePredicate(jf) - - @inline def asJavaDoublePredicate(sf: scala.Function1[Double, Boolean]): java.util.function.DoublePredicate = - new AsJavaDoublePredicate(sf) - - @inline def asScalaFromDoubleSupplier(jf: java.util.function.DoubleSupplier): scala.Function0[Double] = - new FromJavaDoubleSupplier(jf) - - @inline def asJavaDoubleSupplier(sf: scala.Function0[Double]): java.util.function.DoubleSupplier = - new AsJavaDoubleSupplier(sf) - - @inline def asScalaFromDoubleToIntFunction(jf: java.util.function.DoubleToIntFunction): scala.Function1[Double, Int] = - new FromJavaDoubleToIntFunction(jf) - - @inline def asJavaDoubleToIntFunction(sf: scala.Function1[Double, Int]): java.util.function.DoubleToIntFunction = - new AsJavaDoubleToIntFunction(sf) - - @inline def asScalaFromDoubleToLongFunction( - jf: java.util.function.DoubleToLongFunction): scala.Function1[Double, Long] = new FromJavaDoubleToLongFunction(jf) - - @inline def asJavaDoubleToLongFunction(sf: scala.Function1[Double, Long]): java.util.function.DoubleToLongFunction = - new AsJavaDoubleToLongFunction(sf) - - @inline def asScalaFromDoubleUnaryOperator( - jf: java.util.function.DoubleUnaryOperator): scala.Function1[Double, Double] = new FromJavaDoubleUnaryOperator(jf) - - @inline def asJavaDoubleUnaryOperator(sf: scala.Function1[Double, Double]): java.util.function.DoubleUnaryOperator = - new AsJavaDoubleUnaryOperator(sf) - - @inline def asScalaFromFunction[T, R](jf: java.util.function.Function[T, R]): scala.Function1[T, R] = - new FromJavaFunction[T, R](jf) - - @inline def asJavaFunction[T, R](sf: scala.Function1[T, R]): java.util.function.Function[T, R] = - new AsJavaFunction[T, R](sf) - - @inline def asScalaFromIntBinaryOperator(jf: java.util.function.IntBinaryOperator): scala.Function2[Int, Int, Int] = - new FromJavaIntBinaryOperator(jf) - - @inline def asJavaIntBinaryOperator(sf: scala.Function2[Int, Int, Int]): java.util.function.IntBinaryOperator = - new AsJavaIntBinaryOperator(sf) - - @inline def asScalaFromIntConsumer(jf: java.util.function.IntConsumer): scala.Function1[Int, Unit] = - new FromJavaIntConsumer(jf) - - @inline def asJavaIntConsumer(sf: scala.Function1[Int, Unit]): java.util.function.IntConsumer = - new AsJavaIntConsumer(sf) - - @inline def asScalaFromIntFunction[R](jf: java.util.function.IntFunction[R]): scala.Function1[Int, R] = - new FromJavaIntFunction[R](jf) - - @inline def asJavaIntFunction[R](sf: scala.Function1[Int, R]): java.util.function.IntFunction[R] = - new AsJavaIntFunction[R](sf) - - @inline def asScalaFromIntPredicate(jf: java.util.function.IntPredicate): scala.Function1[Int, Boolean] = - new FromJavaIntPredicate(jf) - - @inline def asJavaIntPredicate(sf: scala.Function1[Int, Boolean]): java.util.function.IntPredicate = - new AsJavaIntPredicate(sf) - - @inline def asScalaFromIntSupplier(jf: java.util.function.IntSupplier): scala.Function0[Int] = - new FromJavaIntSupplier(jf) - - @inline def asJavaIntSupplier(sf: scala.Function0[Int]): java.util.function.IntSupplier = new AsJavaIntSupplier(sf) - - @inline def asScalaFromIntToDoubleFunction(jf: java.util.function.IntToDoubleFunction): scala.Function1[Int, Double] = - new FromJavaIntToDoubleFunction(jf) - - @inline def asJavaIntToDoubleFunction(sf: scala.Function1[Int, Double]): java.util.function.IntToDoubleFunction = - new AsJavaIntToDoubleFunction(sf) - - @inline def asScalaFromIntToLongFunction(jf: java.util.function.IntToLongFunction): scala.Function1[Int, Long] = - new FromJavaIntToLongFunction(jf) - - @inline def asJavaIntToLongFunction(sf: scala.Function1[Int, Long]): java.util.function.IntToLongFunction = - new AsJavaIntToLongFunction(sf) - - @inline def asScalaFromIntUnaryOperator(jf: java.util.function.IntUnaryOperator): scala.Function1[Int, Int] = - new FromJavaIntUnaryOperator(jf) - - @inline def asJavaIntUnaryOperator(sf: scala.Function1[Int, Int]): java.util.function.IntUnaryOperator = - new AsJavaIntUnaryOperator(sf) - - @inline def asScalaFromLongBinaryOperator( - jf: java.util.function.LongBinaryOperator): scala.Function2[Long, Long, Long] = new FromJavaLongBinaryOperator(jf) - - @inline def asJavaLongBinaryOperator(sf: scala.Function2[Long, Long, Long]): java.util.function.LongBinaryOperator = - new AsJavaLongBinaryOperator(sf) - - @inline def asScalaFromLongConsumer(jf: java.util.function.LongConsumer): scala.Function1[Long, Unit] = - new FromJavaLongConsumer(jf) - - @inline def asJavaLongConsumer(sf: scala.Function1[Long, Unit]): java.util.function.LongConsumer = - new AsJavaLongConsumer(sf) - - @inline def asScalaFromLongFunction[R](jf: java.util.function.LongFunction[R]): scala.Function1[Long, R] = - new FromJavaLongFunction[R](jf) - - @inline def asJavaLongFunction[R](sf: scala.Function1[Long, R]): java.util.function.LongFunction[R] = - new AsJavaLongFunction[R](sf) - - @inline def asScalaFromLongPredicate(jf: java.util.function.LongPredicate): scala.Function1[Long, Boolean] = - new FromJavaLongPredicate(jf) - - @inline def asJavaLongPredicate(sf: scala.Function1[Long, Boolean]): java.util.function.LongPredicate = - new AsJavaLongPredicate(sf) - - @inline def asScalaFromLongSupplier(jf: java.util.function.LongSupplier): scala.Function0[Long] = - new FromJavaLongSupplier(jf) - - @inline def asJavaLongSupplier(sf: scala.Function0[Long]): java.util.function.LongSupplier = - new AsJavaLongSupplier(sf) - - @inline def asScalaFromLongToDoubleFunction( - jf: java.util.function.LongToDoubleFunction): scala.Function1[Long, Double] = new FromJavaLongToDoubleFunction(jf) - - @inline def asJavaLongToDoubleFunction(sf: scala.Function1[Long, Double]): java.util.function.LongToDoubleFunction = - new AsJavaLongToDoubleFunction(sf) - - @inline def asScalaFromLongToIntFunction(jf: java.util.function.LongToIntFunction): scala.Function1[Long, Int] = - new FromJavaLongToIntFunction(jf) - - @inline def asJavaLongToIntFunction(sf: scala.Function1[Long, Int]): java.util.function.LongToIntFunction = - new AsJavaLongToIntFunction(sf) - - @inline def asScalaFromLongUnaryOperator(jf: java.util.function.LongUnaryOperator): scala.Function1[Long, Long] = - new FromJavaLongUnaryOperator(jf) - - @inline def asJavaLongUnaryOperator(sf: scala.Function1[Long, Long]): java.util.function.LongUnaryOperator = - new AsJavaLongUnaryOperator(sf) - - @inline def asScalaFromObjDoubleConsumer[T]( - jf: java.util.function.ObjDoubleConsumer[T]): scala.Function2[T, Double, Unit] = - new FromJavaObjDoubleConsumer[T](jf) - - @inline def asJavaObjDoubleConsumer[T]( - sf: scala.Function2[T, Double, Unit]): java.util.function.ObjDoubleConsumer[T] = - new AsJavaObjDoubleConsumer[T](sf) - - @inline def asScalaFromObjIntConsumer[T](jf: java.util.function.ObjIntConsumer[T]): scala.Function2[T, Int, Unit] = - new FromJavaObjIntConsumer[T](jf) - - @inline def asJavaObjIntConsumer[T](sf: scala.Function2[T, Int, Unit]): java.util.function.ObjIntConsumer[T] = - new AsJavaObjIntConsumer[T](sf) - - @inline def asScalaFromObjLongConsumer[T](jf: java.util.function.ObjLongConsumer[T]): scala.Function2[T, Long, Unit] = - new FromJavaObjLongConsumer[T](jf) - - @inline def asJavaObjLongConsumer[T](sf: scala.Function2[T, Long, Unit]): java.util.function.ObjLongConsumer[T] = - new AsJavaObjLongConsumer[T](sf) - - @inline def asScalaFromPredicate[T](jf: java.util.function.Predicate[T]): scala.Function1[T, Boolean] = - new FromJavaPredicate[T](jf) - - @inline def asJavaPredicate[T](sf: scala.Function1[T, Boolean]): java.util.function.Predicate[T] = - new AsJavaPredicate[T](sf) - - @inline def asScalaFromSupplier[T](jf: java.util.function.Supplier[T]): scala.Function0[T] = - new FromJavaSupplier[T](jf) - - @inline def asJavaSupplier[T](sf: scala.Function0[T]): java.util.function.Supplier[T] = new AsJavaSupplier[T](sf) - - @inline def asScalaFromToDoubleBiFunction[T, U]( - jf: java.util.function.ToDoubleBiFunction[T, U]): scala.Function2[T, U, Double] = - new FromJavaToDoubleBiFunction[T, U](jf) - - @inline def asJavaToDoubleBiFunction[T, U]( - sf: scala.Function2[T, U, Double]): java.util.function.ToDoubleBiFunction[T, U] = - new AsJavaToDoubleBiFunction[T, U](sf) - - @inline def asScalaFromToDoubleFunction[T](jf: java.util.function.ToDoubleFunction[T]): scala.Function1[T, Double] = - new FromJavaToDoubleFunction[T](jf) - - @inline def asJavaToDoubleFunction[T](sf: scala.Function1[T, Double]): java.util.function.ToDoubleFunction[T] = - new AsJavaToDoubleFunction[T](sf) - - @inline def asScalaFromToIntBiFunction[T, U]( - jf: java.util.function.ToIntBiFunction[T, U]): scala.Function2[T, U, Int] = new FromJavaToIntBiFunction[T, U](jf) - - @inline def asJavaToIntBiFunction[T, U](sf: scala.Function2[T, U, Int]): java.util.function.ToIntBiFunction[T, U] = - new AsJavaToIntBiFunction[T, U](sf) - - @inline def asScalaFromToIntFunction[T](jf: java.util.function.ToIntFunction[T]): scala.Function1[T, Int] = - new FromJavaToIntFunction[T](jf) - - @inline def asJavaToIntFunction[T](sf: scala.Function1[T, Int]): java.util.function.ToIntFunction[T] = - new AsJavaToIntFunction[T](sf) - - @inline def asScalaFromToLongBiFunction[T, U]( - jf: java.util.function.ToLongBiFunction[T, U]): scala.Function2[T, U, Long] = - new FromJavaToLongBiFunction[T, U](jf) - - @inline def asJavaToLongBiFunction[T, U](sf: scala.Function2[T, U, Long]): java.util.function.ToLongBiFunction[T, U] = - new AsJavaToLongBiFunction[T, U](sf) - - @inline def asScalaFromToLongFunction[T](jf: java.util.function.ToLongFunction[T]): scala.Function1[T, Long] = - new FromJavaToLongFunction[T](jf) - - @inline def asJavaToLongFunction[T](sf: scala.Function1[T, Long]): java.util.function.ToLongFunction[T] = - new AsJavaToLongFunction[T](sf) - - @inline def asScalaFromUnaryOperator[T](jf: java.util.function.UnaryOperator[T]): scala.Function1[T, T] = - new FromJavaUnaryOperator[T](jf) - - @inline def asJavaUnaryOperator[T](sf: scala.Function1[T, T]): java.util.function.UnaryOperator[T] = - new AsJavaUnaryOperator[T](sf) - - @inline implicit def enrichAsJavaBooleanSupplier(sf: scala.Function0[Boolean]): RichFunction0AsBooleanSupplier = - new RichFunction0AsBooleanSupplier(sf) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoubleBinaryOperator[A0, A1](sf: scala.Function2[A0, A1, Double])( - implicit evA0: =:=[A0, Double], evA1: =:=[A1, Double]): RichFunction2AsDoubleBinaryOperator = - new RichFunction2AsDoubleBinaryOperator(sf.asInstanceOf[scala.Function2[Double, Double, Double]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoubleConsumer[A0](sf: scala.Function1[A0, Unit])( - implicit evA0: =:=[A0, Double]): RichFunction1AsDoubleConsumer = - new RichFunction1AsDoubleConsumer(sf.asInstanceOf[scala.Function1[Double, Unit]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoublePredicate[A0](sf: scala.Function1[A0, Boolean])( - implicit evA0: =:=[A0, Double]): RichFunction1AsDoublePredicate = - new RichFunction1AsDoublePredicate(sf.asInstanceOf[scala.Function1[Double, Boolean]]) - - @inline implicit def enrichAsJavaDoubleSupplier(sf: scala.Function0[Double]): RichFunction0AsDoubleSupplier = - new RichFunction0AsDoubleSupplier(sf) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoubleToIntFunction[A0](sf: scala.Function1[A0, Int])( - implicit evA0: =:=[A0, Double]): RichFunction1AsDoubleToIntFunction = - new RichFunction1AsDoubleToIntFunction(sf.asInstanceOf[scala.Function1[Double, Int]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoubleToLongFunction[A0](sf: scala.Function1[A0, Long])( - implicit evA0: =:=[A0, Double]): RichFunction1AsDoubleToLongFunction = - new RichFunction1AsDoubleToLongFunction(sf.asInstanceOf[scala.Function1[Double, Long]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaDoubleUnaryOperator[A0](sf: scala.Function1[A0, Double])( - implicit evA0: =:=[A0, Double]): RichFunction1AsDoubleUnaryOperator = - new RichFunction1AsDoubleUnaryOperator(sf.asInstanceOf[scala.Function1[Double, Double]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntBinaryOperator[A0, A1](sf: scala.Function2[A0, A1, Int])( - implicit evA0: =:=[A0, Int], evA1: =:=[A1, Int]): RichFunction2AsIntBinaryOperator = - new RichFunction2AsIntBinaryOperator(sf.asInstanceOf[scala.Function2[Int, Int, Int]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntConsumer[A0](sf: scala.Function1[A0, Unit])( - implicit evA0: =:=[A0, Int]): RichFunction1AsIntConsumer = - new RichFunction1AsIntConsumer(sf.asInstanceOf[scala.Function1[Int, Unit]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntPredicate[A0](sf: scala.Function1[A0, Boolean])( - implicit evA0: =:=[A0, Int]): RichFunction1AsIntPredicate = - new RichFunction1AsIntPredicate(sf.asInstanceOf[scala.Function1[Int, Boolean]]) - - @inline implicit def enrichAsJavaIntSupplier(sf: scala.Function0[Int]): RichFunction0AsIntSupplier = - new RichFunction0AsIntSupplier(sf) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntToDoubleFunction[A0](sf: scala.Function1[A0, Double])( - implicit evA0: =:=[A0, Int]): RichFunction1AsIntToDoubleFunction = - new RichFunction1AsIntToDoubleFunction(sf.asInstanceOf[scala.Function1[Int, Double]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntToLongFunction[A0](sf: scala.Function1[A0, Long])( - implicit evA0: =:=[A0, Int]): RichFunction1AsIntToLongFunction = - new RichFunction1AsIntToLongFunction(sf.asInstanceOf[scala.Function1[Int, Long]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaIntUnaryOperator[A0](sf: scala.Function1[A0, Int])( - implicit evA0: =:=[A0, Int]): RichFunction1AsIntUnaryOperator = - new RichFunction1AsIntUnaryOperator(sf.asInstanceOf[scala.Function1[Int, Int]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongBinaryOperator[A0, A1](sf: scala.Function2[A0, A1, Long])( - implicit evA0: =:=[A0, Long], evA1: =:=[A1, Long]): RichFunction2AsLongBinaryOperator = - new RichFunction2AsLongBinaryOperator(sf.asInstanceOf[scala.Function2[Long, Long, Long]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongConsumer[A0](sf: scala.Function1[A0, Unit])( - implicit evA0: =:=[A0, Long]): RichFunction1AsLongConsumer = - new RichFunction1AsLongConsumer(sf.asInstanceOf[scala.Function1[Long, Unit]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongPredicate[A0](sf: scala.Function1[A0, Boolean])( - implicit evA0: =:=[A0, Long]): RichFunction1AsLongPredicate = - new RichFunction1AsLongPredicate(sf.asInstanceOf[scala.Function1[Long, Boolean]]) - - @inline implicit def enrichAsJavaLongSupplier(sf: scala.Function0[Long]): RichFunction0AsLongSupplier = - new RichFunction0AsLongSupplier(sf) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongToDoubleFunction[A0](sf: scala.Function1[A0, Double])( - implicit evA0: =:=[A0, Long]): RichFunction1AsLongToDoubleFunction = - new RichFunction1AsLongToDoubleFunction(sf.asInstanceOf[scala.Function1[Long, Double]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongToIntFunction[A0](sf: scala.Function1[A0, Int])( - implicit evA0: =:=[A0, Long]): RichFunction1AsLongToIntFunction = - new RichFunction1AsLongToIntFunction(sf.asInstanceOf[scala.Function1[Long, Int]]) - - @nowarn("msg=never used") - @inline implicit def enrichAsJavaLongUnaryOperator[A0](sf: scala.Function1[A0, Long])( - implicit evA0: =:=[A0, Long]): RichFunction1AsLongUnaryOperator = - new RichFunction1AsLongUnaryOperator(sf.asInstanceOf[scala.Function1[Long, Long]]) - - @inline implicit def enrichAsScalaFromBiConsumer[T, U]( - jf: java.util.function.BiConsumer[T, U]): RichBiConsumerAsFunction2[T, U] = - new RichBiConsumerAsFunction2[T, U](jf) - - @inline implicit def enrichAsScalaFromBiFunction[T, U, R]( - jf: java.util.function.BiFunction[T, U, R]): RichBiFunctionAsFunction2[T, U, R] = - new RichBiFunctionAsFunction2[T, U, R](jf) - - @inline implicit def enrichAsScalaFromBiPredicate[T, U]( - jf: java.util.function.BiPredicate[T, U]): RichBiPredicateAsFunction2[T, U] = - new RichBiPredicateAsFunction2[T, U](jf) - - @inline implicit def enrichAsScalaFromBinaryOperator[T]( - jf: java.util.function.BinaryOperator[T]): RichBinaryOperatorAsFunction2[T] = - new RichBinaryOperatorAsFunction2[T](jf) - - @inline implicit def enrichAsScalaFromBooleanSupplier( - jf: java.util.function.BooleanSupplier): RichBooleanSupplierAsFunction0 = new RichBooleanSupplierAsFunction0(jf) - - @inline implicit def enrichAsScalaFromConsumer[T](jf: java.util.function.Consumer[T]): RichConsumerAsFunction1[T] = - new RichConsumerAsFunction1[T](jf) - - @inline implicit def enrichAsScalaFromDoubleBinaryOperator( - jf: java.util.function.DoubleBinaryOperator): RichDoubleBinaryOperatorAsFunction2 = - new RichDoubleBinaryOperatorAsFunction2(jf) - - @inline implicit def enrichAsScalaFromDoubleConsumer( - jf: java.util.function.DoubleConsumer): RichDoubleConsumerAsFunction1 = new RichDoubleConsumerAsFunction1(jf) - - @inline implicit def enrichAsScalaFromDoubleFunction[R]( - jf: java.util.function.DoubleFunction[R]): RichDoubleFunctionAsFunction1[R] = - new RichDoubleFunctionAsFunction1[R](jf) - - @inline implicit def enrichAsScalaFromDoublePredicate( - jf: java.util.function.DoublePredicate): RichDoublePredicateAsFunction1 = new RichDoublePredicateAsFunction1(jf) - - @inline implicit def enrichAsScalaFromDoubleSupplier( - jf: java.util.function.DoubleSupplier): RichDoubleSupplierAsFunction0 = new RichDoubleSupplierAsFunction0(jf) - - @inline implicit def enrichAsScalaFromDoubleToIntFunction( - jf: java.util.function.DoubleToIntFunction): RichDoubleToIntFunctionAsFunction1 = - new RichDoubleToIntFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromDoubleToLongFunction( - jf: java.util.function.DoubleToLongFunction): RichDoubleToLongFunctionAsFunction1 = - new RichDoubleToLongFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromDoubleUnaryOperator( - jf: java.util.function.DoubleUnaryOperator): RichDoubleUnaryOperatorAsFunction1 = - new RichDoubleUnaryOperatorAsFunction1(jf) - - @inline implicit def enrichAsScalaFromFunction[T, R]( - jf: java.util.function.Function[T, R]): RichFunctionAsFunction1[T, R] = new RichFunctionAsFunction1[T, R](jf) - - @inline implicit def enrichAsScalaFromIntBinaryOperator( - jf: java.util.function.IntBinaryOperator): RichIntBinaryOperatorAsFunction2 = - new RichIntBinaryOperatorAsFunction2(jf) - - @inline implicit def enrichAsScalaFromIntConsumer(jf: java.util.function.IntConsumer): RichIntConsumerAsFunction1 = - new RichIntConsumerAsFunction1(jf) - - @inline implicit def enrichAsScalaFromIntFunction[R]( - jf: java.util.function.IntFunction[R]): RichIntFunctionAsFunction1[R] = new RichIntFunctionAsFunction1[R](jf) - - @inline implicit def enrichAsScalaFromIntPredicate(jf: java.util.function.IntPredicate): RichIntPredicateAsFunction1 = - new RichIntPredicateAsFunction1(jf) - - @inline implicit def enrichAsScalaFromIntSupplier(jf: java.util.function.IntSupplier): RichIntSupplierAsFunction0 = - new RichIntSupplierAsFunction0(jf) - - @inline implicit def enrichAsScalaFromIntToDoubleFunction( - jf: java.util.function.IntToDoubleFunction): RichIntToDoubleFunctionAsFunction1 = - new RichIntToDoubleFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromIntToLongFunction( - jf: java.util.function.IntToLongFunction): RichIntToLongFunctionAsFunction1 = - new RichIntToLongFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromIntUnaryOperator( - jf: java.util.function.IntUnaryOperator): RichIntUnaryOperatorAsFunction1 = - new RichIntUnaryOperatorAsFunction1(jf) - - @inline implicit def enrichAsScalaFromLongBinaryOperator( - jf: java.util.function.LongBinaryOperator): RichLongBinaryOperatorAsFunction2 = - new RichLongBinaryOperatorAsFunction2(jf) - - @inline implicit def enrichAsScalaFromLongConsumer(jf: java.util.function.LongConsumer): RichLongConsumerAsFunction1 = - new RichLongConsumerAsFunction1(jf) - - @inline implicit def enrichAsScalaFromLongFunction[R]( - jf: java.util.function.LongFunction[R]): RichLongFunctionAsFunction1[R] = new RichLongFunctionAsFunction1[R](jf) - - @inline implicit def enrichAsScalaFromLongPredicate( - jf: java.util.function.LongPredicate): RichLongPredicateAsFunction1 = new RichLongPredicateAsFunction1(jf) - - @inline implicit def enrichAsScalaFromLongSupplier(jf: java.util.function.LongSupplier): RichLongSupplierAsFunction0 = - new RichLongSupplierAsFunction0(jf) - - @inline implicit def enrichAsScalaFromLongToDoubleFunction( - jf: java.util.function.LongToDoubleFunction): RichLongToDoubleFunctionAsFunction1 = - new RichLongToDoubleFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromLongToIntFunction( - jf: java.util.function.LongToIntFunction): RichLongToIntFunctionAsFunction1 = - new RichLongToIntFunctionAsFunction1(jf) - - @inline implicit def enrichAsScalaFromLongUnaryOperator( - jf: java.util.function.LongUnaryOperator): RichLongUnaryOperatorAsFunction1 = - new RichLongUnaryOperatorAsFunction1(jf) - - @inline implicit def enrichAsScalaFromObjDoubleConsumer[T]( - jf: java.util.function.ObjDoubleConsumer[T]): RichObjDoubleConsumerAsFunction2[T] = - new RichObjDoubleConsumerAsFunction2[T](jf) - - @inline implicit def enrichAsScalaFromObjIntConsumer[T]( - jf: java.util.function.ObjIntConsumer[T]): RichObjIntConsumerAsFunction2[T] = - new RichObjIntConsumerAsFunction2[T](jf) - - @inline implicit def enrichAsScalaFromObjLongConsumer[T]( - jf: java.util.function.ObjLongConsumer[T]): RichObjLongConsumerAsFunction2[T] = - new RichObjLongConsumerAsFunction2[T](jf) - - @inline implicit def enrichAsScalaFromPredicate[T](jf: java.util.function.Predicate[T]): RichPredicateAsFunction1[T] = - new RichPredicateAsFunction1[T](jf) - - @inline implicit def enrichAsScalaFromSupplier[T](jf: java.util.function.Supplier[T]): RichSupplierAsFunction0[T] = - new RichSupplierAsFunction0[T](jf) - - @inline implicit def enrichAsScalaFromToDoubleBiFunction[T, U]( - jf: java.util.function.ToDoubleBiFunction[T, U]): RichToDoubleBiFunctionAsFunction2[T, U] = - new RichToDoubleBiFunctionAsFunction2[T, U](jf) - - @inline implicit def enrichAsScalaFromToDoubleFunction[T]( - jf: java.util.function.ToDoubleFunction[T]): RichToDoubleFunctionAsFunction1[T] = - new RichToDoubleFunctionAsFunction1[T](jf) - - @inline implicit def enrichAsScalaFromToIntBiFunction[T, U]( - jf: java.util.function.ToIntBiFunction[T, U]): RichToIntBiFunctionAsFunction2[T, U] = - new RichToIntBiFunctionAsFunction2[T, U](jf) - - @inline implicit def enrichAsScalaFromToIntFunction[T]( - jf: java.util.function.ToIntFunction[T]): RichToIntFunctionAsFunction1[T] = - new RichToIntFunctionAsFunction1[T](jf) - - @inline implicit def enrichAsScalaFromToLongBiFunction[T, U]( - jf: java.util.function.ToLongBiFunction[T, U]): RichToLongBiFunctionAsFunction2[T, U] = - new RichToLongBiFunctionAsFunction2[T, U](jf) - - @inline implicit def enrichAsScalaFromToLongFunction[T]( - jf: java.util.function.ToLongFunction[T]): RichToLongFunctionAsFunction1[T] = - new RichToLongFunctionAsFunction1[T](jf) - - @inline implicit def enrichAsScalaFromUnaryOperator[T]( - jf: java.util.function.UnaryOperator[T]): RichUnaryOperatorAsFunction1[T] = - new RichUnaryOperatorAsFunction1[T](jf) - -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/FutureConverters.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/FutureConverters.scala deleted file mode 100644 index 0f00f47874..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/FutureConverters.scala +++ /dev/null @@ -1,43 +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.concurrent.CompletionStage -import scala.concurrent.Future - -/** - * INTERNAL API - * - * Remove this once Scala 2.12 support is dropped since all methods are in Scala 2.13+ stdlib - */ -@InternalStableApi -private[pekko] object FutureConverters { - @inline final def asJava[T](f: Future[T]): CompletionStage[T] = scala.compat.java8.FutureConverters.toJava(f) - - implicit final class FutureOps[T](private val f: Future[T]) extends AnyVal { - @inline def asJava: CompletionStage[T] = FutureConverters.asJava(f) - } - - @inline final def asScala[T](cs: CompletionStage[T]): Future[T] = scala.compat.java8.FutureConverters.toScala(cs) - - implicit final class CompletionStageOps[T](private val cs: CompletionStage[T]) extends AnyVal { - @inline def asScala: Future[T] = FutureConverters.asScala(cs) - } -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/OptionConverters.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/OptionConverters.scala deleted file mode 100644 index 85a058c839..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/OptionConverters.scala +++ /dev/null @@ -1,91 +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._ - -/** - * 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 { - import scala.compat.java8.OptionConverters.SpecializerOfOptions - import scala.compat.java8.OptionConverters._ - - @inline final def toScala[A](o: Optional[A]): Option[A] = scala.compat.java8.OptionConverters.toScala(o) - - // The rest of the .toScala methods that work with OptionalDouble/OptionalInt/OptionalLong have to be manually - // redefined because the scala.compat.java8.OptionConverters.toScala variants work with scala.lang primitive types - // where as scala.jdk.javaapi.OptionConverters.toScala works with java.lang primitive types. Since the primary - // usecase of these functions is for calling within Java code its preferrable to return Java primitives, see - // https://github.com/scala/bug/issues/4214 - def toScala(o: OptionalDouble): Option[java.lang.Double] = if (o.isPresent) Some(o.getAsDouble) else None - - def toScala(o: OptionalInt): Option[java.lang.Integer] = if (o.isPresent) Some(o.getAsInt) else None - - def toScala(o: OptionalLong): Option[java.lang.Long] = if (o.isPresent) Some(o.getAsLong) else None - - @inline final def toJava[A](o: Option[A]): Optional[A] = scala.compat.java8.OptionConverters.toJava(o) - - implicit final class RichOptional[A](private val o: java.util.Optional[A]) extends AnyVal { - @inline def toScala: Option[A] = scala.compat.java8.OptionConverters.RichOptionalGeneric(o).asScala - - @inline def toJavaPrimitive[O](implicit specOp: SpecializerOfOptions[A, O]): O = - scala.compat.java8.OptionConverters.RichOptionalGeneric(o).asPrimitive - } - - implicit final class RichOption[A](private val o: Option[A]) extends AnyVal { - @inline def toJava: Optional[A] = scala.compat.java8.OptionConverters.RichOptionForJava8(o).asJava - - @inline def toJavaPrimitive[O](implicit specOp: SpecializerOfOptions[A, O]): O = - scala.compat.java8.OptionConverters.RichOptionForJava8(o).asPrimitive - } - - implicit class RichOptionalDouble(private val o: OptionalDouble) extends AnyVal { - - /** Convert a Java `OptionalDouble` to a Scala `Option` */ - @inline def toScala: Option[Double] = scala.compat.java8.OptionConverters.RichOptionalDouble(o).asScala - - /** Convert a Java `OptionalDouble` to a generic Java `Optional` */ - @inline def toJavaGeneric: Optional[Double] = scala.compat.java8.OptionConverters.RichOptionalDouble(o).asGeneric - } - - /** 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.compat.java8.OptionConverters.RichOptionalInt(o).asScala - - /** Convert a Java `OptionalInt` to a generic Java `Optional` */ - @inline def toJavaGeneric: Optional[Int] = scala.compat.java8.OptionConverters.RichOptionalInt(o).asGeneric - } - - /** 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.compat.java8.OptionConverters.RichOptionalLong(o).asScala - - /** Convert a Java `OptionalLong` to a generic Java `Optional` */ - @inline def toJavaGeneric: Optional[Long] = scala.compat.java8.OptionConverters.RichOptionalLong(o).asGeneric - } -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/CompatImpl.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/CompatImpl.scala deleted file mode 100644 index db151200c3..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/CompatImpl.scala +++ /dev/null @@ -1,31 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2018-2022 Lightbend Inc. - */ - -package org.apache.pekko.util.ccompat - -import scala.collection.generic.CanBuildFrom -import scala.collection.mutable.Builder - -/** - * INTERNAL API - * - * Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.11_2.12/scala/collection/compat/CompatImpl.scala - * but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is - * not promising binary compatibility yet at the time of writing. - */ -private[ccompat] object CompatImpl { - def simpleCBF[A, C](f: => Builder[A, C]): CanBuildFrom[Any, A, C] = new CanBuildFrom[Any, A, C] { - def apply(from: Any): Builder[A, C] = apply() - def apply(): Builder[A, C] = f - } -} diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala deleted file mode 100644 index 1eeaec25b6..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala +++ /dev/null @@ -1,27 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2019-2022 Lightbend Inc. - */ - -package org.apache.pekko.util.ccompat - -import scala.annotation.Annotation - -import org.apache.pekko.annotation.InternalApi - -/** - * INTERNAL API - * - * Annotation to mark files that need ccompat to be imported for Scala 2.11 and/or 2.12, - * but not 2.13. Gets rid of the 'unused import' warning on 2.13. - */ -@InternalApi -private[pekko] class ccompatUsedUntil213 extends Annotation diff --git a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala b/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala deleted file mode 100644 index 23135d3ffc..0000000000 --- a/actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala +++ /dev/null @@ -1,154 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2018-2022 Lightbend Inc. - */ - -/* - * Scala (https://www.scala-lang.org) - * - * Copyright EPFL and Lightbend, Inc. - * - * Licensed under Apache License 2.0 - * (http://www.apache.org/licenses/LICENSE-2.0). - * - * See the NOTICE file distributed with this work for - * additional information regarding copyright ownership. - */ - -package org.apache.pekko.util - -import scala.{ collection => c } -import scala.collection.{ immutable => i, mutable => m, GenTraversable, IterableView } -import scala.collection.generic.{ CanBuildFrom, GenericCompanion, Sorted, SortedSetFactory } -import scala.language.higherKinds -import scala.language.implicitConversions - -/** - * INTERNAL API - * - * Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.11_2.12/scala/collection/compat/PackageShared.scala - * but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is - * not promising binary compatibility yet at the time of writing. - */ -package object ccompat { - import CompatImpl._ - - /** - * A factory that builds a collection of type `C` with elements of type `A`. - * - * @tparam A Type of elements (e.g. `Int`, `Boolean`, etc.) - * @tparam C Type of collection (e.g. `List[Int]`, `TreeMap[Int, String]`, etc.) - */ - private[pekko] type Factory[-A, +C] = CanBuildFrom[Nothing, A, C] - - private[pekko] implicit final class FactoryOps[-A, +C](private val factory: Factory[A, C]) { - - /** - * @return A collection of type `C` containing the same elements - * as the source collection `it`. - * @param it Source collection - */ - def fromSpecific(it: TraversableOnce[A]): C = (factory() ++= it).result() - - /** - * Get a Builder for the collection. For non-strict collection types this will use an intermediate buffer. - * Building collections with `fromSpecific` is preferred because it can be lazy for lazy collections. - */ - def newBuilder: m.Builder[A, C] = factory() - } - - private[pekko] implicit def genericCompanionToCBF[A, CC[X] <: GenTraversable[X]]( - fact: GenericCompanion[CC]): CanBuildFrom[Any, A, CC[A]] = - simpleCBF(fact.newBuilder[A]) - - private[pekko] implicit def sortedSetCompanionToCBF[ - A: Ordering, CC[X] <: c.SortedSet[X] with c.SortedSetLike[X, CC[X]]]( - fact: SortedSetFactory[CC]): CanBuildFrom[Any, A, CC[A]] = - simpleCBF(fact.newBuilder[A]) - - private[ccompat] def build[T, CC](builder: m.Builder[T, CC], source: TraversableOnce[T]): CC = { - builder ++= source - builder.result() - } - - private[pekko] implicit final class ImmutableSortedMapExtensions(private val fact: i.SortedMap.type) extends AnyVal { - def from[K: Ordering, V](source: TraversableOnce[(K, V)]): i.SortedMap[K, V] = - build(i.SortedMap.newBuilder[K, V], source) - } - - private[pekko] implicit final class ImmutableTreeMapExtensions(private val fact: i.TreeMap.type) extends AnyVal { - def from[K: Ordering, V](source: TraversableOnce[(K, V)]): i.TreeMap[K, V] = - build(i.TreeMap.newBuilder[K, V], source) - } - - private[pekko] implicit final class IterableExtensions(private val fact: Iterable.type) extends AnyVal { - // derived from https://github.com/scala/scala/blob/0842f23f6017f93160b115b8bf29ec5347cdbe94/src/library/scala/Predef.scala#L356-L361 - // Apache License 2.0 (see Scala license header at top of this file) - def single[A](a: A): Iterable[A] = new Iterable[A] { - override def iterator = Iterator.single(a) - override def sizeHintIfCheap: Int = 1 - override def hasDefiniteSize: Boolean = true - override def head = a - override def headOption = Some(a) - override def last = a - override def lastOption = Some(a) - override def view = new IterableView[A, Iterable[A]] { - override def iterator: Iterator[A] = Iterator.single(a) - override def sizeHintIfCheap: Int = 1 - override def hasDefiniteSize: Boolean = true - override protected def underlying: Iterable[A] = this - } - override def take(n: Int) = if (n > 0) this else Iterable.empty - override def takeRight(n: Int) = if (n > 0) this else Iterable.empty - override def drop(n: Int) = if (n > 0) Iterable.empty else this - override def dropRight(n: Int) = if (n > 0) Iterable.empty else this - override def tail = Iterable.empty - override def init = Iterable.empty - } - } - - private[pekko] implicit final class SortedExtensionMethods[K, T <: Sorted[K, T]](private val fact: Sorted[K, T]) { - def rangeFrom(from: K): T = fact.from(from) - def rangeTo(to: K): T = fact.to(to) - def rangeUntil(until: K): T = fact.until(until) - } - - // This really belongs into scala.collection but there's already a package object - // in scala-library so we can't add to it - type IterableOnce[+X] = c.TraversableOnce[X] - val IterableOnce = c.TraversableOnce - - implicit def toMapViewExtensionMethods[K, V, C <: scala.collection.Map[K, V]]( - self: IterableView[(K, V), C]): MapViewExtensionMethods[K, V, C] = - new MapViewExtensionMethods[K, V, C](self) - - implicit final class ImmutableSortedSetOps[A](val real: i.SortedSet[A]) extends AnyVal { - def unsorted: i.Set[A] = real - } - - object JavaConverters extends scala.collection.convert.DecorateAsJava with scala.collection.convert.DecorateAsScala - - implicit def toTraversableOnceExtensionMethods[A](self: TraversableOnce[A]): TraversableOnceExtensionMethods[A] = - new TraversableOnceExtensionMethods[A](self) -} - -final class TraversableOnceExtensionMethods[A](private val self: c.TraversableOnce[A]) extends AnyVal { - def iterator: Iterator[A] = self.toIterator -} - -final class MapViewExtensionMethods[K, V, C <: scala.collection.Map[K, V]]( - private val self: IterableView[(K, V), C]) extends AnyVal { - def mapValues[W, That](f: V => W)(implicit bf: CanBuildFrom[IterableView[(K, V), C], (K, W), That]): That = - self.map[(K, W), That] { case (k, v) => (k, f(v)) } - - def filterKeys(p: K => Boolean): IterableView[(K, V), C] = - self.filter { case (k, _) => p(k) } -} diff --git a/actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala b/actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala deleted file mode 100644 index 1eeaec25b6..0000000000 --- a/actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/ccompatUsedUntil213.scala +++ /dev/null @@ -1,27 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2019-2022 Lightbend Inc. - */ - -package org.apache.pekko.util.ccompat - -import scala.annotation.Annotation - -import org.apache.pekko.annotation.InternalApi - -/** - * INTERNAL API - * - * Annotation to mark files that need ccompat to be imported for Scala 2.11 and/or 2.12, - * but not 2.13. Gets rid of the 'unused import' warning on 2.13. - */ -@InternalApi -private[pekko] class ccompatUsedUntil213 extends Annotation diff --git a/actor/src/main/scala-2.13/org/apache/pekko/compat/PartialFunction.scala b/actor/src/main/scala-2.13/org/apache/pekko/compat/PartialFunction.scala deleted file mode 100644 index f76d4d32f0..0000000000 --- a/actor/src/main/scala-2.13/org/apache/pekko/compat/PartialFunction.scala +++ /dev/null @@ -1,31 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.compat - -import org.apache.pekko.annotation.InternalApi - -/** - * INTERNAL API - * - * Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code - * against Scala 2.12, 2.13, 3.0 - * - * Remove these classes as soon as support for Scala 2.12 is dropped! - */ -@InternalApi private[pekko] object PartialFunction { - - @inline def fromFunction[A, B](f: A => B): scala.PartialFunction[A, B] = - scala.PartialFunction.fromFunction(f) - -} diff --git a/actor/src/main/scala-2/org/apache/pekko/util/JavaDurationConverters.scala b/actor/src/main/scala-2.13/org/apache/pekko/util/JavaDurationConverters.scala similarity index 100% rename from actor/src/main/scala-2/org/apache/pekko/util/JavaDurationConverters.scala rename to actor/src/main/scala-2.13/org/apache/pekko/util/JavaDurationConverters.scala diff --git a/actor/src/main/scala-3/org/apache/pekko/compat/PartialFunction.scala b/actor/src/main/scala-3/org/apache/pekko/compat/PartialFunction.scala deleted file mode 100644 index a05afc17bd..0000000000 --- a/actor/src/main/scala-3/org/apache/pekko/compat/PartialFunction.scala +++ /dev/null @@ -1,31 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2009-2022 Lightbend Inc. - */ - -package org.apache.pekko.compat - -import org.apache.pekko.annotation.InternalApi - -/** - * INTERNAL API - * - * Compatibility wrapper for `scala.PartialFunction` to be able to compile the same code - * against Scala 2.12, 2.13, 3.0 - * - * Remove these classes as soon as support for Scala 2.12 is dropped! - */ -@InternalApi private[pekko] object PartialFunction { - - inline def fromFunction[A, B](f: A => B): scala.PartialFunction[A, B] = - scala.PartialFunction.fromFunction(f) - -} diff --git a/actor/src/main/scala/org/apache/pekko/actor/AbstractFSM.scala b/actor/src/main/scala/org/apache/pekko/actor/AbstractFSM.scala index 64ca71a5c9..52bdd1331b 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/AbstractFSM.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/AbstractFSM.scala @@ -180,7 +180,7 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] { * called, not only the first one matching. */ final def onTransition(transitionHandler: Procedure2[S, S]): Unit = { - val pf: PartialFunction[(S, S), Unit] = pekko.compat.PartialFunction.fromFunction(transitionHandler(_: S, _: S)) + val pf: PartialFunction[(S, S), Unit] = PartialFunction.fromFunction(transitionHandler(_: S, _: S)) super.onTransition(pf) } diff --git a/actor/src/main/scala/org/apache/pekko/actor/ActorSelection.scala b/actor/src/main/scala/org/apache/pekko/actor/ActorSelection.scala index 049d39b9ce..e20602b447 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/ActorSelection.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/ActorSelection.scala @@ -31,7 +31,6 @@ import pekko.dispatch.ExecutionContexts import pekko.pattern.ask import pekko.routing.MurmurHash import pekko.util.{ Helpers, JavaDurationConverters, Timeout } -import pekko.util.ccompat._ import pekko.util.FutureConverters /** @@ -39,7 +38,6 @@ import pekko.util.FutureConverters * allowing for broadcasting of messages to that section. */ @SerialVersionUID(1L) -@ccompatUsedUntil213 abstract class ActorSelection extends Serializable { this: ScalaActorSelection => diff --git a/actor/src/main/scala/org/apache/pekko/actor/FaultHandling.scala b/actor/src/main/scala/org/apache/pekko/actor/FaultHandling.scala index 4aa1641046..5e400981ef 100644 --- a/actor/src/main/scala/org/apache/pekko/actor/FaultHandling.scala +++ b/actor/src/main/scala/org/apache/pekko/actor/FaultHandling.scala @@ -19,7 +19,6 @@ import pekko.event.Logging import pekko.event.Logging.{ Error, LogEvent, LogLevel } import pekko.japi.Util.immutableSeq import pekko.util.JavaDurationConverters._ -import pekko.util.ccompat._ import java.lang.reflect.InvocationTargetException import java.lang.{ Iterable => JIterable } @@ -44,7 +43,6 @@ private[pekko] case object ChildNameReserved extends ChildStats * ChildRestartStats is the statistics kept by every parent Actor for every child Actor * and is used for SupervisorStrategies to know how to deal with problems that occur for the children. */ -@ccompatUsedUntil213 final case class ChildRestartStats( child: ActorRef, var maxNrOfRetriesCount: Int = 0, diff --git a/actor/src/main/scala-2.13+/org/apache/pekko/compat/Future.scala b/actor/src/main/scala/org/apache/pekko/compat/Future.scala similarity index 100% rename from actor/src/main/scala-2.13+/org/apache/pekko/compat/Future.scala rename to actor/src/main/scala/org/apache/pekko/compat/Future.scala diff --git a/actor/src/main/scala/org/apache/pekko/io/Dns.scala b/actor/src/main/scala/org/apache/pekko/io/Dns.scala index 5aacd722a8..c112540bcc 100644 --- a/actor/src/main/scala/org/apache/pekko/io/Dns.scala +++ b/actor/src/main/scala/org/apache/pekko/io/Dns.scala @@ -26,7 +26,6 @@ import pekko.annotation.DoNotInherit import pekko.annotation.InternalApi import pekko.event.Logging import pekko.io.dns.DnsProtocol -import pekko.util.ccompat._ import pekko.util.unused /** @@ -34,7 +33,6 @@ import pekko.util.unused * * This used to be a supported extension point but will be removed in future versions of Apache Pekko. */ -@ccompatUsedUntil213 @DoNotInherit abstract class Dns { diff --git a/actor/src/main/scala/org/apache/pekko/io/Udp.scala b/actor/src/main/scala/org/apache/pekko/io/Udp.scala index 61a3f82898..a137bf738d 100644 --- a/actor/src/main/scala/org/apache/pekko/io/Udp.scala +++ b/actor/src/main/scala/org/apache/pekko/io/Udp.scala @@ -26,7 +26,6 @@ import pekko.actor._ import pekko.io.Inet.{ SoJavaFactories, SocketOption } import pekko.util.ByteString import pekko.util.Helpers.Requiring -import pekko.util.ccompat._ /** * UDP Extension for Akka’s IO layer. @@ -40,7 +39,6 @@ import pekko.util.ccompat._ * * The Java API for generating UDP commands is available at [[UdpMessage]]. */ -@ccompatUsedUntil213 object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider { override def lookup = Udp diff --git a/actor/src/main/scala/org/apache/pekko/io/UdpConnected.scala b/actor/src/main/scala/org/apache/pekko/io/UdpConnected.scala index a74935ed55..e1912df49f 100644 --- a/actor/src/main/scala/org/apache/pekko/io/UdpConnected.scala +++ b/actor/src/main/scala/org/apache/pekko/io/UdpConnected.scala @@ -25,7 +25,6 @@ import pekko.actor._ import pekko.io.Inet.SocketOption import pekko.io.Udp.UdpSettings import pekko.util.ByteString -import pekko.util.ccompat._ /** * UDP Extension for Akka’s IO layer. @@ -39,7 +38,6 @@ import pekko.util.ccompat._ * * The Java API for generating UDP commands is available at [[UdpConnectedMessage]]. */ -@ccompatUsedUntil213 object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvider { override def lookup = UdpConnected diff --git a/actor/src/main/scala/org/apache/pekko/io/dns/DnsSettings.scala b/actor/src/main/scala/org/apache/pekko/io/dns/DnsSettings.scala index fa95206e61..6fe0e095ea 100644 --- a/actor/src/main/scala/org/apache/pekko/io/dns/DnsSettings.scala +++ b/actor/src/main/scala/org/apache/pekko/io/dns/DnsSettings.scala @@ -49,12 +49,10 @@ import pekko.io.dns.internal.{ ResolvConf, ResolvConfParser } import pekko.util.Helpers import pekko.util.Helpers.Requiring import pekko.util.JavaDurationConverters._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** INTERNAL API */ @InternalApi -@ccompatUsedUntil213 private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) { import DnsSettings._ diff --git a/actor/src/main/scala/org/apache/pekko/routing/MurmurHash.scala b/actor/src/main/scala/org/apache/pekko/routing/MurmurHash.scala index 24183fa2f3..d1983d9ee3 100644 --- a/actor/src/main/scala/org/apache/pekko/routing/MurmurHash.scala +++ b/actor/src/main/scala/org/apache/pekko/routing/MurmurHash.scala @@ -35,8 +35,6 @@ import java.lang.Integer.{ rotateLeft => rotl } import scala.annotation.nowarn -import org.apache.pekko.util.ccompat._ - /** * An object designed to generate well-distributed non-cryptographic * hashes. It is designed to hash a collection of integers; along with @@ -46,7 +44,6 @@ import org.apache.pekko.util.ccompat._ * incorporate a new integer) to update the values. Only one method * needs to be called to finalize the hash. */ -@ccompatUsedUntil213 object MurmurHash { // Magic values used for MurmurHash's 32 bit hash. // Don't change these without consulting a hashing expert! diff --git a/actor/src/main/scala/org/apache/pekko/routing/RoutedActorCell.scala b/actor/src/main/scala/org/apache/pekko/routing/RoutedActorCell.scala index cc8e6ae6ba..5150037ef5 100644 --- a/actor/src/main/scala/org/apache/pekko/routing/RoutedActorCell.scala +++ b/actor/src/main/scala/org/apache/pekko/routing/RoutedActorCell.scala @@ -30,7 +30,6 @@ import pekko.actor.SupervisorStrategy import pekko.actor.Terminated import pekko.dispatch.Envelope import pekko.dispatch.MessageDispatcher -import pekko.util.ccompat._ /** * INTERNAL API @@ -46,7 +45,6 @@ private[pekko] object RoutedActorCell { /** * INTERNAL API */ -@ccompatUsedUntil213 private[pekko] class RoutedActorCell( _system: ActorSystemImpl, _ref: InternalActorRef, diff --git a/actor/src/main/scala/org/apache/pekko/serialization/Serialization.scala b/actor/src/main/scala/org/apache/pekko/serialization/Serialization.scala index 11f9c331fc..e76d6e5de2 100644 --- a/actor/src/main/scala/org/apache/pekko/serialization/Serialization.scala +++ b/actor/src/main/scala/org/apache/pekko/serialization/Serialization.scala @@ -32,9 +32,7 @@ import org.apache.pekko import pekko.actor._ import pekko.annotation.InternalApi import pekko.event.{ LogMarker, Logging, LoggingAdapter } -import pekko.util.ccompat._ -@ccompatUsedUntil213 object Serialization { /** diff --git a/actor/src/main/scala-2.13+/org/apache/pekko/util/FunctionConverters.scala b/actor/src/main/scala/org/apache/pekko/util/FunctionConverters.scala similarity index 100% rename from actor/src/main/scala-2.13+/org/apache/pekko/util/FunctionConverters.scala rename to actor/src/main/scala/org/apache/pekko/util/FunctionConverters.scala diff --git a/actor/src/main/scala/org/apache/pekko/util/SubclassifiedIndex.scala b/actor/src/main/scala/org/apache/pekko/util/SubclassifiedIndex.scala index d854059133..256ff53c35 100644 --- a/actor/src/main/scala/org/apache/pekko/util/SubclassifiedIndex.scala +++ b/actor/src/main/scala/org/apache/pekko/util/SubclassifiedIndex.scala @@ -15,8 +15,6 @@ package org.apache.pekko.util import scala.collection.immutable -import org.apache.pekko.util.ccompat._ - /** * Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`! */ @@ -88,7 +86,6 @@ private[pekko] object SubclassifiedIndex { * cache, e.g. HashMap, is faster than tree traversal which must use linear * scan at each level. Therefore, no value traversals are published. */ -@ccompatUsedUntil213 private[pekko] class SubclassifiedIndex[K, V] private (protected var values: Set[V])( implicit sc: Subclassification[K]) { diff --git a/actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/package.scala b/actor/src/main/scala/org/apache/pekko/util/ccompat/package.scala similarity index 100% rename from actor/src/main/scala-2.13+/org/apache/pekko/util/ccompat/package.scala rename to actor/src/main/scala/org/apache/pekko/util/ccompat/package.scala diff --git a/cluster-metrics/src/main/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializer.scala b/cluster-metrics/src/main/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializer.scala index 0ce1047548..2f9d712f83 100644 --- a/cluster-metrics/src/main/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializer.scala +++ b/cluster-metrics/src/main/scala/org/apache/pekko/cluster/metrics/protobuf/MessageSerializer.scala @@ -27,13 +27,11 @@ import pekko.dispatch.Dispatchers import pekko.protobufv3.internal.MessageLite import pekko.remote.ByteStringUtils import pekko.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers } -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * Protobuf serializer for [[pekko.cluster.metrics.ClusterMetricsMessage]] types. */ -@ccompatUsedUntil213 class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { private final val BufferSize = 4 * 1024 diff --git a/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala b/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala index a3f91af3b8..d979a93fae 100644 --- a/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala +++ b/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesPerfSpec.scala @@ -22,13 +22,11 @@ import pekko.actor._ import pekko.cluster.MemberStatus import pekko.cluster.sharding.ShardRegion.{ CurrentShardRegionState, GetShardRegionState, Passivate } import pekko.testkit._ -import pekko.util.ccompat._ import com.typesafe.config.ConfigFactory import org.HdrHistogram.Histogram import scala.concurrent.duration._ -@ccompatUsedUntil213 object ClusterShardingRememberEntitiesPerfSpec { val NrRegions = 6 // use 5 for "real" testing diff --git a/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/typed/ClusterShardingPreparingForShutdownSpec.scala b/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/typed/ClusterShardingPreparingForShutdownSpec.scala index 8cac517f49..50c9456c3b 100644 --- a/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/typed/ClusterShardingPreparingForShutdownSpec.scala +++ b/cluster-sharding-typed/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/typed/ClusterShardingPreparingForShutdownSpec.scala @@ -17,7 +17,6 @@ import org.apache.pekko import pekko.actor.testkit.typed.scaladsl.TestProbe import pekko.actor.typed.ActorRef import pekko.actor.typed.Behavior -import pekko.util.ccompat._ import pekko.actor.typed.scaladsl.Behaviors import pekko.cluster.MemberStatus import pekko.cluster.MemberStatus.Removed @@ -71,7 +70,6 @@ class ClusterShardingPreparingForShutdownMultiJvmNode1 extends ClusterShardingPr class ClusterShardingPreparingForShutdownMultiJvmNode2 extends ClusterShardingPreparingForShutdownSpec class ClusterShardingPreparingForShutdownMultiJvmNode3 extends ClusterShardingPreparingForShutdownSpec -@ccompatUsedUntil213 class ClusterShardingPreparingForShutdownSpec extends MultiNodeSpec(ClusterShardingPreparingForShutdownSpec) with MultiNodeTypedClusterSpec { diff --git a/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/ReplicatedShardingSpec.scala b/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/ReplicatedShardingSpec.scala index 13354dd981..741106c1c0 100644 --- a/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/ReplicatedShardingSpec.scala +++ b/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/ReplicatedShardingSpec.scala @@ -42,10 +42,8 @@ import pekko.cluster.sharding.typed.ReplicatedShardingSpec.MyReplicatedIntSet import pekko.cluster.sharding.typed.ReplicatedShardingSpec.MyReplicatedStringSet import pekko.persistence.typed.ReplicationId import com.typesafe.config.Config -import pekko.util.ccompat._ import org.scalatest.time.Span -@ccompatUsedUntil213 object ReplicatedShardingSpec { def commonConfig = ConfigFactory.parseString(""" pekko.loglevel = DEBUG diff --git a/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala b/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala index 774a162f94..b65a0a3a05 100644 --- a/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala +++ b/cluster-sharding-typed/src/test/scala/org/apache/pekko/cluster/sharding/typed/scaladsl/ClusterShardingSpec.scala @@ -39,9 +39,7 @@ import pekko.cluster.typed.Leave import pekko.pattern.AskTimeoutException import pekko.serialization.jackson.CborSerializable import pekko.util.Timeout -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardingSpec { val config = ConfigFactory.parseString(s""" pekko.actor.provider = cluster diff --git a/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala b/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala index 0aef90d30d..0a62af78d2 100644 --- a/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala +++ b/cluster-sharding/src/main/scala/org/apache/pekko/cluster/sharding/protobuf/ClusterShardingMessageSerializer.scala @@ -41,13 +41,11 @@ import pekko.protobufv3.internal.MessageLite import pekko.serialization.BaseSerializer import pekko.serialization.Serialization import pekko.serialization.SerializerWithStringManifest -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * INTERNAL API: Protobuf serializer of ClusterSharding messages. */ -@ccompatUsedUntil213 private[pekko] class ClusterShardingMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowning2Spec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowning2Spec.scala index d099c9719f..72bcf5bbaf 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowning2Spec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowning2Spec.scala @@ -23,9 +23,7 @@ import pekko.cluster.MemberStatus import pekko.remote.transport.ThrottlerTransportAdapter.Direction import pekko.serialization.jackson.CborSerializable import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardCoordinatorDowning2Spec { case class Ping(id: String) extends CborSerializable diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowningSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowningSpec.scala index 10119414b1..7d1b954c4b 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowningSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardCoordinatorDowningSpec.scala @@ -23,9 +23,7 @@ import pekko.cluster.MemberStatus import pekko.remote.transport.ThrottlerTransportAdapter.Direction import pekko.serialization.jackson.CborSerializable import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardCoordinatorDowningSpec { case class Ping(id: String) extends CborSerializable diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingFailureSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingFailureSpec.scala index 1632d82b41..b6f29aedb6 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingFailureSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingFailureSpec.scala @@ -23,9 +23,7 @@ import pekko.remote.testconductor.RoleName import pekko.remote.transport.ThrottlerTransportAdapter.Direction import pekko.serialization.jackson.CborSerializable import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardingFailureSpec { case class Get(id: String) extends CborSerializable case class Add(id: String, i: Int) extends CborSerializable diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeavingSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeavingSpec.scala index 408de11450..0db94cd150 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeavingSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingLeavingSpec.scala @@ -20,9 +20,7 @@ import pekko.actor.{ Actor, ActorRef, Props } import pekko.cluster.MemberStatus import pekko.serialization.jackson.CborSerializable import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardingLeavingSpec { case class Ping(id: String) extends CborSerializable diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingMinMembersSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingMinMembersSpec.scala index 2f565a12c2..668a3b582c 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingMinMembersSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingMinMembersSpec.scala @@ -20,9 +20,7 @@ import pekko.cluster.MemberStatus import pekko.cluster.sharding.ShardCoordinator.ShardAllocationStrategy import pekko.cluster.sharding.ShardRegion.{ ClusterShardingStats, GetClusterShardingStats } import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 abstract class ClusterShardingMinMembersSpecConfig(mode: String) extends MultiNodeClusterShardingConfig( mode, diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala index a24c33d306..368214fec1 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/ClusterShardingRememberEntitiesSpec.scala @@ -22,9 +22,7 @@ import pekko.actor._ import pekko.cluster.{ Cluster, MemberStatus } import pekko.testkit._ import pekko.remote.testkit.MultiNodeSpec -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterShardingRememberEntitiesSpec { val extractEntityId: ShardRegion.ExtractEntityId = { diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiDcClusterShardingSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiDcClusterShardingSpec.scala index f944b10980..d180ab0538 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiDcClusterShardingSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiDcClusterShardingSpec.scala @@ -24,9 +24,7 @@ import pekko.cluster.sharding.ShardRegion.{ CurrentRegions, GetCurrentRegions } import pekko.remote.testconductor.RoleName import pekko.serialization.jackson.CborSerializable import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object MultiDcClusterShardingSpec { sealed trait EntityMsg extends CborSerializable { def id: String diff --git a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiNodeClusterShardingSpec.scala b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiNodeClusterShardingSpec.scala index 43d25fe402..8d50340cf9 100644 --- a/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiNodeClusterShardingSpec.scala +++ b/cluster-sharding/src/multi-jvm/scala/org/apache/pekko/cluster/sharding/MultiNodeClusterShardingSpec.scala @@ -25,11 +25,9 @@ import pekko.persistence.journal.leveldb.{ SharedLeveldbJournal, SharedLeveldbSt import pekko.remote.testconductor.RoleName import pekko.serialization.jackson.CborSerializable import pekko.testkit.{ TestActors, TestProbe } -import pekko.util.ccompat._ import scala.annotation.nowarn -@ccompatUsedUntil213 object MultiNodeClusterShardingSpec { object EntityActor { diff --git a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala index c232989b8e..7cdb857d02 100644 --- a/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala +++ b/cluster-sharding/src/test/scala/org/apache/pekko/cluster/sharding/CoordinatedShutdownShardingSpec.scala @@ -27,7 +27,6 @@ import pekko.testkit.PekkoSpec import pekko.testkit.TestActors.EchoActor import pekko.testkit.TestProbe import pekko.testkit.WithLogCapturing -import pekko.util.ccompat._ object CoordinatedShutdownShardingSpec { val config = @@ -50,7 +49,6 @@ object CoordinatedShutdownShardingSpec { } } -@ccompatUsedUntil213 class CoordinatedShutdownShardingSpec extends PekkoSpec(CoordinatedShutdownShardingSpec.config) with WithLogCapturing { import CoordinatedShutdownShardingSpec._ diff --git a/cluster-tools/src/main/scala/org/apache/pekko/cluster/client/ClusterClient.scala b/cluster-tools/src/main/scala/org/apache/pekko/cluster/client/ClusterClient.scala index 54881a9fa2..5956af5c68 100644 --- a/cluster-tools/src/main/scala/org/apache/pekko/cluster/client/ClusterClient.scala +++ b/cluster-tools/src/main/scala/org/apache/pekko/cluster/client/ClusterClient.scala @@ -51,10 +51,8 @@ import pekko.remote.DeadlineFailureDetector import pekko.routing.ConsistentHash import pekko.routing.MurmurHash import pekko.util.MessageBuffer -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ -@ccompatUsedUntil213 @deprecated( "Use Pekko gRPC instead, see https://pekko.apache.org/docs/pekko/current/cluster-client.html#migration-to-pekko-grpc", since = "Akka 2.6.0") diff --git a/cluster-tools/src/main/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala b/cluster-tools/src/main/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala index 23c1c9dcac..865c573ad9 100644 --- a/cluster-tools/src/main/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala +++ b/cluster-tools/src/main/scala/org/apache/pekko/cluster/pubsub/protobuf/DistributedPubSubMessageSerializer.scala @@ -28,13 +28,11 @@ import pekko.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm } import pekko.protobufv3.internal.{ ByteString, MessageLite } import pekko.remote.ByteStringUtils import pekko.serialization._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * INTERNAL API: Protobuf serializer of DistributedPubSubMediator messages. */ -@ccompatUsedUntil213 private[pekko] class DistributedPubSubMessageSerializer(val system: ExtendedActorSystem) extends SerializerWithStringManifest with BaseSerializer { diff --git a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManagerDownedSpec.scala b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManagerDownedSpec.scala index eea59ea44c..c3bbb8754a 100644 --- a/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManagerDownedSpec.scala +++ b/cluster-tools/src/multi-jvm/scala/org/apache/pekko/cluster/singleton/ClusterSingletonManagerDownedSpec.scala @@ -30,9 +30,7 @@ import pekko.remote.testkit.MultiNodeSpec import pekko.remote.testkit.STMultiNodeSpec import pekko.remote.transport.ThrottlerTransportAdapter import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ClusterSingletonManagerDownedSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala index c7546db178..75b3e971d9 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonLeavingSpeedSpec.scala @@ -83,8 +83,6 @@ class ClusterSingletonLeavingSpeedSpec override def expectedTestDuration: FiniteDuration = 10.minutes - import pekko.util.ccompat._ - @ccompatUsedUntil213 def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = { from.actorOf( diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala index 94846fc3cd..ce10330e5f 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestart2Spec.scala @@ -69,8 +69,6 @@ class ClusterSingletonRestart2Spec ConfigFactory.parseString("pekko.cluster.roles = [other]").withFallback(system.settings.config)) var sys4: ActorSystem = null - import pekko.util.ccompat._ - @ccompatUsedUntil213 def join(from: ActorSystem, to: ActorSystem): Unit = { if (Cluster(from).selfRoles.contains("singleton")) from.actorOf( diff --git a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala index b9acf76dc2..5ab67e494a 100644 --- a/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala +++ b/cluster-tools/src/test/scala/org/apache/pekko/cluster/singleton/ClusterSingletonRestartSpec.scala @@ -48,8 +48,6 @@ class ClusterSingletonRestartSpec val sys2 = ActorSystem(system.name, system.settings.config) var sys3: ActorSystem = null - import pekko.util.ccompat._ - @ccompatUsedUntil213 def join(from: ActorSystem, to: ActorSystem): Unit = { from.actorOf( ClusterSingletonManager.props( diff --git a/cluster-typed/src/main/scala-2.12/org/apache/pekko/cluster/typed/internal/receptionist/ClusterReceptionistProtocol.scala b/cluster-typed/src/main/scala-2.12/org/apache/pekko/cluster/typed/internal/receptionist/ClusterReceptionistProtocol.scala deleted file mode 100644 index ecb02231df..0000000000 --- a/cluster-typed/src/main/scala-2.12/org/apache/pekko/cluster/typed/internal/receptionist/ClusterReceptionistProtocol.scala +++ /dev/null @@ -1,27 +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, which was derived from Akka. - */ - -/* - * Copyright (C) 2021-2022 Lightbend Inc. - */ - -package org.apache.pekko.cluster.typed.internal.receptionist - -import org.apache.pekko -import pekko.actor.typed.ActorRef -import pekko.actor.typed.internal.receptionist.{ AbstractServiceKey, ReceptionistMessages } -import pekko.annotation.InternalApi - -/** - * INTERNAL API - */ -@InternalApi -private[receptionist] object ClusterReceptionistProtocol { - type SubscriptionsKV[K <: AbstractServiceKey] = ActorRef[ReceptionistMessages.Listing[K#Protocol]] -} diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/ClusterEvent.scala b/cluster/src/main/scala/org/apache/pekko/cluster/ClusterEvent.scala index b83be7fb7c..f9515b2fc6 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/ClusterEvent.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/ClusterEvent.scala @@ -27,7 +27,6 @@ import pekko.cluster.ClusterSettings.DataCenter import pekko.cluster.MemberStatus._ import pekko.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import pekko.event.EventStream -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** @@ -411,7 +410,6 @@ object ClusterEvent { * The nodes that have seen current version of the Gossip. */ @InternalApi - @ccompatUsedUntil213 private[cluster] final case class SeenChanged(convergence: Boolean, seenBy: Set[Address]) extends ClusterDomainEvent /** diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/ClusterHeartbeat.scala b/cluster/src/main/scala/org/apache/pekko/cluster/ClusterHeartbeat.scala index 199f1124a5..c4136c8643 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/ClusterHeartbeat.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/ClusterHeartbeat.scala @@ -32,7 +32,6 @@ import pekko.event.ActorWithLogClass import pekko.event.Logging import pekko.remote.FailureDetectorRegistry import pekko.remote.HeartbeatMessage -import pekko.util.ccompat._ /** * INTERNAL API. @@ -40,7 +39,6 @@ import pekko.util.ccompat._ * Receives Heartbeat messages and replies. */ @InternalApi -@ccompatUsedUntil213 private[cluster] final class ClusterHeartbeatReceiver(getCluster: () => Cluster) extends Actor { import ClusterHeartbeatSender._ diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/CrossDcClusterHeartbeat.scala b/cluster/src/main/scala/org/apache/pekko/cluster/CrossDcClusterHeartbeat.scala index 9c786cc899..671c3b030b 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/CrossDcClusterHeartbeat.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/CrossDcClusterHeartbeat.scala @@ -25,7 +25,6 @@ import pekko.event.ActorWithLogClass import pekko.event.Logging import pekko.remote.FailureDetectorRegistry import pekko.util.ConstantFun -import pekko.util.ccompat._ /** * INTERNAL API @@ -45,7 +44,6 @@ import pekko.util.ccompat._ * nodes which aggressively come and go as the traffic in the service changes. */ @InternalApi -@ccompatUsedUntil213 private[cluster] class CrossDcHeartbeatSender extends Actor { import CrossDcHeartbeatSender._ diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/JoinConfigCompatChecker.scala b/cluster/src/main/scala/org/apache/pekko/cluster/JoinConfigCompatChecker.scala index d92b2ebb2f..1f303e784c 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/JoinConfigCompatChecker.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/JoinConfigCompatChecker.scala @@ -20,7 +20,6 @@ import com.typesafe.config.{ Config, ConfigFactory, ConfigValue } import org.apache.pekko import pekko.actor.ExtendedActorSystem import pekko.annotation.{ DoNotInherit, InternalApi } -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ abstract class JoinConfigCompatChecker { @@ -107,7 +106,6 @@ object JoinConfigCompatChecker { * information that users may have added to their configuration. */ @InternalApi - @ccompatUsedUntil213 private[cluster] def filterWithKeys(requiredKeys: im.Seq[String], config: Config): Config = { val filtered = for { diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/MembershipState.scala b/cluster/src/main/scala/org/apache/pekko/cluster/MembershipState.scala index 72624215c4..136826cfb6 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/MembershipState.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/MembershipState.scala @@ -24,12 +24,10 @@ import org.apache.pekko import pekko.annotation.InternalApi import pekko.cluster.ClusterSettings.DataCenter import pekko.cluster.MemberStatus._ -import pekko.util.ccompat._ /** * INTERNAL API */ -@ccompatUsedUntil213 @InternalApi private[pekko] object MembershipState { import MemberStatus._ private val leaderMemberStatus = Set[MemberStatus](Up, Leaving, PreparingForShutdown, ReadyForShutdown) diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/Reachability.scala b/cluster/src/main/scala/org/apache/pekko/cluster/Reachability.scala index cee5e4bff9..3df1f913f9 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/Reachability.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/Reachability.scala @@ -17,12 +17,10 @@ import scala.collection.immutable import org.apache.pekko import pekko.annotation.InternalApi -import pekko.util.ccompat._ /** * INTERNAL API */ -@ccompatUsedUntil213 private[cluster] object Reachability { val empty = new Reachability(Vector.empty, Map.empty) diff --git a/cluster/src/main/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializer.scala b/cluster/src/main/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializer.scala index a97b54e63c..8ae9181eaf 100644 --- a/cluster/src/main/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializer.scala +++ b/cluster/src/main/scala/org/apache/pekko/cluster/protobuf/ClusterMessageSerializer.scala @@ -31,14 +31,12 @@ import pekko.remote.ByteStringUtils import pekko.routing.Pool import pekko.serialization._ import pekko.util.Version -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * INTERNAL API */ @InternalApi -@ccompatUsedUntil213 private[pekko] object ClusterMessageSerializer { // Kept for one version iteration from Akka 2.6.4 to allow rolling migration to short manifests // can be removed in Akka 2.6.6 or later. diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterShutdownSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterShutdownSpec.scala index 029f8bc607..2cf9333643 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterShutdownSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/ClusterShutdownSpec.scala @@ -16,7 +16,6 @@ package org.apache.pekko.cluster import org.apache.pekko import pekko.cluster.MemberStatus.Removed import pekko.remote.testkit.MultiNodeConfig -import pekko.util.ccompat._ import org.scalatest.concurrent.Eventually import scala.concurrent.duration._ @@ -35,7 +34,6 @@ class ClusterShutdownSpecMultiJvmNode2 extends ClusterShutdownSpec class ClusterShutdownSpecMultiJvmNode3 extends ClusterShutdownSpec class ClusterShutdownSpecMultiJvmNode4 extends ClusterShutdownSpec -@ccompatUsedUntil213 abstract class ClusterShutdownSpec extends MultiNodeClusterSpec(ClusterShutdownSpec) with Eventually { import ClusterShutdownSpec._ diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MinMembersBeforeUpSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MinMembersBeforeUpSpec.scala index 36820f32de..e2c84eb121 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MinMembersBeforeUpSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MinMembersBeforeUpSpec.scala @@ -20,9 +20,7 @@ import pekko.cluster.MemberStatus._ import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcHeartbeatTakingOverSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcHeartbeatTakingOverSpec.scala index fa999c6571..5d800bb6e6 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcHeartbeatTakingOverSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiDcHeartbeatTakingOverSpec.scala @@ -26,9 +26,7 @@ import pekko.annotation.InternalApi import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig { val first = role("first") // alpha val second = role("second") // alpha diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiNodeClusterSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiNodeClusterSpec.scala index 37eb80f67b..a8edcf9ae9 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiNodeClusterSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/MultiNodeClusterSpec.scala @@ -35,9 +35,7 @@ import pekko.remote.testkit.{ MultiNodeSpec, STMultiNodeSpec } import pekko.serialization.jackson.CborSerializable import pekko.testkit._ import pekko.testkit.TestEvent._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object MultiNodeClusterSpec { def clusterConfigWithFailureDetectorPuppet: Config = diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeMembershipSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeMembershipSpec.scala index bf1998fbc4..d613f1a5ae 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeMembershipSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/NodeMembershipSpec.scala @@ -16,9 +16,7 @@ package org.apache.pekko.cluster import org.apache.pekko import pekko.remote.testkit.MultiNodeConfig import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object NodeMembershipMultiJvmSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala index 6b43c7a37d..a9264b8384 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartFirstSeedNodeSpec.scala @@ -29,9 +29,7 @@ import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig { val seed1 = role("seed1") val seed2 = role("seed2") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala index 1c12f00663..c7e2be84ba 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode2Spec.scala @@ -29,9 +29,7 @@ import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig { val seed1 = role("seed1") val seed2 = role("seed2") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala index 55bcf7ccc5..5c19d769f9 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNode3Spec.scala @@ -30,9 +30,7 @@ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec import pekko.remote.transport.ThrottlerTransportAdapter.Direction import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object RestartNode3MultiJvmSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala index ffb76c21cb..734418c83e 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/RestartNodeSpec.scala @@ -34,9 +34,7 @@ import pekko.cluster.MemberStatus._ import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object RestartNodeMultiJvmSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala index 5eadcd4e41..75c93abb0d 100644 --- a/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala +++ b/cluster/src/multi-jvm/scala/org/apache/pekko/cluster/UnreachableNodeJoinsAgainSpec.scala @@ -29,9 +29,7 @@ import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.remote.transport.ThrottlerTransportAdapter.Direction import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/DeltaPropagationSelector.scala b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/DeltaPropagationSelector.scala index 9cc96c9ab6..89083fcbe1 100644 --- a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/DeltaPropagationSelector.scala +++ b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/DeltaPropagationSelector.scala @@ -21,13 +21,11 @@ import pekko.cluster.UniqueAddress import pekko.cluster.ddata.Key.KeyId import pekko.cluster.ddata.Replicator.Internal.DeltaPropagation import pekko.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder -import pekko.util.ccompat._ /** * INTERNAL API: Used by the Replicator actor. * Extracted to separate trait to make it easy to test. */ -@ccompatUsedUntil213 @InternalApi private[pekko] trait DeltaPropagationSelector { diff --git a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/Replicator.scala b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/Replicator.scala index e3ff9ccc2f..df9a774291 100644 --- a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/Replicator.scala +++ b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/Replicator.scala @@ -69,9 +69,7 @@ import pekko.serialization.SerializationExtension import pekko.util.ByteString import pekko.util.Helpers.toRootLowerCase import pekko.util.JavaDurationConverters._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object ReplicatorSettings { /** diff --git a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializer.scala b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializer.scala index fd53bc2b74..3c62b7fe07 100644 --- a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializer.scala +++ b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatedDataSerializer.scala @@ -36,10 +36,8 @@ import pekko.serialization.BaseSerializer import pekko.serialization.Serialization import pekko.serialization.SerializerWithStringManifest import pekko.util.ByteString.UTF_8 -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ -@ccompatUsedUntil213 private object ReplicatedDataSerializer { /* * Generic superclass to allow to compare Entry types used in protobuf. diff --git a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala index 528b23fe7c..20172b1f26 100644 --- a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala +++ b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/ReplicatorMessageSerializer.scala @@ -41,13 +41,11 @@ import pekko.serialization.BaseSerializer import pekko.serialization.Serialization import pekko.serialization.SerializerWithStringManifest import pekko.util.{ ByteString => PekkoByteString } -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * INTERNAL API */ -@ccompatUsedUntil213 @InternalApi private[pekko] object ReplicatorMessageSerializer { diff --git a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/SerializationSupport.scala b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/SerializationSupport.scala index b817b16cc7..421b6a65e0 100644 --- a/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/SerializationSupport.scala +++ b/distributed-data/src/main/scala/org/apache/pekko/cluster/ddata/protobuf/SerializationSupport.scala @@ -30,13 +30,11 @@ import pekko.protobufv3.internal.ByteString import pekko.protobufv3.internal.MessageLite import pekko.remote.ByteStringUtils import pekko.serialization._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * Some useful serialization helper methods. */ -@ccompatUsedUntil213 trait SerializationSupport { private final val BufferSize = 1024 * 4 diff --git a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala index 36edb54f96..9ceb596632 100644 --- a/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala +++ b/distributed-data/src/multi-jvm/scala/org/apache/pekko/cluster/ddata/DurablePruningSpec.scala @@ -27,9 +27,7 @@ import pekko.remote.testconductor.RoleName import pekko.remote.testkit.MultiNodeConfig import pekko.remote.testkit.MultiNodeSpec import pekko.testkit._ -import pekko.util.ccompat._ -@ccompatUsedUntil213 object DurablePruningSpec extends MultiNodeConfig { val first = role("first") val second = role("second") diff --git a/docs/src/main/paradox/general/configuration.md b/docs/src/main/paradox/general/configuration.md index 91017aa9aa..2a337e0c13 100644 --- a/docs/src/main/paradox/general/configuration.md +++ b/docs/src/main/paradox/general/configuration.md @@ -149,7 +149,7 @@ If in doubt, you can inspect your configuration objects before or after using them to construct an actor system: ``` -Welcome to Scala 2.12 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0). +Welcome to Scala 2.13 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0). Type in expressions to have them evaluated. Type :help for more information. diff --git a/legal/pekko-actor-jar-license.txt b/legal/pekko-actor-jar-license.txt index 3a0e6d27b2..28e5a16abf 100644 --- a/legal/pekko-actor-jar-license.txt +++ b/legal/pekko-actor-jar-license.txt @@ -212,7 +212,7 @@ Copyright (c) 2003-2011, LAMP/EPFL pekko-actor contains code from scala-collection-compat in the `org.apache.pekko.util.ccompat` package which has released under an Apache 2.0 license. -- actor/src/main/scala-2.12/org/apache/pekko/util/ccompat/package.scala +- actor/src/main/scala/org/apache/pekko/util/ccompat/package.scala Scala (https://www.scala-lang.org) diff --git a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala index 3c0a452a3b..148bd8f953 100644 --- a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala +++ b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testconductor/Player.scala @@ -37,9 +37,7 @@ import pekko.pattern.{ ask, AskTimeoutException } import pekko.remote.testconductor.RemoteConnection.getAddrString import pekko.remote.transport.ThrottlerTransportAdapter.{ Blackhole, SetThrottle, TokenBucket, Unthrottled } import pekko.util.Timeout -import pekko.util.ccompat._ -@ccompatUsedUntil213 object Player { final class Waiter extends Actor with RequiresMessageQueue[UnboundedMessageQueueSemantics] { diff --git a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala index 3b46c29822..c344647c65 100644 --- a/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala +++ b/multi-node-testkit/src/main/scala/org/apache/pekko/remote/testkit/MultiNodeSpec.scala @@ -34,12 +34,10 @@ import pekko.testkit._ import pekko.testkit.TestEvent._ import pekko.testkit.TestKit import pekko.util.Timeout -import pekko.util.ccompat._ /** * Configure the role names and participants of the test, including configuration settings. */ -@ccompatUsedUntil213 abstract class MultiNodeConfig { private var _commonConf: Option[Config] = None diff --git a/persistence/src/main/scala/org/apache/pekko/persistence/AtLeastOnceDelivery.scala b/persistence/src/main/scala/org/apache/pekko/persistence/AtLeastOnceDelivery.scala index a06e8beb07..7f32f5c839 100644 --- a/persistence/src/main/scala/org/apache/pekko/persistence/AtLeastOnceDelivery.scala +++ b/persistence/src/main/scala/org/apache/pekko/persistence/AtLeastOnceDelivery.scala @@ -22,9 +22,7 @@ import pekko.actor.Cancellable import pekko.actor.DeadLetterSuppression import pekko.annotation.InternalApi import pekko.persistence.serialization.Message -import pekko.util.ccompat._ -@ccompatUsedUntil213 object AtLeastOnceDelivery { /** diff --git a/persistence/src/main/scala/org/apache/pekko/persistence/journal/EventAdapters.scala b/persistence/src/main/scala/org/apache/pekko/persistence/journal/EventAdapters.scala index abf98ca69d..2ad060a694 100644 --- a/persistence/src/main/scala/org/apache/pekko/persistence/journal/EventAdapters.scala +++ b/persistence/src/main/scala/org/apache/pekko/persistence/journal/EventAdapters.scala @@ -26,12 +26,10 @@ import com.typesafe.config.Config import org.apache.pekko import pekko.actor.ExtendedActorSystem import pekko.event.{ Logging, LoggingAdapter } -import pekko.util.ccompat._ /** * `EventAdapters` serves as a per-journal collection of bound event adapters. */ -@ccompatUsedUntil213 class EventAdapters( map: ConcurrentHashMap[Class[_], EventAdapter], bindings: immutable.Seq[(Class[_], EventAdapter)], diff --git a/persistence/src/main/scala/org/apache/pekko/persistence/journal/japi/AsyncWriteJournal.scala b/persistence/src/main/scala/org/apache/pekko/persistence/journal/japi/AsyncWriteJournal.scala index 88d3bdf572..7d2b0e2b2b 100644 --- a/persistence/src/main/scala/org/apache/pekko/persistence/journal/japi/AsyncWriteJournal.scala +++ b/persistence/src/main/scala/org/apache/pekko/persistence/journal/japi/AsyncWriteJournal.scala @@ -24,13 +24,11 @@ import pekko.persistence._ import pekko.persistence.journal.{ AsyncWriteJournal => SAsyncWriteJournal } import pekko.util.ConstantFun.scalaAnyToUnit import pekko.util.FutureConverters._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** * Java API: abstract journal, optimized for asynchronous, non-blocking writes. */ -@ccompatUsedUntil213 abstract class AsyncWriteJournal extends AsyncRecovery with SAsyncWriteJournal with AsyncWritePlugin { import SAsyncWriteJournal.successUnit diff --git a/persistence/src/main/scala/org/apache/pekko/persistence/serialization/MessageSerializer.scala b/persistence/src/main/scala/org/apache/pekko/persistence/serialization/MessageSerializer.scala index b092057f0f..07bd76c683 100644 --- a/persistence/src/main/scala/org/apache/pekko/persistence/serialization/MessageSerializer.scala +++ b/persistence/src/main/scala/org/apache/pekko/persistence/serialization/MessageSerializer.scala @@ -29,7 +29,6 @@ import pekko.persistence.serialization.{ MessageFormats => mf } import pekko.protobufv3.internal.ByteString import pekko.protobufv3.internal.UnsafeByteOperations import pekko.serialization._ -import pekko.util.ccompat._ /** * Marker trait for all protobuf-serializable messages in `pekko.persistence`. @@ -39,7 +38,6 @@ trait Message extends Serializable /** * Protobuf serializer for [[pekko.persistence.PersistentRepr]], [[pekko.persistence.AtLeastOnceDelivery]] and [[pekko.persistence.fsm.PersistentFSM.StateChangeEvent]] messages. */ -@ccompatUsedUntil213 class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer { import PersistentRepr.Undefined diff --git a/persistence/src/main/scala/org/apache/pekko/persistence/snapshot/local/LocalSnapshotStore.scala b/persistence/src/main/scala/org/apache/pekko/persistence/snapshot/local/LocalSnapshotStore.scala index c9e30103ba..ffd45cdb56 100644 --- a/persistence/src/main/scala/org/apache/pekko/persistence/snapshot/local/LocalSnapshotStore.scala +++ b/persistence/src/main/scala/org/apache/pekko/persistence/snapshot/local/LocalSnapshotStore.scala @@ -31,14 +31,12 @@ import pekko.persistence.serialization._ import pekko.persistence.snapshot._ import pekko.serialization.SerializationExtension import pekko.util.ByteString.UTF_8 -import pekko.util.ccompat._ /** * INTERNAL API * * Local filesystem backed snapshot store. */ -@ccompatUsedUntil213 private[persistence] class LocalSnapshotStore(config: Config) extends SnapshotStore with ActorLogging { private val FilenamePattern = """^snapshot-(.+)-(\d+)-(\d+)""".r private val persistenceIdStartIdx = 9 // Persistence ID starts after the "snapshot-" substring diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 0dc7423f62..14d13caf02 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -43,10 +43,9 @@ object Dependencies { val jacksonCoreVersion = "2.20.0" val jacksonDatabindVersion = jacksonCoreVersion - val scala212Version = "2.12.20" val scala213Version = "2.13.16" val scala3Version = "3.3.6" - val allScalaVersions = Seq(scala213Version, scala212Version, scala3Version) + val allScalaVersions = Seq(scala213Version, scala3Version) val reactiveStreamsVersion = "1.0.4" diff --git a/remote/src/main/scala/org/apache/pekko/remote/Remoting.scala b/remote/src/main/scala/org/apache/pekko/remote/Remoting.scala index a6698cef54..51a7deae0f 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/Remoting.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/Remoting.scala @@ -30,7 +30,6 @@ import com.typesafe.config.Config import org.apache.pekko import pekko.Done import pekko.actor._ -import pekko.actor.ActorInitializationException import pekko.actor.SupervisorStrategy._ import pekko.annotation.InternalStableApi import pekko.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } @@ -44,7 +43,6 @@ import pekko.remote.transport.PekkoPduCodec.Message import pekko.remote.transport.Transport.{ ActorAssociationEventListener, AssociationEventListener, InboundAssociation } import pekko.util.ByteString.UTF_8 import pekko.util.OptionVal -import pekko.util.ccompat._ /** * INTERNAL API @@ -145,7 +143,6 @@ private[remote] object Remoting { * INTERNAL API */ @nowarn("msg=deprecated") -@ccompatUsedUntil213 private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends RemoteTransport(_system, _provider) { 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 7fdbdad170..0d40b416cb 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 @@ -68,7 +68,6 @@ import pekko.stream.scaladsl.Source import pekko.util.OptionVal import pekko.util.PrettyDuration._ import pekko.util.WildcardIndex -import pekko.util.ccompat._ /** * INTERNAL API @@ -139,7 +138,6 @@ private[remote] object Association { * Thread-safe, mutable holder for association state. Main entry point for remote destined message to a specific * remote address. */ -@ccompatUsedUntil213 private[remote] class Association( val transport: ArteryTransport, val materializer: Materializer, diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/RemoteInstrument.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/RemoteInstrument.scala index 7123483fd6..f61940865e 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/RemoteInstrument.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/RemoteInstrument.scala @@ -29,7 +29,6 @@ import pekko.annotation.InternalStableApi import pekko.event.Logging import pekko.event.LoggingAdapter import pekko.remote.RemoteActorRefProvider -import pekko.util.ccompat._ import pekko.util.OptionVal import pekko.util.unused @@ -45,7 +44,6 @@ import pekko.util.unused * will be created for each encoder and decoder. It's only called from the operator, so if it doesn't * delegate to any shared instance it doesn't have to be thread-safe. */ -@ccompatUsedUntil213 abstract class RemoteInstrument { /** diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala index 208c8afc80..bdfe30123d 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/aeron/ArteryAeronUdpTransport.scala @@ -57,12 +57,10 @@ import pekko.stream.scaladsl.Flow import pekko.stream.scaladsl.Keep import pekko.stream.scaladsl.Sink import pekko.stream.scaladsl.Source -import pekko.util.ccompat._ /** * INTERNAL API */ -@ccompatUsedUntil213 private[remote] class ArteryAeronUdpTransport(_system: ExtendedActorSystem, _provider: RemoteActorRefProvider) extends ArteryTransport(_system, _provider) { import AeronSource.AeronLifecycle diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/ArteryTcpTransport.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/ArteryTcpTransport.scala index aadfb60472..965d399ccf 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/ArteryTcpTransport.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/ArteryTcpTransport.scala @@ -59,7 +59,6 @@ import pekko.stream.scaladsl.Source import pekko.stream.scaladsl.Tcp import pekko.stream.scaladsl.Tcp.ServerBinding import pekko.util.{ ByteString, OptionVal } -import pekko.util.ccompat._ /** * INTERNAL API @@ -77,7 +76,6 @@ private[remote] object ArteryTcpTransport { /** * INTERNAL API */ -@ccompatUsedUntil213 private[remote] class ArteryTcpTransport( _system: ExtendedActorSystem, _provider: RemoteActorRefProvider, diff --git a/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/SSLEngineProvider.scala b/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/SSLEngineProvider.scala index 0d1bd66e37..23d554ef12 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/SSLEngineProvider.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/artery/tcp/SSLEngineProvider.scala @@ -17,11 +17,9 @@ package tcp import org.apache.pekko import pekko.actor.ExtendedActorSystem import pekko.actor.setup.Setup -import pekko.util.ccompat._ import javax.net.ssl.SSLEngine import javax.net.ssl.SSLSession -@ccompatUsedUntil213 trait SSLEngineProvider { def createServerSSLEngine(hostname: String, port: Int): SSLEngine diff --git a/remote/src/main/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializer.scala b/remote/src/main/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializer.scala index 78b0e34171..303f68ff52 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializer.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/serialization/DaemonMsgCreateSerializer.scala @@ -25,7 +25,6 @@ import pekko.remote.DaemonMsgCreate import pekko.remote.WireFormats.{ DaemonMsgCreateData, DeployData, PropsData } import pekko.routing.{ NoRouter, RouterConfig } import pekko.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest } -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ /** @@ -36,7 +35,6 @@ import pekko.util.ccompat.JavaConverters._ * * INTERNAL API */ -@ccompatUsedUntil213 private[pekko] final class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) extends BaseSerializer { import Deploy.NoDispatcherGiven import ProtobufSerializer.deserializeActorRef diff --git a/remote/src/main/scala/org/apache/pekko/remote/serialization/MessageContainerSerializer.scala b/remote/src/main/scala/org/apache/pekko/remote/serialization/MessageContainerSerializer.scala index 602a187de8..484f2af6ec 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/serialization/MessageContainerSerializer.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/serialization/MessageContainerSerializer.scala @@ -25,9 +25,7 @@ import pekko.protobufv3.internal.ByteString import pekko.remote.ByteStringUtils import pekko.remote.ContainerFormats import pekko.serialization.{ BaseSerializer, SerializationExtension, Serializers } -import pekko.util.ccompat._ -@ccompatUsedUntil213 class MessageContainerSerializer(val system: ExtendedActorSystem) extends BaseSerializer { private lazy val serialization = SerializationExtension(system) diff --git a/remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettySSLSupport.scala b/remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettySSLSupport.scala index 8551cddc5f..421ce5e782 100644 --- a/remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettySSLSupport.scala +++ b/remote/src/main/scala/org/apache/pekko/remote/transport/netty/NettySSLSupport.scala @@ -17,7 +17,6 @@ import scala.annotation.nowarn import com.typesafe.config.Config import org.apache.pekko -import pekko.util.ccompat.ccompatUsedUntil213 import pekko.util.ccompat.JavaConverters._ import io.netty.channel.Channel @@ -56,7 +55,6 @@ private[pekko] class SSLSettings(config: Config) { * Used for adding SSL support to Netty pipeline. * The `SSLEngine` is created via the configured [[SSLEngineProvider]]. */ -@ccompatUsedUntil213 @nowarn("msg=deprecated") private[pekko] object NettySSLSupport { diff --git a/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala b/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala index e69877cffa..eac1642be7 100644 --- a/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala +++ b/remote/src/test/scala/org/apache/pekko/remote/DaemonicSpec.scala @@ -20,10 +20,8 @@ import com.typesafe.config.ConfigFactory import org.apache.pekko import pekko.actor.{ ActorSystem, Address } import pekko.testkit._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ -@ccompatUsedUntil213 class DaemonicSpec extends PekkoSpec { "Remoting configured with daemonic = on" must { diff --git a/stream-testkit/src/main/scala/org/apache/pekko/stream/testkit/StreamTestKit.scala b/stream-testkit/src/main/scala/org/apache/pekko/stream/testkit/StreamTestKit.scala index f2bf528920..141e759792 100644 --- a/stream-testkit/src/main/scala/org/apache/pekko/stream/testkit/StreamTestKit.scala +++ b/stream-testkit/src/main/scala/org/apache/pekko/stream/testkit/StreamTestKit.scala @@ -35,7 +35,6 @@ import pekko.stream.impl._ import pekko.testkit.{ TestActor, TestProbe } import pekko.testkit.TestActor.AutoPilot import pekko.util.JavaDurationConverters._ -import pekko.util.ccompat._ import pekko.util.ccompat.JavaConverters._ import org.reactivestreams.{ Publisher, Subscriber, Subscription } @@ -120,7 +119,6 @@ object TestPublisher { type Self <: ManualProbe[I] - @ccompatUsedUntil213 private val probe: TestProbe = TestProbe() // this is a way to pause receiving message from probe until subscription is done diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CompressionTestingTools.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CompressionTestingTools.scala index 250f2f9379..7296c17cd4 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CompressionTestingTools.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/io/compression/CompressionTestingTools.scala @@ -22,10 +22,8 @@ import org.apache.pekko import pekko.stream.Materializer import pekko.stream.scaladsl.Source import pekko.util.ByteString -import pekko.util.ccompat._ // a few useful helpers copied over from pekko-http -@ccompatUsedUntil213 object CompressionTestingTools { implicit class AddFutureAwaitResult[T](val future: Future[T]) extends AnyVal { diff --git a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowStatefulMapConcatSpec.scala b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowStatefulMapConcatSpec.scala index 6e19228d37..5f3fda57d6 100644 --- a/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowStatefulMapConcatSpec.scala +++ b/stream-tests/src/test/scala/org/apache/pekko/stream/scaladsl/FlowStatefulMapConcatSpec.scala @@ -21,10 +21,8 @@ import pekko.stream.ActorAttributes import pekko.stream.Supervision import pekko.stream.testkit._ import pekko.stream.testkit.scaladsl.TestSink -import pekko.util.ccompat._ @nowarn("msg=deprecated") -@ccompatUsedUntil213 class FlowStatefulMapConcatSpec extends StreamSpec(""" pekko.stream.materializer.initial-input-buffer-size = 2 """) with ScriptedTest { diff --git a/stream/src/main/scala-2.12/org/apache/pekko/stream/javadsl/CollectionUtil.scala b/stream/src/main/scala-2.12/org/apache/pekko/stream/javadsl/CollectionUtil.scala deleted file mode 100644 index a77daba8a6..0000000000 --- a/stream/src/main/scala-2.12/org/apache/pekko/stream/javadsl/CollectionUtil.scala +++ /dev/null @@ -1,39 +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.stream -package javadsl - -import scala.collection.immutable - -import org.apache.pekko -import pekko.annotation.InternalApi -import pekko.japi.Util.immutableSeq - -/** - * INTERNAL API - * - * Utility methods for converting Java collections to Scala collections. - */ -@InternalApi -private[javadsl] object CollectionUtil { - @inline def toSeq[T](jlist: java.util.List[T]): immutable.Seq[T] = - immutableSeq(jlist) - - @inline def toSeq[T](jiterable: java.lang.Iterable[T]): immutable.Seq[T] = - immutableSeq(jiterable) -} diff --git a/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/MapConcat.scala b/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/MapConcat.scala index 6f765f2680..3af67f9e37 100644 --- a/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/MapConcat.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/MapConcat.scala @@ -28,13 +28,11 @@ import pekko.stream.Attributes.SourceLocation import pekko.stream.Supervision.Decider import pekko.stream.impl.Stages.DefaultAttributes import pekko.stream.stage.{ GraphStage, GraphStageLogic, InHandler, OutHandler } -import pekko.util.ccompat._ /** * INTERNAL API */ @InternalApi -@ccompatUsedUntil213 private[pekko] final class MapConcat[In, Out](f: In => IterableOnce[Out]) extends GraphStage[FlowShape[In, Out]] { require(f != null, "f function should not be null") diff --git a/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/Ops.scala b/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/Ops.scala index 3738579198..63f83777f9 100644 --- a/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/Ops.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/impl/fusing/Ops.scala @@ -53,7 +53,6 @@ import pekko.stream.scaladsl.{ } import pekko.stream.stage._ import pekko.util.{ unused, ConstantFun, OptionVal } -import pekko.util.ccompat._ /** * INTERNAL API @@ -2325,7 +2324,6 @@ private[pekko] final class StatefulMap[S, In, Out](create: () => S, f: (S, In) = * INTERNAL API */ @InternalApi -@ccompatUsedUntil213 private[pekko] final class StatefulMapConcat[In, Out](val factory: StatefulMapConcatAccumulatorFactory[In, Out]) extends GraphStage[FlowShape[In, Out]] { val in = Inlet[In]("StatefulMapConcat.in") diff --git a/stream/src/main/scala/org/apache/pekko/stream/javadsl/Flow.scala b/stream/src/main/scala/org/apache/pekko/stream/javadsl/Flow.scala index 6e76d89579..197223a46b 100755 --- a/stream/src/main/scala/org/apache/pekko/stream/javadsl/Flow.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/javadsl/Flow.scala @@ -40,7 +40,6 @@ import pekko.util.JavaDurationConverters._ import pekko.util.OptionConverters._ import pekko.util.Timeout import pekko.util.unused -import pekko.util.ccompat._ import org.reactivestreams.Processor object Flow { @@ -366,7 +365,6 @@ object Flow { /** * A `Flow` is a set of stream processing steps that has one open input and one open output. */ -@ccompatUsedUntil213 final class Flow[In, Out, Mat](delegate: scaladsl.Flow[In, Out, Mat]) extends Graph[FlowShape[In, Out], Mat] { import org.apache.pekko.util.ccompat.JavaConverters._ diff --git a/stream/src/main/scala/org/apache/pekko/stream/javadsl/Source.scala b/stream/src/main/scala/org/apache/pekko/stream/javadsl/Source.scala index 96e5f852c7..1f14162908 100755 --- a/stream/src/main/scala/org/apache/pekko/stream/javadsl/Source.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/javadsl/Source.scala @@ -38,7 +38,6 @@ import pekko.util.FutureConverters._ import pekko.util.JavaDurationConverters._ import pekko.util.OptionConverters._ import pekko.util.ccompat.JavaConverters._ -import pekko.util.ccompat._ import org.reactivestreams.{ Publisher, Subscriber } @@ -828,7 +827,6 @@ object Source { * A `Source` is a set of stream processing steps that has one open output and an attached input. * Can be used as a `Publisher` */ -@ccompatUsedUntil213 final class Source[Out, Mat](delegate: scaladsl.Source[Out, Mat]) extends Graph[SourceShape[Out], Mat] { import org.apache.pekko.util.ccompat.JavaConverters._ diff --git a/stream/src/main/scala/org/apache/pekko/stream/javadsl/StatefulMapConcatAccumulator.scala b/stream/src/main/scala/org/apache/pekko/stream/javadsl/StatefulMapConcatAccumulator.scala index fde249b338..72d1e94fad 100644 --- a/stream/src/main/scala/org/apache/pekko/stream/javadsl/StatefulMapConcatAccumulator.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/javadsl/StatefulMapConcatAccumulator.scala @@ -19,7 +19,6 @@ package org.apache.pekko.stream.javadsl import org.apache.pekko import pekko.japi.function -import pekko.util.ccompat.ccompatUsedUntil213 import java.util.Collections import scala.annotation.unchecked.uncheckedVariance @@ -29,7 +28,6 @@ import scala.annotation.unchecked.uncheckedVariance * * @since 1.2.0 */ -@ccompatUsedUntil213 @FunctionalInterface trait StatefulMapConcatAccumulator[-In, Out] extends function.Function[In, java.lang.Iterable[Out]] { diff --git a/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubFlow.scala b/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubFlow.scala index 947432a6f1..f55a58caa3 100755 --- a/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubFlow.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubFlow.scala @@ -32,7 +32,6 @@ import pekko.util.FutureConverters._ import pekko.util.JavaDurationConverters._ import pekko.util.OptionConverters._ import pekko.util.ccompat.JavaConverters._ -import pekko.util.ccompat._ object SubFlow { @@ -52,7 +51,6 @@ object SubFlow { * SubFlows cannot contribute to the super-flow’s materialized value since they * are materialized later, during the runtime of the flow graph processing. */ -@ccompatUsedUntil213 final class SubFlow[In, Out, Mat]( delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Flow[In, Out, Mat]#Repr, scaladsl.Sink[In, Mat]]) { diff --git a/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubSource.scala b/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubSource.scala index 387c4f1b19..d09c74fdf1 100755 --- a/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubSource.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/javadsl/SubSource.scala @@ -32,7 +32,6 @@ import pekko.util.FutureConverters._ import pekko.util.JavaDurationConverters._ import pekko.util.OptionConverters._ import pekko.util.ccompat.JavaConverters._ -import pekko.util.ccompat._ /** * * Upcast a stream of elements to a stream of supertypes of that element. Useful in combination with @@ -47,7 +46,6 @@ object SubSource { * SubFlows cannot contribute to the super-flow’s materialized value since they * are materialized later, during the runtime of the flow graph processing. */ -@ccompatUsedUntil213 final class SubSource[Out, Mat]( delegate: scaladsl.SubFlow[Out, Mat, scaladsl.Source[Out, Mat]#Repr, scaladsl.RunnableGraph[Mat]]) { diff --git a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/Flow.scala b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/Flow.scala index b77be6cc03..708efdbb5a 100755 --- a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/Flow.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/Flow.scala @@ -47,7 +47,6 @@ import pekko.stream.stage._ import pekko.util.ConstantFun import pekko.util.OptionVal import pekko.util.Timeout -import pekko.util.ccompat._ import org.reactivestreams.Processor import org.reactivestreams.Publisher @@ -844,7 +843,6 @@ final case class RunnableGraph[+Mat](override val traversalBuilder: TraversalBui * Binary compatibility is only maintained for callers of this trait’s interface. */ @DoNotInherit -@ccompatUsedUntil213 trait FlowOps[+Out, +Mat] { import GraphDSL.Implicits._ diff --git a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/FlowWithContextOps.scala b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/FlowWithContextOps.scala index 88848d2fca..54e2dbfaf2 100644 --- a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/FlowWithContextOps.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/FlowWithContextOps.scala @@ -26,13 +26,11 @@ import pekko.event.{ LogMarker, LoggingAdapter, MarkerLoggingAdapter } import pekko.stream._ import pekko.stream.impl.Throttle import pekko.util.ConstantFun -import pekko.util.ccompat._ /** * Shared stream operations for [[FlowWithContext]] and [[SourceWithContext]] that automatically propagate a context * element with each data element. */ -@ccompatUsedUntil213 trait FlowWithContextOps[+Out, +Ctx, +Mat] { type ReprMat[+O, +C, +M] <: FlowWithContextOps[O, C, M] { type ReprMat[+OO, +CC, +MatMat] = FlowWithContextOps.this.ReprMat[OO, CC, MatMat] diff --git a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/StatefulMapConcatAccumulator.scala b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/StatefulMapConcatAccumulator.scala index df469b9362..da71840e7e 100644 --- a/stream/src/main/scala/org/apache/pekko/stream/scaladsl/StatefulMapConcatAccumulator.scala +++ b/stream/src/main/scala/org/apache/pekko/stream/scaladsl/StatefulMapConcatAccumulator.scala @@ -20,14 +20,12 @@ package org.apache.pekko.stream.scaladsl import org.apache.pekko import pekko.japi.function import pekko.stream.{ javadsl, scaladsl } -import pekko.util.ccompat._ /** * A special accumulator for `StatefulMapConcat` operator that allows to emit elements when the upstream has completed. * * @since 1.2.0 */ -@ccompatUsedUntil213 @FunctionalInterface trait StatefulMapConcatAccumulator[-In, +Out] extends (In => IterableOnce[Out]) { diff --git a/testkit/src/main/scala/org/apache/pekko/testkit/TestEventListener.scala b/testkit/src/main/scala/org/apache/pekko/testkit/TestEventListener.scala index b19a79b0d6..f275bc540a 100644 --- a/testkit/src/main/scala/org/apache/pekko/testkit/TestEventListener.scala +++ b/testkit/src/main/scala/org/apache/pekko/testkit/TestEventListener.scala @@ -29,7 +29,6 @@ import pekko.event.Logging import pekko.event.Logging.{ Debug, Error, Info, InitializeLogger, LogEvent, LoggerInitialized, Warning } import pekko.japi.Util.immutableSeq import pekko.util.BoxedType -import pekko.util.ccompat._ /** * Implementation helpers of the EventFilter facilities: send `Mute` @@ -553,7 +552,6 @@ final case class DeadLettersFilter(val messageClass: Class[_])(occurrences: Int) * } * */ -@ccompatUsedUntil213 class TestEventListener extends Logging.DefaultLogger { import TestEvent._ diff --git a/testkit/src/main/scala/org/apache/pekko/testkit/package.scala b/testkit/src/main/scala/org/apache/pekko/testkit/package.scala index 609cff6467..c2c724f58b 100644 --- a/testkit/src/main/scala/org/apache/pekko/testkit/package.scala +++ b/testkit/src/main/scala/org/apache/pekko/testkit/package.scala @@ -21,10 +21,8 @@ import scala.reflect.ClassTag import org.apache.pekko import pekko.actor.ActorSystem -import pekko.util.ccompat._ package object testkit { - @ccompatUsedUntil213 def filterEvents[T](eventFilters: Iterable[EventFilter])(block: => T)(implicit system: ActorSystem): T = { def now = System.currentTimeMillis