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 =>
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 headMessage = head.message.asInstanceOf[ChunkedMessage]
// 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 serializerId = ser.identifier
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 {
val builder = Vector.newBuilder[ChunkedMessage]
val chunksIter = ByteString(bytes).grouped(chunkSize)
val chunksIter = ByteString.fromArrayUnsafe(bytes).grouped(chunkSize)
var first = true
while (chunksIter.hasNext) {
val chunk = chunksIter.next()

View file

@ -154,7 +154,7 @@ object ByteString {
}
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)
val SerializationIdentity = 1.toByte
@ -264,7 +264,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion {
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], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty
@ -803,8 +803,8 @@ sealed abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimiz
* 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 and which can lead to better performance,
* however it also means that one MUST NOT modify the returned in array, or unexpected
* 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

View file

@ -158,7 +158,7 @@ object ByteString {
private[akka] object ByteString1C extends Companion {
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)
val SerializationIdentity = 1.toByte
@ -276,7 +276,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion {
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], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty
@ -857,8 +857,8 @@ sealed abstract class ByteString
* 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 and which can lead to better performance,
* however it also means that one MUST NOT modify the returned in array, or unexpected
* 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

View file

@ -160,7 +160,7 @@ object ByteString {
private[akka] object ByteString1C extends Companion {
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)
val SerializationIdentity = 1.toByte
@ -277,7 +277,7 @@ object ByteString {
/** INTERNAL API: ByteString backed by exactly one array, with start / end markers */
private[akka] object ByteString1 extends Companion {
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], startIndex: Int, length: Int): ByteString1 =
if (length == 0) empty
@ -857,8 +857,8 @@ sealed abstract class ByteString
* 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 and which can lead to better performance,
* however it also means that one MUST NOT modify the returned in array, or unexpected
* 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

View file

@ -23,7 +23,9 @@ import akka.event.Logging
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]]

View file

@ -48,7 +48,9 @@ trait Serializer {
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]
@ -130,7 +132,9 @@ abstract class SerializerWithStringManifest extends Serializer {
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]

View file

@ -8,15 +8,14 @@ import java.{ lang => jl }
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream, ObjectOutputStream }
import java.io.NotSerializableException
import java.util.zip.{ GZIPInputStream, GZIPOutputStream }
import scala.annotation.tailrec
import scala.collection.immutable
import akka.actor.{ Address, ExtendedActorSystem }
import akka.cluster.metrics._
import akka.cluster.metrics.protobuf.msg.{ ClusterMetricsMessages => cm }
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.util.ClassLoaderObjectInputStream
import akka.util.ccompat._
@ -122,7 +121,9 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
val builder = cm.MetricsSelector.newBuilder()
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)
builder.setManifest(manifest)
@ -198,7 +199,10 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
val out = new ObjectOutputStream(bos)
out.writeObject(number)
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.util.zip.GZIPInputStream
import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec
import scala.collection.immutable.TreeMap
import akka.actor.{ Address, ExtendedActorSystem }
import akka.actor.ActorRef
import akka.cluster.pubsub.DistributedPubSubMediator._
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
import akka.cluster.pubsub.protobuf.msg.{ DistributedPubSubMessages => dm }
import akka.protobufv3.internal.{ ByteString, MessageLite }
import akka.remote.ByteStringUtils
import akka.serialization._
import akka.util.ccompat._
import akka.util.ccompat.JavaConverters._
@ -230,7 +229,7 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
val msgSerializer = serialization.findSerializerFor(m)
val builder = dm.Payload
.newBuilder()
.setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(m)))
.setEnclosedMessage(ByteStringUtils.toProtoByteStringUnsafe(msgSerializer.toBinary(m)))
.setSerializerId(msgSerializer.identifier)
val ms = Serializers.manifestFor(msgSerializer, m)

View file

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

View file

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

View file

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

View file

