Byte array allocation/copy hunt (#30266)

* Loads of byte array copy/allocations squashed
* Use a central place for the proto unsafe instead of ad hoc
* Extra docs on serializer toBinary about not mutating bytes after returned
* Found some more toArray:s that can potentially benefit from being unsafe
* Array handed to user callback so extra docs needed
* Unsafe reading of underlying bytes of strings fixed
This commit is contained in:
Johan Andrén 2021-06-02 17:28:37 +02:00 committed by GitHub
parent b5cbf383d6
commit e43f2be6cd
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
31 changed files with 145 additions and 98 deletions

View file

@ -528,7 +528,7 @@ private class ConsumerControllerImpl[A] private (
reverseCollectedChunks.foreach { seqMsg => reverseCollectedChunks.foreach { seqMsg =>
builder ++= seqMsg.message.asInstanceOf[ChunkedMessage].serialized builder ++= seqMsg.message.asInstanceOf[ChunkedMessage].serialized
} }
val bytes = builder.result().toArray val bytes = builder.result().toArrayUnsafe()
val head = collectedChunks.head // this is the last chunk val head = collectedChunks.head // this is the last chunk
val headMessage = head.message.asInstanceOf[ChunkedMessage] val headMessage = head.message.asInstanceOf[ChunkedMessage]
// serialization exceptions are thrown, because it will anyway be stuck with same error if retried and // serialization exceptions are thrown, because it will anyway be stuck with same error if retried and

View file

@ -346,10 +346,10 @@ object ProducerControllerImpl {
val manifest = Serializers.manifestFor(ser, mAnyRef) val manifest = Serializers.manifestFor(ser, mAnyRef)
val serializerId = ser.identifier val serializerId = ser.identifier
if (bytes.length <= chunkSize) { if (bytes.length <= chunkSize) {
ChunkedMessage(ByteString(bytes), firstChunk = true, lastChunk = true, serializerId, manifest) :: Nil ChunkedMessage(ByteString.fromArrayUnsafe(bytes), firstChunk = true, lastChunk = true, serializerId, manifest) :: Nil
} else { } else {
val builder = Vector.newBuilder[ChunkedMessage] val builder = Vector.newBuilder[ChunkedMessage]
val chunksIter = ByteString(bytes).grouped(chunkSize) val chunksIter = ByteString.fromArrayUnsafe(bytes).grouped(chunkSize)
var first = true var first = true
while (chunksIter.hasNext) { while (chunksIter.hasNext) {
val chunk = chunksIter.next() val chunk = chunksIter.next()

View file

@ -154,7 +154,7 @@ object ByteString {
} }
private[akka] object ByteString1C extends Companion { private[akka] object ByteString1C extends Companion {
def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes) def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes(StandardCharsets.UTF_8))
def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes)
val SerializationIdentity = 1.toByte val SerializationIdentity = 1.toByte
@ -264,7 +264,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */ /** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion { private[akka] object ByteString1 extends Companion {
val empty: ByteString1 = new ByteString1(Array.empty[Byte]) val empty: ByteString1 = new ByteString1(Array.empty[Byte])
def fromString(s: String): ByteString1 = apply(s.getBytes) 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]): ByteString1 = apply(bytes, 0, bytes.length)
def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 = def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty if (length == 0) empty
@ -803,8 +803,8 @@ sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimiz
* into it before returning it. * into it before returning it.
* *
* This method of exposing the bytes of a ByteString can save one array * This method of exposing the bytes of a ByteString can save one array
* copy and allocation in the happy path scenario and which can lead to better performance, * 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 in array, or unexpected * however it also means that one MUST NOT modify the returned array, or unexpected
* immutable data structure contract-breaking behavior will manifest itself. * 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 * This API is intended for users who need to pass the byte array to some other API, which will

View file

@ -158,7 +158,7 @@ object ByteString {
private[akka] object ByteString1C extends Companion { private[akka] object ByteString1C extends Companion {
val empty = new ByteString1C(Array.emptyByteArray) val empty = new ByteString1C(Array.emptyByteArray)
def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes) def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes(StandardCharsets.UTF_8))
def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes)
val SerializationIdentity = 1.toByte val SerializationIdentity = 1.toByte
@ -276,7 +276,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */ /** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion { private[akka] object ByteString1 extends Companion {
val empty: ByteString1 = new ByteString1(Array.emptyByteArray, 0, 0) val empty: ByteString1 = new ByteString1(Array.emptyByteArray, 0, 0)
def fromString(s: String): ByteString1 = apply(s.getBytes) 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]): ByteString1 = apply(bytes, 0, bytes.length)
def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 = def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty if (length == 0) empty
@ -857,8 +857,8 @@ sealed abstract class ByteString
* into it before returning it. * into it before returning it.
* *
* This method of exposing the bytes of a ByteString can save one array * This method of exposing the bytes of a ByteString can save one array
* copy and allocation in the happy path scenario and which can lead to better performance, * 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 in array, or unexpected * however it also means that one MUST NOT modify the returned array, or unexpected
* immutable data structure contract-breaking behavior will manifest itself. * 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 * This API is intended for users who need to pass the byte array to some other API, which will

View file

@ -160,7 +160,7 @@ object ByteString {
private[akka] object ByteString1C extends Companion { private[akka] object ByteString1C extends Companion {
val empty = new ByteString1C(Array.emptyByteArray) val empty = new ByteString1C(Array.emptyByteArray)
def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes) def fromString(s: String): ByteString1C = new ByteString1C(s.getBytes(StandardCharsets.UTF_8))
def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes)
val SerializationIdentity = 1.toByte val SerializationIdentity = 1.toByte
@ -277,7 +277,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */ /** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion { private[akka] object ByteString1 extends Companion {
val empty: ByteString1 = new ByteString1(Array.emptyByteArray, 0, 0) val empty: ByteString1 = new ByteString1(Array.emptyByteArray, 0, 0)
def fromString(s: String): ByteString1 = apply(s.getBytes) 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]): ByteString1 = apply(bytes, 0, bytes.length)
def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 = def apply(bytes: Array[Byte], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty if (length == 0) empty
@ -857,8 +857,8 @@ sealed abstract class ByteString
* into it before returning it. * into it before returning it.
* *
* This method of exposing the bytes of a ByteString can save one array * This method of exposing the bytes of a ByteString can save one array
* copy and allocation in the happy path scenario and which can lead to better performance, * 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 in array, or unexpected * however it also means that one MUST NOT modify the returned array, or unexpected
* immutable data structure contract-breaking behavior will manifest itself. * 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 * This API is intended for users who need to pass the byte array to some other API, which will

View file

@ -23,7 +23,9 @@ import akka.event.Logging
trait AsyncSerializer { trait AsyncSerializer {
/** /**
* Serializes the given object into an Array of Byte * Serializes the given object into an Array of Byte.
*
* Note that the array must not be mutated by the serializer after it has been used to complete the returned `Future`.
*/ */
def toBinaryAsync(o: AnyRef): Future[Array[Byte]] def toBinaryAsync(o: AnyRef): Future[Array[Byte]]

View file

@ -48,7 +48,9 @@ trait Serializer {
def identifier: Int def identifier: Int
/** /**
* Serializes the given object into an Array of Byte * Serializes the given object into an Array of Byte.
*
* Note that the array must not be mutated by the serializer after it has been returned.
*/ */
def toBinary(o: AnyRef): Array[Byte] def toBinary(o: AnyRef): Array[Byte]
@ -130,7 +132,9 @@ abstract class SerializerWithStringManifest extends Serializer {
def manifest(o: AnyRef): String def manifest(o: AnyRef): String
/** /**
* Serializes the given object into an Array of Byte * Serializes the given object into an Array of Byte.
*
* Note that the array must not be mutated by the serializer after it has been returned.
*/ */
def toBinary(o: AnyRef): Array[Byte] def toBinary(o: AnyRef): Array[Byte]

View file

@ -8,15 +8,14 @@ import java.{ lang => jl }
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream } import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import java.io.NotSerializableException import java.io.NotSerializableException
import java.util.zip.{ GZIPInputStream, GZIPOutputStream } import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import akka.actor.{ Address, ExtendedActorSystem } import akka.actor.{ Address, ExtendedActorSystem }
import akka.cluster.metrics._ import akka.cluster.metrics._
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm } import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.protobufv3.internal.{ ByteString, MessageLite } import akka.protobufv3.internal.MessageLite
import akka.remote.ByteStringUtils
import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers } import akka.serialization.{ BaseSerializer, SerializationExtension, SerializerWithStringManifest, Serializers }
import akka.util.ClassLoaderObjectInputStream import akka.util.ClassLoaderObjectInputStream
import akka.util.ccompat._ import akka.util.ccompat._
@ -122,7 +121,9 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
val builder = cm.MetricsSelector.newBuilder() val builder = cm.MetricsSelector.newBuilder()
val serializer = serialization.findSerializerFor(selector) val serializer = serialization.findSerializerFor(selector)
builder.setData(ByteString.copyFrom(serializer.toBinary(selector))).setSerializerId(serializer.identifier) builder
.setData(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(selector)))
.setSerializerId(serializer.identifier)
val manifest = Serializers.manifestFor(serializer, selector) val manifest = Serializers.manifestFor(serializer, selector)
builder.setManifest(manifest) builder.setManifest(manifest)
@ -198,7 +199,10 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
val out = new ObjectOutputStream(bos) val out = new ObjectOutputStream(bos)
out.writeObject(number) out.writeObject(number)
out.close() out.close()
Number.newBuilder().setType(NumberType.Serialized).setSerialized(ByteString.copyFrom(bos.toByteArray)) Number
.newBuilder()
.setType(NumberType.Serialized)
.setSerialized(ByteStringUtils.toProtoByteStringUnsafe(bos.toByteArray))
} }
} }

View file

@ -8,16 +8,15 @@ import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
import java.io.NotSerializableException import java.io.NotSerializableException
import java.util.zip.GZIPInputStream import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
import akka.actor.{ Address, ExtendedActorSystem } import akka.actor.{ Address, ExtendedActorSystem }
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.cluster.pubsub.DistributedPubSubMediator._ import akka.cluster.pubsub.DistributedPubSubMediator._
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._ import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
import akka.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm } import akka.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm }
import akka.protobufv3.internal.{ ByteString, MessageLite } import akka.protobufv3.internal.{ ByteString, MessageLite }
import akka.remote.ByteStringUtils
import akka.serialization._ import akka.serialization._
import akka.util.ccompat._ import akka.util.ccompat._
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._
@ -230,7 +229,7 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
val msgSerializer = serialization.findSerializerFor(m) val msgSerializer = serialization.findSerializerFor(m)
val builder = dm.Payload val builder = dm.Payload
.newBuilder() .newBuilder()
.setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(m))) .setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(msgSerializer.toBinary(m)))
.setSerializerId(msgSerializer.identifier) .setSerializerId(msgSerializer.identifier)
val ms = Serializers.manifestFor(msgSerializer, m) val ms = Serializers.manifestFor(msgSerializer, m)

View file

@ -5,7 +5,6 @@
package akka.cluster.typed.internal.delivery package akka.cluster.typed.internal.delivery
import java.io.NotSerializableException import java.io.NotSerializableException
import akka.actor.typed.ActorRefResolver import akka.actor.typed.ActorRefResolver
import akka.actor.typed.delivery.ConsumerController import akka.actor.typed.delivery.ConsumerController
import akka.actor.typed.delivery.DurableProducerQueue import akka.actor.typed.delivery.DurableProducerQueue
@ -23,6 +22,7 @@ import akka.serialization.BaseSerializer
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
/** /**
* INTERNAL API * INTERNAL API
@ -97,7 +97,7 @@ import akka.protobufv3.internal.ByteString
private def chunkedMessageToProto(chunk: ChunkedMessage): Payload.Builder = { private def chunkedMessageToProto(chunk: ChunkedMessage): Payload.Builder = {
val payloadBuilder = ContainerFormats.Payload.newBuilder() val payloadBuilder = ContainerFormats.Payload.newBuilder()
payloadBuilder.setEnclosedMessage(ByteString.copyFrom(chunk.serialized.toArray)) payloadBuilder.setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(chunk.serialized.toArrayUnsafe()))
payloadBuilder.setMessageManifest(ByteString.copyFromUtf8(chunk.manifest)) payloadBuilder.setMessageManifest(ByteString.copyFromUtf8(chunk.manifest))
payloadBuilder.setSerializerId(chunk.serializerId) payloadBuilder.setSerializerId(chunk.serializerId)
payloadBuilder payloadBuilder
@ -207,7 +207,7 @@ import akka.protobufv3.internal.ByteString
val manifest = val manifest =
if (seqMsg.getMessage.hasMessageManifest) seqMsg.getMessage.getMessageManifest.toStringUtf8 else "" if (seqMsg.getMessage.hasMessageManifest) seqMsg.getMessage.getMessageManifest.toStringUtf8 else ""
ChunkedMessage( ChunkedMessage(
akka.util.ByteString(seqMsg.getMessage.getEnclosedMessage.toByteArray), akka.util.ByteString.fromArrayUnsafe(seqMsg.getMessage.getEnclosedMessage.toByteArray),
seqMsg.getFirstChunk, seqMsg.getFirstChunk,
seqMsg.getLastChunk, seqMsg.getLastChunk,
seqMsg.getMessage.getSerializerId, seqMsg.getMessage.getSerializerId,
@ -260,7 +260,7 @@ import akka.protobufv3.internal.ByteString
val manifest = val manifest =
if (sent.getMessage.hasMessageManifest) sent.getMessage.getMessageManifest.toStringUtf8 else "" if (sent.getMessage.hasMessageManifest) sent.getMessage.getMessageManifest.toStringUtf8 else ""
ChunkedMessage( ChunkedMessage(
akka.util.ByteString(sent.getMessage.getEnclosedMessage.toByteArray), akka.util.ByteString.fromArrayUnsafe(sent.getMessage.getEnclosedMessage.toByteArray),
sent.getFirstChunk, sent.getFirstChunk,
sent.getLastChunk, sent.getLastChunk,
sent.getMessage.getSerializerId, sent.getMessage.getSerializerId,

View file

@ -6,21 +6,19 @@ package akka.cluster.protobuf
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream } import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
import java.util.zip.{ GZIPInputStream, GZIPOutputStream } import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration.Deadline import scala.concurrent.duration.Deadline
import scala.annotation.nowarn import scala.annotation.nowarn
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions } import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
import akka.actor.{ Address, ExtendedActorSystem } import akka.actor.{ Address, ExtendedActorSystem }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.cluster._ import akka.cluster._
import akka.cluster.InternalClusterAction._ import akka.cluster.InternalClusterAction._
import akka.cluster.protobuf.msg.{ ClusterMessages => cm } import akka.cluster.protobuf.msg.{ ClusterMessages => cm }
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings } import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
import akka.protobufv3.internal.{ ByteString, MessageLite } import akka.protobufv3.internal.MessageLite
import akka.remote.ByteStringUtils
import akka.routing.Pool import akka.routing.Pool
import akka.serialization._ import akka.serialization._
import akka.util.Version import akka.util.Version
@ -228,7 +226,9 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
private def poolToProto(pool: Pool): cm.Pool = { private def poolToProto(pool: Pool): cm.Pool = {
val builder = cm.Pool.newBuilder() val builder = cm.Pool.newBuilder()
val serializer = serialization.findSerializerFor(pool) val serializer = serialization.findSerializerFor(pool)
builder.setSerializerId(serializer.identifier).setData(ByteString.copyFrom(serializer.toBinary(pool))) builder
.setSerializerId(serializer.identifier)
.setData(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(pool)))
val manifest = Serializers.manifestFor(serializer, pool) val manifest = Serializers.manifestFor(serializer, pool)
builder.setManifest(manifest) builder.setManifest(manifest)
builder.build() builder.build()
@ -519,7 +519,7 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
.newBuilder() .newBuilder()
.setFrom(uniqueAddressToProto(envelope.from)) .setFrom(uniqueAddressToProto(envelope.from))
.setTo(uniqueAddressToProto(envelope.to)) .setTo(uniqueAddressToProto(envelope.to))
.setSerializedGossip(ByteString.copyFrom(compress(gossipToProto(envelope.gossip).build))) .setSerializedGossip(ByteStringUtils.toProtoByteStringUnsafe(compress(gossipToProto(envelope.gossip).build)))
.build .build
private def gossipStatusToProto(status: GossipStatus): cm.GossipStatus = { private def gossipStatusToProto(status: GossipStatus): cm.GossipStatus = {
@ -530,7 +530,7 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem)
.setFrom(uniqueAddressToProto(status.from)) .setFrom(uniqueAddressToProto(status.from))
.addAllAllHashes(allHashes.asJava) .addAllAllHashes(allHashes.asJava)
.setVersion(vectorClockToProto(status.version, hashMapping)) .setVersion(vectorClockToProto(status.version, hashMapping))
.setSeenDigest(ByteString.copyFrom(status.seenDigest)) .setSeenDigest(ByteStringUtils.toProtoByteStringUnsafe(status.seenDigest))
.build() .build()
} }

View file

@ -10,12 +10,9 @@ import java.util
import java.util.ArrayList import java.util.ArrayList
import java.util.Collections import java.util.Collections
import java.util.Comparator import java.util.Comparator
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import scala.annotation.nowarn import scala.annotation.nowarn
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.cluster.ddata._ import akka.cluster.ddata._
@ -23,8 +20,8 @@ import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.protobuf.msg.{ ReplicatedDataMessages => rd } import akka.cluster.ddata.protobuf.msg.{ ReplicatedDataMessages => rd }
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm } import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.GeneratedMessageV3 import akka.protobufv3.internal.GeneratedMessageV3
import akka.remote.ByteStringUtils
import akka.serialization.BaseSerializer import akka.serialization.BaseSerializer
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
@ -624,7 +621,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
rd.GCounter.Entry rd.GCounter.Entry
.newBuilder() .newBuilder()
.setNode(uniqueAddressToProto(address)) .setNode(uniqueAddressToProto(address))
.setValue(ByteString.copyFrom(value.toByteArray))) .setValue(ByteStringUtils.toProtoByteStringUnsafe(value.toByteArray)))
} }
b.build() b.build()
} }

View file

@ -7,13 +7,11 @@ package akka.cluster.ddata.protobuf
import java.io.NotSerializableException import java.io.NotSerializableException
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import akka.actor.Address import akka.actor.Address
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi import akka.annotation.InternalApi
@ -28,7 +26,7 @@ import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._ import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.VersionVector import akka.cluster.ddata.VersionVector
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm } import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
import akka.protobufv3.internal.ByteString import akka.remote.ByteStringUtils
import akka.serialization.BaseSerializer import akka.serialization.BaseSerializer
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
@ -267,7 +265,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
b.setChunk(status.chunk).setTotChunks(status.totChunks) b.setChunk(status.chunk).setTotChunks(status.totChunks)
status.digests.foreach { status.digests.foreach {
case (key, digest) => case (key, digest) =>
b.addEntries(dm.Status.Entry.newBuilder().setKey(key).setDigest(ByteString.copyFrom(digest.toArray))) b.addEntries(
dm.Status.Entry
.newBuilder()
.setKey(key)
.setDigest(ByteStringUtils.toProtoByteStringUnsafe(digest.toArrayUnsafe())))
} }
status.toSystemUid.foreach(b.setToSystemUid) // can be None when sending back to a node of version 2.5.21 status.toSystemUid.foreach(b.setToSystemUid) // can be None when sending back to a node of version 2.5.21
b.setFromSystemUid(status.fromSystemUid.get) b.setFromSystemUid(status.fromSystemUid.get)
@ -279,7 +281,9 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
val toSystemUid = if (status.hasToSystemUid) Some(status.getToSystemUid) else None val toSystemUid = if (status.hasToSystemUid) Some(status.getToSystemUid) else None
val fromSystemUid = if (status.hasFromSystemUid) Some(status.getFromSystemUid) else None val fromSystemUid = if (status.hasFromSystemUid) Some(status.getFromSystemUid) else None
Status( Status(
status.getEntriesList.asScala.iterator.map(e => e.getKey -> AkkaByteString(e.getDigest.toByteArray())).toMap, status.getEntriesList.asScala.iterator
.map(e => e.getKey -> AkkaByteString.fromArrayUnsafe(e.getDigest.toByteArray()))
.toMap,
status.getChunk, status.getChunk,
status.getTotChunks, status.getTotChunks,
toSystemUid, toSystemUid,

View file

@ -8,10 +8,8 @@ import java.io.ByteArrayInputStream
import java.io.ByteArrayOutputStream import java.io.ByteArrayOutputStream
import java.util.zip.GZIPInputStream import java.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Address import akka.actor.Address
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
@ -20,6 +18,7 @@ import akka.cluster.ddata.VersionVector
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm } import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.MessageLite import akka.protobufv3.internal.MessageLite
import akka.remote.ByteStringUtils
import akka.serialization._ import akka.serialization._
import akka.util.ccompat._ import akka.util.ccompat._
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._
@ -143,7 +142,7 @@ trait SerializationSupport {
val msgSerializer = serialization.findSerializerFor(m) val msgSerializer = serialization.findSerializerFor(m)
val builder = dm.OtherMessage val builder = dm.OtherMessage
.newBuilder() .newBuilder()
.setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(m))) .setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(msgSerializer.toBinary(m)))
.setSerializerId(msgSerializer.identifier) .setSerializerId(msgSerializer.identifier)
val ms = Serializers.manifestFor(msgSerializer, m) val ms = Serializers.manifestFor(msgSerializer, m)

View file

@ -7,7 +7,6 @@ package akka.persistence.typed.serialization
import java.io.NotSerializableException import java.io.NotSerializableException
import java.util.{ ArrayList, Collections, Comparator } import java.util.{ ArrayList, Collections, Comparator }
import java.{ lang => jl } import java.{ lang => jl }
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.persistence.typed.PersistenceId import akka.persistence.typed.PersistenceId
@ -18,7 +17,7 @@ import akka.persistence.typed.internal.ReplicatedEventMetadata
import akka.persistence.typed.internal.ReplicatedSnapshotMetadata import akka.persistence.typed.internal.ReplicatedSnapshotMetadata
import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData import akka.persistence.typed.internal.ReplicatedPublishedEventMetaData
import akka.persistence.typed.internal.VersionVector import akka.persistence.typed.internal.VersionVector
import akka.protobufv3.internal.ByteString import akka.remote.ByteStringUtils
import akka.remote.ContainerFormats.Payload import akka.remote.ContainerFormats.Payload
import akka.remote.serialization.WrappedPayloadSupport import akka.remote.serialization.WrappedPayloadSupport
import akka.serialization.{ BaseSerializer, SerializerWithStringManifest } import akka.serialization.{ BaseSerializer, SerializerWithStringManifest }
@ -192,14 +191,14 @@ import scala.collection.immutable.TreeMap
def counterToProtoByteArray(counter: Counter): Array[Byte] = def counterToProtoByteArray(counter: Counter): Array[Byte] =
ReplicatedEventSourcing.Counter ReplicatedEventSourcing.Counter
.newBuilder() .newBuilder()
.setValue(ByteString.copyFrom(counter.value.toByteArray)) .setValue(ByteStringUtils.toProtoByteStringUnsafe(counter.value.toByteArray))
.build() .build()
.toByteArray .toByteArray
def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] = def counterUpdatedToProtoBufByteArray(updated: Counter.Updated): Array[Byte] =
ReplicatedEventSourcing.CounterUpdate ReplicatedEventSourcing.CounterUpdate
.newBuilder() .newBuilder()
.setDelta(ByteString.copyFrom(updated.delta.toByteArray)) .setDelta(ByteStringUtils.toProtoByteStringUnsafe(updated.delta.toByteArray))
.build() .build()
.toByteArray .toByteArray

View file

@ -5,14 +5,11 @@
package akka.persistence.serialization package akka.persistence.serialization
import java.io.NotSerializableException import java.io.NotSerializableException
import scala.collection.immutable import scala.collection.immutable
import scala.collection.immutable.VectorBuilder import scala.collection.immutable.VectorBuilder
import scala.concurrent.duration import scala.concurrent.duration
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.annotation.nowarn import scala.annotation.nowarn
import akka.actor.{ ActorPath, ExtendedActorSystem } import akka.actor.{ ActorPath, ExtendedActorSystem }
import akka.actor.Actor import akka.actor.Actor
import akka.persistence._ import akka.persistence._
@ -20,6 +17,7 @@ import akka.persistence.AtLeastOnceDelivery._
import akka.persistence.fsm.PersistentFSM.{ PersistentFSMSnapshot, StateChangeEvent } import akka.persistence.fsm.PersistentFSM.{ PersistentFSMSnapshot, StateChangeEvent }
import akka.persistence.serialization.{ MessageFormats => mf } import akka.persistence.serialization.{ MessageFormats => mf }
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.UnsafeByteOperations
import akka.serialization._ import akka.serialization._
import akka.util.ccompat._ import akka.util.ccompat._
@ -187,7 +185,7 @@ class MessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer
val ms = Serializers.manifestFor(serializer, payload) val ms = Serializers.manifestFor(serializer, payload)
if (ms.nonEmpty) builder.setPayloadManifest(ByteString.copyFromUtf8(ms)) if (ms.nonEmpty) builder.setPayloadManifest(ByteString.copyFromUtf8(ms))
builder.setPayload(ByteString.copyFrom(serializer.toBinary(payload))) builder.setPayload(UnsafeByteOperations.unsafeWrap(serializer.toBinary(payload)))
builder.setSerializerId(serializer.identifier) builder.setSerializerId(serializer.identifier)
builder builder
} }

View file

@ -28,7 +28,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
val cfg = ConfigFactory.parseString(s""" val cfg = ConfigFactory.parseString(s"""
# for serious measurements you should increase the totalMessagesFactor (80) # for serious measurements you should increase the totalMessagesFactor (80)
akka.test.MaxThroughputSpec.totalMessagesFactor = 10.0 akka.test.MaxThroughputSpec.totalMessagesFactor = 160.0
akka.test.MaxThroughputSpec.real-message = off akka.test.MaxThroughputSpec.real-message = off
akka.test.MaxThroughputSpec.actor-selection = off akka.test.MaxThroughputSpec.actor-selection = off
akka { akka {

View file

@ -0,0 +1,39 @@
/*
* Copyright (C) 2009-2021 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.remote
import akka.annotation.InternalApi
import akka.protobufv3.internal.UnsafeByteOperations
import akka.util.ByteString
import akka.protobufv3.internal.{ ByteString => ProtoByteString }
import akka.util.ByteString.ByteString1
import akka.util.ByteString.ByteString1C
/**
* INTERNAL API
*/
@InternalApi
private[akka] object ByteStringUtils {
def toProtoByteStringUnsafe(bytes: ByteString): ProtoByteString = {
if (bytes.isEmpty)
ProtoByteString.EMPTY
else if (bytes.isInstanceOf[ByteString1C] || (bytes.isInstanceOf[ByteString1] && bytes.isCompact)) {
UnsafeByteOperations.unsafeWrap(bytes.toArrayUnsafe())
} else {
// zero copy, reuse the same underlying byte arrays
bytes.asByteBuffers.foldLeft(ProtoByteString.EMPTY) { (acc, byteBuffer) =>
acc.concat(UnsafeByteOperations.unsafeWrap(byteBuffer))
}
}
}
def toProtoByteStringUnsafe(bytes: Array[Byte]): ProtoByteString = {
if (bytes.isEmpty)
ProtoByteString.EMPTY
else {
UnsafeByteOperations.unsafeWrap(bytes)
}
}
}