@ -7,13 +7,11 @@ package akka.cluster.ddata.protobuf
import java.io.NotSerializableException
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration._
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
import akka.actor.Address
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
@ -28,7 +26,7 @@ import akka.cluster.ddata.Replicator._
import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.VersionVector
import akka.cluster.ddata.protobuf.msg.{ ReplicatorMessages => dm }
import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
import akka.serialization.BaseSerializer
import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest
@ -267,7 +265,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
b.setChunk(status.chunk).setTotChunks(status.totChunks)
status.digests.foreach {
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
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 fromSystemUid = if (status.hasFromSystemUid) Some(status.getFromSystemUid) else None
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.getTotChunks,
toSystemUid,

View file

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

View file

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

View file

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

View file

@ -28,7 +28,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
val cfg = ConfigFactory.parseString(s"""
# 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.actor-selection = off
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
import scala.util.control.NonFatal
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.protobufv3.internal.ByteString
@ -52,7 +51,7 @@ private[akka] object MessageSerializer {
if (oldInfo eq null)
Serialization.currentTransportInformation.value = system.provider.serializationInformation
builder.setMessage(ByteString.copyFrom(serializer.toBinary(message)))
builder.setMessage(ByteStringUtils.toProtoByteStringUnsafe(serializer.toBinary(message)))
builder.setSerializerId(serializer.identifier)
val ms = Serializers.manifestFor(serializer, message)

View file

@ -41,18 +41,19 @@ import akka.util.ByteString
* The streamId` is encoded as 1 byte.
*/
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
* of the frame. The `frameLength` is encoded as 4 bytes (little endian).
*/
def encodeFrameHeader(frameLength: Int): ByteString =
ByteString(
(frameLength & 0xff).toByte,
((frameLength & 0xff00) >> 8).toByte,
((frameLength & 0xff0000) >> 16).toByte,
((frameLength & 0xff000000) >> 24).toByte)
ByteString.fromArrayUnsafe(
Array[Byte](
(frameLength & 0xff).toByte,
((frameLength & 0xff00) >> 8).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.reflect.ClassTag
import com.typesafe.config.{ Config, ConfigFactory }
import util.{ Failure, Success }
import util.{ Failure, Success }
import akka.actor.{ Deploy, ExtendedActorSystem, NoScopeGiven, Props, Scope }
import akka.protobufv3.internal.ByteString
import akka.remote.ByteStringUtils
import akka.remote.DaemonMsgCreate
import akka.remote.WireFormats.{ DaemonMsgCreateData, DeployData, PropsData }
import akka.routing.{ NoRouter, RouterConfig }
@ -46,21 +46,21 @@ private[akka] final class DaemonMsgCreateSerializer(val system: ExtendedActorSys
val (serId, _, manifest, bytes) = serialize(d.config)
builder.setConfigSerializerId(serId)
builder.setConfigManifest(manifest)
builder.setConfig(ByteString.copyFrom(bytes))
builder.setConfig(ByteStringUtils.toProtoByteStringUnsafe(bytes))
}
if (d.routerConfig != NoRouter) {
val (serId, _, manifest, bytes) = serialize(d.routerConfig)
builder.setRouterConfigSerializerId(serId)
builder.setRouterConfigManifest(manifest)
builder.setRouterConfig(ByteString.copyFrom(bytes))
builder.setRouterConfig(ByteStringUtils.toProtoByteStringUnsafe(bytes))
}
if (d.scope != NoScopeGiven) {
val (serId, _, manifest, bytes) = serialize(d.scope)
builder.setScopeSerializerId(serId)
builder.setScopeManifest(manifest)
builder.setScope(ByteString.copyFrom(bytes))
builder.setScope(ByteStringUtils.toProtoByteStringUnsafe(bytes))
}
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))
props.args.foreach { arg =>
val (serializerId, hasManifest, manifest, bytes) = serialize(arg)
builder.addArgs(ByteString.copyFrom(bytes))
builder.addArgs(ByteStringUtils.toProtoByteStringUnsafe(bytes))
builder.addManifests(manifest)
builder.addSerializerIds(serializerId)
builder.addHasManifest(hasManifest)

View file

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

View file

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

View file

@ -5,10 +5,8 @@
package akka.remote.transport
import scala.annotation.nowarn
import akka.AkkaException
import akka.actor.{ ActorRef, Address, AddressFromURIString, InternalActorRef }
import akka.protobufv3.internal.{ ByteString => PByteString }
import akka.protobufv3.internal.InvalidProtocolBufferException
import akka.remote._
import akka.remote.WireFormats._
@ -158,19 +156,15 @@ private[remote] object AkkaPduProtobufCodec extends AkkaPduCodec {
envelopeBuilder.setMessage(serializedMessage)
ackAndEnvelopeBuilder.setEnvelope(envelopeBuilder)
ByteString.ByteString1C(ackAndEnvelopeBuilder.build.toByteArray) //Reuse Byte Array (naughty!)
ByteString.fromArrayUnsafe(ackAndEnvelopeBuilder.build.toByteArray)
}
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 =
ByteString.ByteString1C(
AkkaProtocolMessage
.newBuilder()
.setPayload(PByteString.copyFrom(payload.asByteBuffer))
.build
.toByteArray) //Reuse Byte Array (naughty!)
ByteString.fromArrayUnsafe(
AkkaProtocolMessage.newBuilder().setPayload(ByteStringUtils.toProtoByteStringUnsafe(payload)).build.toByteArray)
override def constructAssociate(info: HandshakeInfo): ByteString = {
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 = {
try {
val pdu = AkkaProtocolMessage.parseFrom(raw.toArray)
if (pdu.hasPayload) Payload(ByteString(pdu.getPayload.asReadOnlyByteBuffer()))
val pdu = AkkaProtocolMessage.parseFrom(raw.toArrayUnsafe())
if (pdu.hasPayload) Payload(ByteString.fromByteBuffer(pdu.getPayload.asReadOnlyByteBuffer()))
else if (pdu.hasInstruction) decodeControlPdu(pdu.getInstruction)
else
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,
provider: RemoteActorRefProvider,
localAddress: Address): (Option[Ack], Option[Message]) = {
val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArray)
val ackAndEnvelope = AckAndEnvelopeContainer.parseFrom(raw.toArrayUnsafe())
val ackOption = if (ackAndEnvelope.hasAck) {
import akka.util.ccompat.JavaConverters._

View file

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

View file

@ -34,7 +34,7 @@ import akka.util.{ ByteString, ByteStringBuilder }
protected def compressWithBuffer(input: ByteString, buffer: Array[Byte]): ByteString = {
require(deflater.needsInput())
deflater.setInput(input.toArray)
deflater.setInput(input.toArrayUnsafe())
drainDeflater(deflater, buffer)
}
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] {
override def canWorkWithPartialData = true
override def parse(reader: ByteStringParser.ByteReader): ParseResult[ByteString] = {
inflater.setInput(reader.remainingData.toArray)
inflater.setInput(reader.remainingData.toArrayUnsafe())
val read = inflater.inflate(buffer)

View file

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

View file

@ -64,7 +64,7 @@ import akka.util.ByteString
}
private def crc16(data: ByteString) = {
val crc = new CRC32
crc.update(data.toArray)
crc.update(data.toArrayUnsafe())
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]`.
* 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.
* Must not mutate the given byte array.
*
*/
def lengthField(
@ -415,7 +416,7 @@ object Framing {
} else if (buffSize >= minimumChunkSize) {
val parsedLength = intDecoder(buffer.iterator.drop(lengthFieldOffset), lengthFieldLength)
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
}
if (frameSize > maximumFrameLength) {

View file

@ -7,10 +7,13 @@ package akka.stream.serialization
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.protobufv3.internal.ByteString
import akka.protobufv3.internal.UnsafeByteOperations
import akka.serialization._
import akka.stream.StreamRefMessages
import akka.stream.impl.streamref._
import java.nio.charset.StandardCharsets
/** INTERNAL API */
@InternalApi
private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
@ -86,7 +89,10 @@ private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
private def serializeRemoteSinkFailure(
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(
@ -108,7 +114,7 @@ private[akka] final class StreamRefSerializer(val system: ExtendedActorSystem)
val payloadBuilder = StreamRefMessages.Payload
.newBuilder()
.setEnclosedMessage(ByteString.copyFrom(msgSerializer.toBinary(p)))
.setEnclosedMessage(UnsafeByteOperations.unsafeWrap(msgSerializer.toBinary(p)))
.setSerializerId(msgSerializer.identifier)
val ms = Serializers.manifestFor(msgSerializer, p)