View file

@ -5,7 +5,6 @@
package akka.remote package akka.remote
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
@ -52,7 +51,7 @@ private[akka] object MessageSerializer {
if (oldInfo eq null) if (oldInfo eq null)
Serialization.currentTransportInformation.value = system.provider.serializationInformation Serialization.currentTransportInformation.value = system.provider.serializationInformation
builder.setMessage(ByteString.copyFrom(serializer.toBinary(message))) builder.setMessage(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(message)))
builder.setSerializerId(serializer.identifier) builder.setSerializerId(serializer.identifier)
val ms = Serializers.manifestFor(serializer, message) val ms = Serializers.manifestFor(serializer, message)

View file

@ -41,18 +41,19 @@ import akka.util.ByteString
* The streamId` is encoded as 1 byte. * The streamId` is encoded as 1 byte.
*/ */
def encodeConnectionHeader(streamId: Int): ByteString = def encodeConnectionHeader(streamId: Int): ByteString =
Magic ++ ByteString(streamId.toByte) Magic ++ ByteString.fromArrayUnsafe(Array(streamId.toByte))
/** /**
* Each frame starts with the frame header that contains the length * Each frame starts with the frame header that contains the length
* of the frame. The `frameLength` is encoded as 4 bytes (little endian). * of the frame. The `frameLength` is encoded as 4 bytes (little endian).
*/ */
def encodeFrameHeader(frameLength: Int): ByteString = def encodeFrameHeader(frameLength: Int): ByteString =
ByteString( ByteString.fromArrayUnsafe(
(frameLength & 0xff).toByte, Array[Byte](
((frameLength & 0xff00) >> 8).toByte, (frameLength & 0xff).toByte,
((frameLength & 0xff0000) >> 16).toByte, ((frameLength & 0xff00) >> 8).toByte,
((frameLength & 0xff000000) >> 24).toByte) ((frameLength & 0xff0000) >> 16).toByte,
((frameLength & 0xff000000) >> 24).toByte))
} }
/** /**

View file

@ -6,12 +6,12 @@ package akka.remote.serialization
import scala.collection.immutable import scala.collection.immutable
import scala.reflect.ClassTag import scala.reflect.ClassTag
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import util.{ Failure, Success }
import util.{ Failure, Success }
import akka.actor.{ Deploy, ExtendedActorSystem, NoScopeGiven, Props, Scope } import akka.actor.{ Deploy, ExtendedActorSystem, NoScopeGiven, Props, Scope }
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
import akka.remote.DaemonMsgCreate import akka.remote.DaemonMsgCreate
import akka.remote.WireFormats.{ DaemonMsgCreateData, DeployData, PropsData } import akka.remote.WireFormats.{ DaemonMsgCreateData, DeployData, PropsData }
import akka.routing.{ NoRouter, RouterConfig } import akka.routing.{ NoRouter, RouterConfig }
@ -46,21 +46,21 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys
val (serId, _, manifest, bytes) = serialize(d.config) val (serId, _, manifest, bytes) = serialize(d.config)
builder.setConfigSerializerId(serId) builder.setConfigSerializerId(serId)
builder.setConfigManifest(manifest) builder.setConfigManifest(manifest)
builder.setConfig(ByteString.copyFrom(bytes)) builder.setConfig(ByteStringUtils.toProtoByteStringUnsafe(bytes))
} }
if (d.routerConfig != NoRouter) { if (d.routerConfig != NoRouter) {
val (serId, _, manifest, bytes) = serialize(d.routerConfig) val (serId, _, manifest, bytes) = serialize(d.routerConfig)
builder.setRouterConfigSerializerId(serId) builder.setRouterConfigSerializerId(serId)
builder.setRouterConfigManifest(manifest) builder.setRouterConfigManifest(manifest)
builder.setRouterConfig(ByteString.copyFrom(bytes)) builder.setRouterConfig(ByteStringUtils.toProtoByteStringUnsafe(bytes))
} }
if (d.scope != NoScopeGiven) { if (d.scope != NoScopeGiven) {
val (serId, _, manifest, bytes) = serialize(d.scope) val (serId, _, manifest, bytes) = serialize(d.scope)
builder.setScopeSerializerId(serId) builder.setScopeSerializerId(serId)
builder.setScopeManifest(manifest) builder.setScopeManifest(manifest)
builder.setScope(ByteString.copyFrom(bytes)) builder.setScope(ByteStringUtils.toProtoByteStringUnsafe(bytes))
} }
if (d.dispatcher != NoDispatcherGiven) { if (d.dispatcher != NoDispatcherGiven) {
@ -76,7 +76,7 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys
val builder = PropsData.newBuilder.setClazz(props.clazz.getName).setDeploy(deployProto(props.deploy)) val builder = PropsData.newBuilder.setClazz(props.clazz.getName).setDeploy(deployProto(props.deploy))
props.args.foreach { arg => props.args.foreach { arg =>
val (serializerId, hasManifest, manifest, bytes) = serialize(arg) val (serializerId, hasManifest, manifest, bytes) = serialize(arg)
builder.addArgs(ByteString.copyFrom(bytes)) builder.addArgs(ByteStringUtils.toProtoByteStringUnsafe(bytes))
builder.addManifests(manifest) builder.addManifests(manifest)
builder.addSerializerIds(serializerId) builder.addSerializerIds(serializerId)
builder.addHasManifest(hasManifest) builder.addHasManifest(hasManifest)

View file

@ -5,7 +5,6 @@
package akka.remote.serialization package akka.remote.serialization
import scala.collection.immutable import scala.collection.immutable
import akka.actor.ActorSelectionMessage import akka.actor.ActorSelectionMessage
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.SelectChildName import akka.actor.SelectChildName
@ -13,6 +12,7 @@ import akka.actor.SelectChildPattern
import akka.actor.SelectParent import akka.actor.SelectParent
import akka.actor.SelectionPathElement import akka.actor.SelectionPathElement
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
import akka.remote.ContainerFormats import akka.remote.ContainerFormats
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers } import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers }
import akka.util.ccompat._ import akka.util.ccompat._
@ -36,7 +36,7 @@ class MessageContainerSerializer(val system: ExtendedActorSystem) extends BaseSe
val message = sel.msg.asInstanceOf[AnyRef] val message = sel.msg.asInstanceOf[AnyRef]
val serializer = serialization.findSerializerFor(message) val serializer = serialization.findSerializerFor(message)
builder builder
.setEnclosedMessage(ByteString.copyFrom(serializer.toBinary(message))) .setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(message)))
.setSerializerId(serializer.identifier) .setSerializerId(serializer.identifier)
.setWildcardFanOut(sel.wildcardFanOut) .setWildcardFanOut(sel.wildcardFanOut)

View file

@ -7,6 +7,7 @@ package akka.remote.serialization
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.event.Logging import akka.event.Logging
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
import akka.remote.ContainerFormats import akka.remote.ContainerFormats
import akka.serialization.{ SerializationExtension, Serializers } import akka.serialization.{ SerializationExtension, Serializers }
import akka.serialization.DisabledJavaSerializer import akka.serialization.DisabledJavaSerializer
@ -41,14 +42,14 @@ private[akka] class WrappedPayloadSupport(system: ExtendedActorSystem) {
notSerializableException.originalMessage) notSerializableException.originalMessage)
val serializer2 = serialization.findSerializerFor(notSerializableException) val serializer2 = serialization.findSerializerFor(notSerializableException)
builder builder
.setEnclosedMessage(ByteString.copyFrom(serializer2.toBinary(notSerializableException))) .setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(serializer2.toBinary(notSerializableException)))
.setSerializerId(serializer2.identifier) .setSerializerId(serializer2.identifier)
val manifest = Serializers.manifestFor(serializer2, notSerializableException) val manifest = Serializers.manifestFor(serializer2, notSerializableException)
if (manifest.nonEmpty) builder.setMessageManifest(ByteString.copyFromUtf8(manifest)) if (manifest.nonEmpty) builder.setMessageManifest(ByteString.copyFromUtf8(manifest))
case _ => case _ =>
builder builder
.setEnclosedMessage(ByteString.copyFrom(serializer.toBinary(payload))) .setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(payload)))
.setSerializerId(serializer.identifier) .setSerializerId(serializer.identifier)
val manifest = Serializers.manifestFor(serializer, payload) val manifest = Serializers.manifestFor(serializer, payload)
if (manifest.nonEmpty) builder.setMessageManifest(ByteString.copyFromUtf8(manifest)) if (manifest.nonEmpty) builder.setMessageManifest(ByteString.copyFromUtf8(manifest))

View file

@ -5,10 +5,8 @@
package akka.remote.transport package akka.remote.transport
import scala.annotation.nowarn import scala.annotation.nowarn
import akka.AkkaException import akka.AkkaException
import akka.actor.{ ActorRef, Address, AddressFromURIString, InternalActorRef } import akka.actor.{ ActorRef, Address, AddressFromURIString, InternalActorRef }
import akka.protobufv3.internal.{ ByteString => PByteString }
import akka.protobufv3.internal.InvalidProtocolBufferException import akka.protobufv3.internal.InvalidProtocolBufferException
import akka.remote._ import akka.remote._
import akka.remote.WireFormats._ import akka.remote.WireFormats._
@ -158,19 +156,15 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
envelopeBuilder.setMessage(serializedMessage) envelopeBuilder.setMessage(serializedMessage)
ackAndEnvelopeBuilder.setEnvelope(envelopeBuilder) ackAndEnvelopeBuilder.setEnvelope(envelopeBuilder)
ByteString.ByteString1C(ackAndEnvelopeBuilder.build.toByteArray) //Reuse Byte Array (naughty!) ByteString.fromArrayUnsafe(ackAndEnvelopeBuilder.build.toByteArray)
} }
override def constructPureAck(ack: Ack): ByteString = override def constructPureAck(ack: Ack): ByteString =
ByteString.ByteString1C(AckAndEnvelopeContainer.newBuilder.setAck(ackBuilder(ack)).build().toByteArray) //Reuse Byte Array (naughty!) ByteString.fromArrayUnsafe(AckAndEnvelopeContainer.newBuilder.setAck(ackBuilder(ack)).build().toByteArray)
override def constructPayload(payload: ByteString): ByteString = override def constructPayload(payload: ByteString): ByteString =
ByteString.ByteString1C( ByteString.fromArrayUnsafe(
AkkaProtocolMessage AkkaProtocolMessage.newBuilder().setPayload(ByteStringUtils.toProtoByteStringUnsafe(payload)).build.toByteArray)
.newBuilder()
.setPayload(PByteString.copyFrom(payload.asByteBuffer))
.build
.toByteArray) //Reuse Byte Array (naughty!)
override def constructAssociate(info: HandshakeInfo): ByteString = { override def constructAssociate(info: HandshakeInfo): ByteString = {
val handshakeInfo = AkkaHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid.toLong) val handshakeInfo = AkkaHandshakeInfo.newBuilder.setOrigin(serializeAddress(info.origin)).setUid(info.uid.toLong)
@ -194,8 +188,8 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
override def decodePdu(raw: ByteString): AkkaPdu = { override def decodePdu(raw: ByteString): AkkaPdu = {
try { try {
val pdu = AkkaProtocolMessage.parseFrom(raw.toArray) val pdu = AkkaProtocolMessage.parseFrom(raw.toArrayUnsafe())
if (pdu.hasPayload) Payload(ByteString(pdu.getPayload.asReadOnlyByteBuffer())) if (pdu.hasPayload) Payload(ByteString.fromByteBuffer(pdu.getPayload.asReadOnlyByteBuffer()))
else if (pdu.hasInstruction) decodeControlPdu(pdu.getInstruction) else if (pdu.hasInstruction) decodeControlPdu(pdu.getInstruction)
else else
throw new PduCodecException("Error decoding Akka PDU: Neither message nor control message were contained", null) throw new PduCodecException("Error decoding Akka PDU: Neither message nor control message were contained", null)
@ -208,7 +202,7 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
raw: ByteString, raw: ByteString,
provider: RemoteActorRefProvider, provider: RemoteActorRefProvider,
localAddress: Address): (Option[Ack], Option[Message]) = { localAddress: Address): (Option[Ack], Option[Message]) = {
val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArray) val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArrayUnsafe())
val ackOption = if (ackAndEnvelope.hasAck) { val ackOption = if (ackAndEnvelope.hasAck) {
import akka.util.ccompat.JavaConverters._ import akka.util.ccompat.JavaConverters._

View file

@ -50,7 +50,7 @@ private[akka] final class OutputStreamGraphStage(factory: () => OutputStream, au
override def onPush(): Unit = { override def onPush(): Unit = {
val next = grab(in) val next = grab(in)
try { try {
outputStream.write(next.toArray) outputStream.write(next.toArrayUnsafe())
if (autoFlush) outputStream.flush() if (autoFlush) outputStream.flush()
bytesWritten += next.size bytesWritten += next.size

View file

@ -34,7 +34,7 @@ import akka.util.{ ByteString, ByteStringBuilder }
protected def compressWithBuffer(input: ByteString, buffer: Array[Byte]): ByteString = { protected def compressWithBuffer(input: ByteString, buffer: Array[Byte]): ByteString = {
require(deflater.needsInput()) require(deflater.needsInput())
deflater.setInput(input.toArray) deflater.setInput(input.toArrayUnsafe())
drainDeflater(deflater, buffer) drainDeflater(deflater, buffer)
} }
protected def flushWithBuffer(buffer: Array[Byte]): ByteString = { protected def flushWithBuffer(buffer: Array[Byte]): ByteString = {

View file

@ -32,7 +32,7 @@ import akka.util.ByteString
abstract class Inflate(noPostProcessing: Boolean) extends ParseStep[ByteString] { abstract class Inflate(noPostProcessing: Boolean) extends ParseStep[ByteString] {
override def canWorkWithPartialData = true override def canWorkWithPartialData = true
override def parse(reader: ByteStringParser.ByteReader): ParseResult[ByteString] = { override def parse(reader: ByteStringParser.ByteReader): ParseResult[ByteString] = {
inflater.setInput(reader.remainingData.toArray) inflater.setInput(reader.remainingData.toArrayUnsafe())
val read = inflater.inflate(buffer) val read = inflater.inflate(buffer)

View file

@ -26,7 +26,7 @@ import akka.util.ByteString
header() ++ super.finishWithBuffer(buffer) ++ trailer() header() ++ super.finishWithBuffer(buffer) ++ trailer()
private def updateCrc(input: ByteString): Unit = { private def updateCrc(input: ByteString): Unit = {
checkSum.update(input.toArray) checkSum.update(input.toArrayUnsafe())
bytesRead += input.length bytesRead += input.length
} }
private def header(): ByteString = private def header(): ByteString =

View file

@ -64,7 +64,7 @@ import akka.util.ByteString
} }
private def crc16(data: ByteString) = { private def crc16(data: ByteString) = {
val crc = new CRC32 val crc = new CRC32
crc.update(data.toArray) crc.update(data.toArrayUnsafe())
crc.getValue.toInt & 0xFFFF crc.getValue.toInt & 0xFFFF
} }
} }

View file

@ -83,6 +83,7 @@ object Framing {
* For example, frame can have a shape like this: `[offset bytes][body size bytes][body bytes][footer bytes]`. * For example, frame can have a shape like this: `[offset bytes][body size bytes][body bytes][footer bytes]`.
* Then computeFrameSize can be used to compute the frame size: `(offset bytes, computed size) => (actual frame size)`. * Then computeFrameSize can be used to compute the frame size: `(offset bytes, computed size) => (actual frame size)`.
* ''Actual frame size'' must be equal or bigger than sum of `fieldOffset` and `fieldLength`, the operator fails otherwise. * ''Actual frame size'' must be equal or bigger than sum of `fieldOffset` and `fieldLength`, the operator fails otherwise.
* Must not mutate the given byte array.
* *
*/ */
def lengthField( def lengthField(
@ -415,7 +416,7 @@ object Framing {
} else if (buffSize >= minimumChunkSize) { } else if (buffSize >= minimumChunkSize) {
val parsedLength = intDecoder(buffer.iterator.drop(lengthFieldOffset), lengthFieldLength) val parsedLength = intDecoder(buffer.iterator.drop(lengthFieldOffset), lengthFieldLength)
frameSize = computeFrameSize match { frameSize = computeFrameSize match {
case Some(f) => f(buffer.take(lengthFieldOffset).toArray, parsedLength) case Some(f) => f(buffer.take(lengthFieldOffset).toArrayUnsafe(), parsedLength)
case None => parsedLength + minimumChunkSize case None => parsedLength + minimumChunkSize
} }
if (frameSize > maximumFrameLength) { if (frameSize > maximumFrameLength) {

View file

@ -7,10 +7,13 @@ package akka.stream.serialization
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.protobufv3.internal.ByteString import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.UnsafeByteOperations
import akka.serialization._ import akka.serialization._
import akka.stream.StreamRefMessages import akka.stream.StreamRefMessages
import akka.stream.impl.streamref._ import akka.stream.impl.streamref._
import java.nio.charset.StandardCharsets
/** INTERNAL API */ /** INTERNAL API */
@InternalApi @InternalApi
private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem) private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
@ -86,7 +89,10 @@ private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
private def serializeRemoteSinkFailure( private def serializeRemoteSinkFailure(
d: StreamRefsProtocol.RemoteStreamFailure): StreamRefMessages.RemoteStreamFailure = { d: StreamRefsProtocol.RemoteStreamFailure): StreamRefMessages.RemoteStreamFailure = {
StreamRefMessages.RemoteStreamFailure.newBuilder().setCause(ByteString.copyFrom(d.msg.getBytes)).build() StreamRefMessages.RemoteStreamFailure
.newBuilder()
.setCause(UnsafeByteOperations.unsafeWrap(d.msg.getBytes(StandardCharsets.UTF_8)))
.build()
} }
private def serializeRemoteSinkCompleted( private def serializeRemoteSinkCompleted(
@ -108,7 +114,7 @@ private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
val payloadBuilder = StreamRefMessages.Payload val payloadBuilder = StreamRefMessages.Payload
.newBuilder() .newBuilder()
.setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(p))) .setEnclosedMessage(UnsafeByteOperations.unsafeWrap(msgSerializer.toBinary(p)))
.setSerializerId(msgSerializer.identifier) .setSerializerId(msgSerializer.identifier)
val ms = Serializers.manifestFor(msgSerializer, p) val ms = Serializers.manifestFor(msgSerializer, p)