Avoid Array.ofDim where possible #22516

This commit is contained in:
Johan Andrén 2017-03-13 17:49:45 +01:00 committed by GitHub
parent 5ea2e0536c
commit 7a0e5b31f8
33 changed files with 74 additions and 75 deletions

View file

@ -151,10 +151,10 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 2 val elemSize = 2
val (bytes, from, until) = slice val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize) val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Short](n) val reference = new Array[Short](n)
bytes.asByteBuffer.order(byteOrder).asShortBuffer.get(reference, 0, n) bytes.asByteBuffer.order(byteOrder).asShortBuffer.get(reference, 0, n)
val input = bytes.iterator val input = bytes.iterator
val decoded = Array.ofDim[Short](n) val decoded = new Array[Short](n)
for (i 0 until a) decoded(i) = input.getShort(byteOrder) for (i 0 until a) decoded(i) = input.getShort(byteOrder)
input.getShorts(decoded, a, b - a)(byteOrder) input.getShorts(decoded, a, b - a)(byteOrder)
for (i b until n) decoded(i) = input.getShort(byteOrder) for (i b until n) decoded(i) = input.getShort(byteOrder)
@ -165,10 +165,10 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 4 val elemSize = 4
val (bytes, from, until) = slice val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize) val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Int](n) val reference = new Array[Int](n)
bytes.asByteBuffer.order(byteOrder).asIntBuffer.get(reference, 0, n) bytes.asByteBuffer.order(byteOrder).asIntBuffer.get(reference, 0, n)
val input = bytes.iterator val input = bytes.iterator
val decoded = Array.ofDim[Int](n) val decoded = new Array[Int](n)
for (i 0 until a) decoded(i) = input.getInt(byteOrder) for (i 0 until a) decoded(i) = input.getInt(byteOrder)
input.getInts(decoded, a, b - a)(byteOrder) input.getInts(decoded, a, b - a)(byteOrder)
for (i b until n) decoded(i) = input.getInt(byteOrder) for (i b until n) decoded(i) = input.getInt(byteOrder)
@ -179,10 +179,10 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 8 val elemSize = 8
val (bytes, from, until) = slice val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize) val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Long](n) val reference = new Array[Long](n)
bytes.asByteBuffer.order(byteOrder).asLongBuffer.get(reference, 0, n) bytes.asByteBuffer.order(byteOrder).asLongBuffer.get(reference, 0, n)
val input = bytes.iterator val input = bytes.iterator
val decoded = Array.ofDim[Long](n) val decoded = new Array[Long](n)
for (i 0 until a) decoded(i) = input.getLong(byteOrder) for (i 0 until a) decoded(i) = input.getLong(byteOrder)
input.getLongs(decoded, a, b - a)(byteOrder) input.getLongs(decoded, a, b - a)(byteOrder)
for (i b until n) decoded(i) = input.getLong(byteOrder) for (i b until n) decoded(i) = input.getLong(byteOrder)
@ -193,10 +193,10 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 4 val elemSize = 4
val (bytes, from, until) = slice val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize) val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Float](n) val reference = new Array[Float](n)
bytes.asByteBuffer.order(byteOrder).asFloatBuffer.get(reference, 0, n) bytes.asByteBuffer.order(byteOrder).asFloatBuffer.get(reference, 0, n)
val input = bytes.iterator val input = bytes.iterator
val decoded = Array.ofDim[Float](n) val decoded = new Array[Float](n)
for (i 0 until a) decoded(i) = input.getFloat(byteOrder) for (i 0 until a) decoded(i) = input.getFloat(byteOrder)
input.getFloats(decoded, a, b - a)(byteOrder) input.getFloats(decoded, a, b - a)(byteOrder)
for (i b until n) decoded(i) = input.getFloat(byteOrder) for (i b until n) decoded(i) = input.getFloat(byteOrder)
@ -208,10 +208,10 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 8 val elemSize = 8
val (bytes, from, until) = slice val (bytes, from, until) = slice
val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize) val (n, a, b) = (bytes.length / elemSize, from / elemSize, until / elemSize)
val reference = Array.ofDim[Double](n) val reference = new Array[Double](n)
bytes.asByteBuffer.order(byteOrder).asDoubleBuffer.get(reference, 0, n) bytes.asByteBuffer.order(byteOrder).asDoubleBuffer.get(reference, 0, n)
val input = bytes.iterator val input = bytes.iterator
val decoded = Array.ofDim[Double](n) val decoded = new Array[Double](n)
for (i 0 until a) decoded(i) = input.getDouble(byteOrder) for (i 0 until a) decoded(i) = input.getDouble(byteOrder)
input.getDoubles(decoded, a, b - a)(byteOrder) input.getDoubles(decoded, a, b - a)(byteOrder)
for (i b until n) decoded(i) = input.getDouble(byteOrder) for (i b until n) decoded(i) = input.getDouble(byteOrder)
@ -222,7 +222,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
def testShortEncoding(slice: ArraySlice[Short], byteOrder: ByteOrder): Boolean = { def testShortEncoding(slice: ArraySlice[Short], byteOrder: ByteOrder): Boolean = {
val elemSize = 2 val elemSize = 2
val (data, from, to) = slice val (data, from, to) = slice
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asShortBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asShortBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until from) builder.putShort(data(i))(byteOrder) for (i 0 until from) builder.putShort(data(i))(byteOrder)
@ -234,7 +234,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
def testIntEncoding(slice: ArraySlice[Int], byteOrder: ByteOrder): Boolean = { def testIntEncoding(slice: ArraySlice[Int], byteOrder: ByteOrder): Boolean = {
val elemSize = 4 val elemSize = 4
val (data, from, to) = slice val (data, from, to) = slice
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asIntBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asIntBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until from) builder.putInt(data(i))(byteOrder) for (i 0 until from) builder.putInt(data(i))(byteOrder)
@ -246,7 +246,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
def testLongEncoding(slice: ArraySlice[Long], byteOrder: ByteOrder): Boolean = { def testLongEncoding(slice: ArraySlice[Long], byteOrder: ByteOrder): Boolean = {
val elemSize = 8 val elemSize = 8
val (data, from, to) = slice val (data, from, to) = slice
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asLongBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asLongBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until from) builder.putLong(data(i))(byteOrder) for (i 0 until from) builder.putLong(data(i))(byteOrder)
@ -259,7 +259,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val elemSize = 8 val elemSize = 8
val (data, nBytes) = anb val (data, nBytes) = anb
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asLongBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asLongBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until data.length) builder.putLongPart(data(i), nBytes)(byteOrder) for (i 0 until data.length) builder.putLongPart(data(i), nBytes)(byteOrder)
@ -273,7 +273,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
def testFloatEncoding(slice: ArraySlice[Float], byteOrder: ByteOrder): Boolean = { def testFloatEncoding(slice: ArraySlice[Float], byteOrder: ByteOrder): Boolean = {
val elemSize = 4 val elemSize = 4
val (data, from, to) = slice val (data, from, to) = slice
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asFloatBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asFloatBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until from) builder.putFloat(data(i))(byteOrder) for (i 0 until from) builder.putFloat(data(i))(byteOrder)
@ -285,7 +285,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
def testDoubleEncoding(slice: ArraySlice[Double], byteOrder: ByteOrder): Boolean = { def testDoubleEncoding(slice: ArraySlice[Double], byteOrder: ByteOrder): Boolean = {
val elemSize = 8 val elemSize = 8
val (data, from, to) = slice val (data, from, to) = slice
val reference = Array.ofDim[Byte](data.length * elemSize) val reference = new Array[Byte](data.length * elemSize)
ByteBuffer.wrap(reference).order(byteOrder).asDoubleBuffer.put(data) ByteBuffer.wrap(reference).order(byteOrder).asDoubleBuffer.put(data)
val builder = ByteString.newBuilder val builder = ByteString.newBuilder
for (i 0 until from) builder.putDouble(data(i))(byteOrder) for (i 0 until from) builder.putDouble(data(i))(byteOrder)
@ -734,7 +734,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
check { slice: ByteStringSlice check { slice: ByteStringSlice
slice match { slice match {
case (xs, from, until) likeVector(xs)({ it case (xs, from, until) likeVector(xs)({ it
val array = Array.ofDim[Byte](xs.length) val array = new Array[Byte](xs.length)
it.slice(from, until).copyToArray(array, from, until) it.slice(from, until).copyToArray(array, from, until)
array.toSeq array.toSeq
}) })
@ -815,7 +815,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
check { slice: ByteStringSlice check { slice: ByteStringSlice
slice match { slice match {
case (xs, from, until) likeVecIt(xs)({ it case (xs, from, until) likeVecIt(xs)({ it
val array = Array.ofDim[Byte](xs.length) val array = new Array[Byte](xs.length)
it.slice(from, until).copyToArray(array, from, until) it.slice(from, until).copyToArray(array, from, until)
array.toSeq array.toSeq
}, strict = false) }, strict = false)
@ -830,7 +830,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
check { slice: ByteStringSlice check { slice: ByteStringSlice
val (bytes, from, to) = slice val (bytes, from, to) = slice
val input = bytes.iterator val input = bytes.iterator
val output = Array.ofDim[Byte](bytes.length) val output = new Array[Byte](bytes.length)
for (i 0 until from) output(i) = input.getByte for (i 0 until from) output(i) = input.getByte
input.getBytes(output, from, to - from) input.getBytes(output, from, to - from)
for (i to until bytes.length) output(i) = input.getByte for (i to until bytes.length) output(i) = input.getByte
@ -863,7 +863,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val a = (0 max from) min bytes.length val a = (0 max from) min bytes.length
val b = (a max to) min bytes.length val b = (a max to) min bytes.length
val input = bytes.iterator val input = bytes.iterator
val output = Array.ofDim[Byte](bytes.length) val output = new Array[Byte](bytes.length)
input.asInputStream.skip(a) input.asInputStream.skip(a)
@ -890,7 +890,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val buffer = ByteBuffer.allocate(bytes.size) val buffer = ByteBuffer.allocate(bytes.size)
bytes.copyToBuffer(buffer) bytes.copyToBuffer(buffer)
buffer.flip() buffer.flip()
val array = Array.ofDim[Byte](bytes.size) val array = new Array[Byte](bytes.size)
buffer.get(array) buffer.get(array)
bytes == array.toSeq bytes == array.toSeq
} }

View file

@ -151,7 +151,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
case ps if ps.length == 0 SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array()) case ps if ps.length == 0 SerializedMethodCall(method.getDeclaringClass, method.getName, method.getParameterTypes, Array())
case ps case ps
val serialization = SerializationExtension(akka.serialization.JavaSerializer.currentSystem.value) val serialization = SerializationExtension(akka.serialization.JavaSerializer.currentSystem.value)
val serializedParameters = Array.ofDim[(Int, Class[_], Array[Byte])](ps.length) val serializedParameters = new Array[(Int, Class[_], Array[Byte])](ps.length)
for (i 0 until ps.length) { for (i 0 until ps.length) {
val p = ps(i) val p = ps(i)
val s = serialization.findSerializerFor(p) val s = serialization.findSerializerFor(p)
@ -182,7 +182,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
case null null case null null
case a if a.length == 0 Array[AnyRef]() case a if a.length == 0 Array[AnyRef]()
case a case a
val deserializedParameters: Array[AnyRef] = Array.ofDim[AnyRef](a.length) //Mutable for the sake of sanity val deserializedParameters: Array[AnyRef] = new Array[AnyRef](a.length) //Mutable for the sake of sanity
for (i 0 until a.length) { for (i 0 until a.length) {
val (sId, manifest, bytes) = a(i) val (sId, manifest, bytes) = a(i)
deserializedParameters(i) = deserializedParameters(i) =

View file

@ -188,7 +188,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
case ser: ByteBufferSerializer case ser: ByteBufferSerializer
ser.fromBinary(buf, manifest) ser.fromBinary(buf, manifest)
case _ case _
val bytes = Array.ofDim[Byte](buf.remaining()) val bytes = new Array[Byte](buf.remaining())
buf.get(bytes) buf.get(bytes)
deserializeByteArray(bytes, serializer, manifest) deserializeByteArray(bytes, serializer, manifest)
} }
@ -362,7 +362,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
*/ */
private val quickSerializerByIdentity: Array[Serializer] = { private val quickSerializerByIdentity: Array[Serializer] = {
val size = 1024 val size = 1024
val table = Array.ofDim[Serializer](size) val table = new Array[Serializer](size)
serializerByIdentity.foreach { serializerByIdentity.foreach {
case (id, ser) if (0 <= id && id < size) table(id) = ser case (id, ser) if (0 <= id && id < size) table(id) = ser
} }

View file

@ -168,7 +168,7 @@ abstract class SerializerWithStringManifest extends Serializer {
* try { * try {
* toBinary(o, buf) * toBinary(o, buf)
* buf.flip() * buf.flip()
* val bytes = Array.ofDim[Byte](buf.remaining) * val bytes = new Array[Byte](buf.remaining)
* buf.get(bytes) * buf.get(bytes)
* bytes * bytes
* } finally { * } finally {
@ -402,7 +402,7 @@ class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerialize
} }
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
val bytes = Array.ofDim[Byte](buf.remaining()) val bytes = new Array[Byte](buf.remaining())
buf.get(bytes) buf.get(bytes)
bytes bytes
} }

View file

@ -13,7 +13,6 @@ import scala.reflect.ClassTag
object ByteIterator { object ByteIterator {
object ByteArrayIterator { object ByteArrayIterator {
private val emptyArray: Array[Byte] = Array.ofDim[Byte](0)
protected[akka] def apply(array: Array[Byte]): ByteArrayIterator = protected[akka] def apply(array: Array[Byte]): ByteArrayIterator =
new ByteArrayIterator(array, 0, array.length) new ByteArrayIterator(array, 0, array.length)
@ -21,7 +20,7 @@ object ByteIterator {
protected[akka] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator = protected[akka] def apply(array: Array[Byte], from: Int, until: Int): ByteArrayIterator =
new ByteArrayIterator(array, from, until) new ByteArrayIterator(array, from, until)
val empty: ByteArrayIterator = apply(emptyArray) val empty: ByteArrayIterator = apply(Array.emptyByteArray)
} }
class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int) extends ByteIterator { class ByteArrayIterator private (private var array: Array[Byte], private var from: Int, private var until: Int) extends ByteIterator {
@ -38,7 +37,7 @@ object ByteIterator {
else { val i = from; from = from + 1; array(i) } else { val i = from; from = from + 1; array(i) }
} }
def clear(): Unit = { this.array = ByteArrayIterator.emptyArray; from = 0; until = from } def clear(): Unit = { this.array = Array.emptyByteArray; from = 0; until = from }
final override def length: Int = { val l = len; clear(); l } final override def length: Int = { val l = len; clear(); l }
@ -460,7 +459,7 @@ abstract class ByteIterator extends BufferedIterator[Byte] {
} }
override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = { override def toArray[B >: Byte](implicit arg0: ClassTag[B]): Array[B] = {
val target = Array.ofDim[B](len) val target = new Array[B](len)
copyToArray(target) copyToArray(target)
target target
} }

View file

@ -17,7 +17,7 @@ import akka.stream.stage.OutHandler
*/ */
class BenchTestSource(elementCount: Int) extends GraphStage[SourceShape[java.lang.Integer]] { class BenchTestSource(elementCount: Int) extends GraphStage[SourceShape[java.lang.Integer]] {
private val elements = Array.ofDim[java.lang.Integer](elementCount) private val elements = new Array[java.lang.Integer](elementCount)
(1 to elementCount).map(n => elements(n - 1) = n) (1 to elementCount).map(n => elements(n - 1) = n)
val out: Outlet[java.lang.Integer] = Outlet("BenchTestSource") val out: Outlet[java.lang.Integer] = Outlet("BenchTestSource")

View file

@ -468,7 +468,7 @@ private[metrics] class WeightedRoutees(routees: immutable.IndexedSeq[Routee], se
case a a case a a
} }
} }
val buckets = Array.ofDim[Int](routees.size) val buckets = new Array[Int](routees.size)
val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size
val w = weights.withDefaultValue(meanWeight) // we dont necessarily have metrics for all addresses val w = weights.withDefaultValue(meanWeight) // we dont necessarily have metrics for all addresses
var i = 0 var i = 0

View file

@ -173,10 +173,10 @@ final class LmdbDurableStore(config: Config) extends Actor with ActorLogging {
var n = 0 var n = 0
val loadData = LoadData(iter.asScala.map { entry val loadData = LoadData(iter.asScala.map { entry
n += 1 n += 1
val keyArray = Array.ofDim[Byte](entry.key.remaining) val keyArray = new Array[Byte](entry.key.remaining)
entry.key.get(keyArray) entry.key.get(keyArray)
val key = new String(keyArray, ByteString.UTF_8) val key = new String(keyArray, ByteString.UTF_8)
val valArray = Array.ofDim[Byte](entry.`val`.remaining) val valArray = new Array[Byte](entry.`val`.remaining)
entry.`val`.get(valArray) entry.`val`.get(valArray)
val envelope = serializer.fromBinary(valArray, manifest).asInstanceOf[DurableDataEnvelope] val envelope = serializer.fromBinary(valArray, manifest).asInstanceOf[DurableDataEnvelope]
key envelope key envelope

View file

@ -52,7 +52,7 @@ import akka.cluster.ddata.PruningState.PruningPerformed
private val n = new AtomicInteger(0) private val n = new AtomicInteger(0)
private val mask = size - 1 private val mask = size - 1
private val elements = Array.ofDim[(A, B)](size) private val elements = new Array[(A, B)](size)
private val ttlNanos = timeToLive.toNanos private val ttlNanos = timeToLive.toNanos
// in theory this should be volatile, but since the cache has low // in theory this should be volatile, but since the cache has low

View file

@ -24,7 +24,7 @@ class ByteBufferSerializerDocSpec {
toBinary(o, buf) toBinary(o, buf)
buf.flip() buf.flip()
val bytes = Array.ofDim[Byte](buf.remaining) val bytes = new Array[Byte](buf.remaining)
buf.get(bytes) buf.get(bytes)
bytes bytes
} }

View file

@ -89,7 +89,7 @@ abstract class JournalPerfSpec(config: Config) extends JournalSpec(config) {
/** Executes a block of code multiple times (no warm-up) */ /** Executes a block of code multiple times (no warm-up) */
def measure(msg: Duration String)(block: Unit): Unit = { def measure(msg: Duration String)(block: Unit): Unit = {
val measurements = Array.ofDim[Duration](measurementIterations) val measurements = new Array[Duration](measurementIterations)
var i = 0 var i = 0
while (i < measurementIterations) { while (i < measurementIterations) {
val start = System.nanoTime() val start = System.nanoTime()

View file

@ -78,7 +78,7 @@ class SnapshotSerializer(val system: ExtendedActorSystem) extends BaseSerializer
val manifest = val manifest =
if (remaining == 0) "" if (remaining == 0) ""
else { else {
val manifestBytes = Array.ofDim[Byte](remaining) val manifestBytes = new Array[Byte](remaining)
in.read(manifestBytes) in.read(manifestBytes)
new String(manifestBytes, UTF_8) new String(manifestBytes, UTF_8)
} }

View file

@ -12,7 +12,7 @@ package object serialization {
*/ */
def streamToBytes(inputStream: InputStream): Array[Byte] = { def streamToBytes(inputStream: InputStream): Array[Byte] = {
val len = 16384 val len = 16384
val buf = Array.ofDim[Byte](len) val buf = new Array[Byte](len)
val out = new ByteArrayOutputStream val out = new ByteArrayOutputStream
@scala.annotation.tailrec @scala.annotation.tailrec

View file

@ -310,7 +310,7 @@ object MaxThroughputSpec extends MultiNodeConfig {
val buf = ByteBuffer.allocate(8) val buf = ByteBuffer.allocate(8)
toBinary(o, buf) toBinary(o, buf)
buf.flip() buf.flip()
val bytes = Array.ofDim[Byte](buf.remaining) val bytes = new Array[Byte](buf.remaining)
buf.get(bytes) buf.get(bytes)
bytes bytes
} }

View file

@ -136,7 +136,7 @@ private[remote] class Association(
private val queueSize = advancedSettings.OutboundMessageQueueSize private val queueSize = advancedSettings.OutboundMessageQueueSize
private val largeQueueSize = advancedSettings.OutboundLargeMessageQueueSize private val largeQueueSize = advancedSettings.OutboundLargeMessageQueueSize
private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = Array.ofDim(2 + outboundLanes) private[this] val queues: Array[SendQueue.ProducerApi[OutboundEnvelope]] = new Array(2 + outboundLanes)
queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream queues(ControlQueueIndex) = QueueWrapperImpl(createQueue(controlQueueSize)) // control stream
queues(LargeQueueIndex) = queues(LargeQueueIndex) =
if (transport.largeMessageChannelEnabled) // large messages stream if (transport.largeMessageChannelEnabled) // large messages stream

View file

@ -365,8 +365,8 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
import EnvelopeBuffer._ import EnvelopeBuffer._
val aeronBuffer = new UnsafeBuffer(byteBuffer) val aeronBuffer = new UnsafeBuffer(byteBuffer)
private var literalChars = Array.ofDim[Char](64) private var literalChars = new Array[Char](64)
private var literalBytes = Array.ofDim[Byte](64) private var literalBytes = new Array[Byte](64)
def writeHeader(h: HeaderBuilder): Unit = writeHeader(h, null) def writeHeader(h: HeaderBuilder): Unit = writeHeader(h, null)
@ -514,8 +514,8 @@ private[remote] final class EnvelopeBuffer(val byteBuffer: ByteBuffer) {
private def ensureLiteralCharsLength(length: Int): Unit = { private def ensureLiteralCharsLength(length: Int): Unit = {
if (length > literalChars.length) { if (length > literalChars.length) {
literalChars = Array.ofDim[Char](length) literalChars = new Array[Char](length)
literalBytes = Array.ofDim[Byte](length) literalBytes = new Array[Byte](length)
} }
} }

View file

@ -147,7 +147,7 @@ private[akka] final class FlightRecorderReader(fileChannel: FileChannel) {
val recordStartOffset = recordOffset + RollingEventLogSection.CommitEntrySize val recordStartOffset = recordOffset + RollingEventLogSection.CommitEntrySize
// FIXME: extract magic numbers // FIXME: extract magic numbers
val metadata = Array.ofDim[Byte](fileBuffer.getByte(recordStartOffset + 20)) val metadata = new Array[Byte](fileBuffer.getByte(recordStartOffset + 20))
fileBuffer.getBytes(recordStartOffset + 21, metadata) fileBuffer.getBytes(recordStartOffset + 21, metadata)
val entry = RichEntry( val entry = RichEntry(

View file

@ -59,19 +59,19 @@ private[akka] class ImmutableLongMap[A >: Null] private (
val i = Arrays.binarySearch(keys, key) val i = Arrays.binarySearch(keys, key)
if (i >= 0) { if (i >= 0) {
// existing key, replace value // existing key, replace value
val newValues = Array.ofDim[A](values.length) val newValues = new Array[A](values.length)
System.arraycopy(values, 0, newValues, 0, values.length) System.arraycopy(values, 0, newValues, 0, values.length)
newValues(i) = value newValues(i) = value
new ImmutableLongMap(keys, newValues) new ImmutableLongMap(keys, newValues)
} else { } else {
// insert the entry at the right position, and keep the arrays sorted // insert the entry at the right position, and keep the arrays sorted
val j = -(i + 1) val j = -(i + 1)
val newKeys = Array.ofDim[Long](size + 1) val newKeys = new Array[Long](size + 1)
System.arraycopy(keys, 0, newKeys, 0, j) System.arraycopy(keys, 0, newKeys, 0, j)
newKeys(j) = key newKeys(j) = key
System.arraycopy(keys, j, newKeys, j + 1, keys.length - j) System.arraycopy(keys, j, newKeys, j + 1, keys.length - j)
val newValues = Array.ofDim[A](size + 1) val newValues = new Array[A](size + 1)
System.arraycopy(values, 0, newValues, 0, j) System.arraycopy(values, 0, newValues, 0, j)
newValues(j) = value newValues(j) = value
System.arraycopy(values, j, newValues, j + 1, values.length - j) System.arraycopy(values, j, newValues, j + 1, values.length - j)
@ -87,11 +87,11 @@ private[akka] class ImmutableLongMap[A >: Null] private (
if (size == 1) if (size == 1)
ImmutableLongMap.empty ImmutableLongMap.empty
else { else {
val newKeys = Array.ofDim[Long](size - 1) val newKeys = new Array[Long](size - 1)
System.arraycopy(keys, 0, newKeys, 0, i) System.arraycopy(keys, 0, newKeys, 0, i)
System.arraycopy(keys, i + 1, newKeys, i, keys.length - i - 1) System.arraycopy(keys, i + 1, newKeys, i, keys.length - i - 1)
val newValues = Array.ofDim[A](size - 1) val newValues = new Array[A](size - 1)
System.arraycopy(values, 0, newValues, 0, i) System.arraycopy(values, 0, newValues, 0, i)
System.arraycopy(values, i + 1, newValues, i, values.length - i - 1) System.arraycopy(values, i + 1, newValues, i, values.length - i - 1)

View file

@ -67,7 +67,7 @@ private[akka] abstract class LruBoundedCache[K: ClassTag, V <: AnyRef: ClassTag]
private[this] val keys = Array.ofDim[K](capacity) private[this] val keys = Array.ofDim[K](capacity)
private[this] val values = Array.ofDim[V](capacity) private[this] val values = Array.ofDim[V](capacity)
private[this] val hashes = Array.ofDim[Int](capacity) private[this] val hashes = new Array[Int](capacity)
private[this] val epochs = Array.fill[Int](capacity)(epoch - evictAgeThreshold) // Guarantee existing "values" are stale private[this] val epochs = Array.fill[Int](capacity)(epoch - evictAgeThreshold) // Guarantee existing "values" are stale
final def get(k: K): Option[V] = { final def get(k: K): Option[V] = {

View file

@ -31,8 +31,8 @@ private[remote] final case class CompressionTable[T](originUid: Long, version: B
require(dictionary.values.sum + dictionary.size == expectedGaplessSum, "Given compression map does not seem to be gap-less and starting from zero, " + require(dictionary.values.sum + dictionary.size == expectedGaplessSum, "Given compression map does not seem to be gap-less and starting from zero, " +
"which makes compressing it into an Array difficult, bailing out! Map was: " + dictionary) "which makes compressing it into an Array difficult, bailing out! Map was: " + dictionary)
val tups = Array.ofDim[(Object, Int)](dictionary.size).asInstanceOf[Array[(T, Int)]] val tups = new Array[(Object, Int)](dictionary.size).asInstanceOf[Array[(T, Int)]]
val ts = Array.ofDim[Object](dictionary.size).asInstanceOf[Array[T]] val ts = new Array[Object](dictionary.size).asInstanceOf[Array[T]]
var i = 0 var i = 0
val mit = dictionary.iterator val mit = dictionary.iterator

View file

@ -34,14 +34,14 @@ private[remote] final class TopHeavyHitters[T >: Null](val max: Int)(implicit cl
// Contains the hash value for each entry in the hashmap. Used for quicker lookups (equality check can be avoided // Contains the hash value for each entry in the hashmap. Used for quicker lookups (equality check can be avoided
// if hashes don't match) // if hashes don't match)
private[this] val hashes: Array[Int] = Array.ofDim(capacity) private[this] val hashes: Array[Int] = new Array(capacity)
// Actual stored elements in the hashmap // Actual stored elements in the hashmap
private[this] val items: Array[T] = Array.ofDim[T](capacity) private[this] val items: Array[T] = Array.ofDim[T](capacity)
// Index of stored element in the associated heap // Index of stored element in the associated heap
private[this] val heapIndex: Array[Int] = Array.fill(capacity)(-1) private[this] val heapIndex: Array[Int] = Array.fill(capacity)(-1)
// Weights associated with an entry in the hashmap. Used to maintain the heap property and give easy access to low // Weights associated with an entry in the hashmap. Used to maintain the heap property and give easy access to low
// weight entries // weight entries
private[this] val weights: Array[Long] = Array.ofDim(capacity) private[this] val weights: Array[Long] = new Array(capacity)
// Heap structure containing indices to slots in the hashmap // Heap structure containing indices to slots in the hashmap
private[this] val heap: Array[Int] = Array.fill(max)(-1) private[this] val heap: Array[Int] = Array.fill(max)(-1)

View file

@ -19,7 +19,7 @@ class LongSerializer(val system: ExtendedActorSystem) extends BaseSerializer wit
} }
override def toBinary(o: AnyRef): Array[Byte] = { override def toBinary(o: AnyRef): Array[Byte] = {
val result = Array.ofDim[Byte](8) val result = new Array[Byte](8)
var long = Long.unbox(o) var long = Long.unbox(o)
var i = 0 var i = 0
while (long != 0) { while (long != 0) {
@ -50,7 +50,7 @@ class IntSerializer(val system: ExtendedActorSystem) extends BaseSerializer with
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = Int.box(buf.getInt) override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = Int.box(buf.getInt)
override def toBinary(o: AnyRef): Array[Byte] = { override def toBinary(o: AnyRef): Array[Byte] = {
val result = Array.ofDim[Byte](4) val result = new Array[Byte](4)
var int = Int.unbox(o) var int = Int.unbox(o)
var i = 0 var i = 0
while (int != 0) { while (int != 0) {
@ -79,7 +79,7 @@ class StringSerializer(val system: ExtendedActorSystem) extends BaseSerializer w
override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.put(toBinary(o)) override def toBinary(o: AnyRef, buf: ByteBuffer): Unit = buf.put(toBinary(o))
override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = { override def fromBinary(buf: ByteBuffer, manifest: String): AnyRef = {
val bytes = Array.ofDim[Byte](buf.remaining()) val bytes = new Array[Byte](buf.remaining())
buf.get(bytes) buf.get(bytes)
new String(bytes, "UTF-8") new String(bytes, "UTF-8")
} }
@ -106,7 +106,7 @@ class ByteStringSerializer(val system: ExtendedActorSystem) extends BaseSerializ
override def toBinary(o: AnyRef): Array[Byte] = { override def toBinary(o: AnyRef): Array[Byte] = {
val bs = o.asInstanceOf[ByteString] val bs = o.asInstanceOf[ByteString]
val result = Array.ofDim[Byte](bs.length) val result = new Array[Byte](bs.length)
bs.copyToArray(result, 0, bs.length) bs.copyToArray(result, 0, bs.length)
result result
} }

View file

@ -64,7 +64,7 @@ class AeronSinkSpec extends AkkaSpec with ImplicitSender {
.runWith(Sink.ignore) .runWith(Sink.ignore)
// use large enough messages to fill up buffers // use large enough messages to fill up buffers
val payload = Array.ofDim[Byte](100000) val payload = new Array[Byte](100000)
val done = Source(1 to 1000).map(_ payload) val done = Source(1 to 1000).map(_ payload)
.map { n .map { n
val envelope = pool.acquire() val envelope = pool.acquire()

View file

@ -151,7 +151,7 @@ class FlightRecorderSpec extends AkkaSpec {
"properly truncate low frequency event metadata if necessary" in withFlightRecorder { (recorder, reader, channel) "properly truncate low frequency event metadata if necessary" in withFlightRecorder { (recorder, reader, channel)
val sink = recorder.createEventSink() val sink = recorder.createEventSink()
val longMetadata = Array.ofDim[Byte](1024) val longMetadata = new Array[Byte](1024)
sink.loFreq(0, longMetadata) sink.loFreq(0, longMetadata)
channel.force(false) channel.force(false)
@ -160,7 +160,7 @@ class FlightRecorderSpec extends AkkaSpec {
val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq val entries = reader.structure.loFreqLog.logs(0).richEntries.toSeq
entries.size should ===(1) entries.size should ===(1)
entries.head.metadata should ===(Array.ofDim[Byte](FlightRecorder.LoFreqRecordSize - 32)) entries.head.metadata should ===(new Array[Byte](FlightRecorder.LoFreqRecordSize - 32))
} }

View file

@ -106,7 +106,7 @@ class LargeMessagesStreamSpec extends ArteryMultiNodeSpec(
val remoteProbe = TestProbe()(systemA) val remoteProbe = TestProbe()(systemA)
val largeBytes = 2000000 val largeBytes = 2000000
largeRemote.tell(Ping(ByteString.fromArray(Array.ofDim[Byte](largeBytes))), remoteProbe.ref) largeRemote.tell(Ping(ByteString.fromArray(new Array[Byte](largeBytes))), remoteProbe.ref)
regularRemote.tell(Ping(), remoteProbe.ref) regularRemote.tell(Ping(), remoteProbe.ref)
// should be no problems sending regular small messages while large messages are being sent // should be no problems sending regular small messages while large messages are being sent

View file

@ -119,7 +119,7 @@ class RollingEventLogSimulationSpec extends AkkaSpec {
class Simulator(writerCount: Int, entryCount: Int, totalWrites: Int) { class Simulator(writerCount: Int, entryCount: Int, totalWrites: Int) {
var headPointer = 0 var headPointer = 0
val simulatedBuffer = Array.ofDim[Byte](4 * entryCount) val simulatedBuffer = new Array[Byte](4 * entryCount)
val writers = Array.tabulate(writerCount)(new Writer(_, entryCount, totalWrites)) val writers = Array.tabulate(writerCount)(new Writer(_, entryCount, totalWrites))
var activeWriters = writerCount var activeWriters = writerCount
var log: List[String] = Nil var log: List[String] = Nil

View file

@ -47,7 +47,7 @@ class PrimitivesSerializationSpec extends AkkaSpec(PrimitivesSerializationSpec.t
buffer.flip() buffer.flip()
// also make sure that the Array and ByteBuffer formats are equal, given LITTLE_ENDIAN // also make sure that the Array and ByteBuffer formats are equal, given LITTLE_ENDIAN
val array1 = Array.ofDim[Byte](buffer.remaining()) val array1 = new Array[Byte](buffer.remaining())
buffer.get(array1) buffer.get(array1)
val array2 = serializer.toBinary(msg) val array2 = serializer.toBinary(msg)
ByteString(array1) should ===(ByteString(array2)) ByteString(array1) should ===(ByteString(array2))

View file

@ -100,8 +100,8 @@ object TraversalTestUtils {
var islandStack: List[(IslandTag, Attributes)] = (TestDefaultIsland, Attributes.none) :: Nil var islandStack: List[(IslandTag, Attributes)] = (TestDefaultIsland, Attributes.none) :: Nil
val connections = b.inSlots val connections = b.inSlots
val inlets = Array.ofDim[InPort](connections) val inlets = new Array[InPort](connections)
val outlets = Array.ofDim[OutPort](connections) val outlets = new Array[OutPort](connections)
// Track next assignable number for input ports // Track next assignable number for input ports
var inOffs = 0 var inOffs = 0

View file

@ -38,7 +38,7 @@ object GraphInterpreterSpecKit {
var inOwners = SMap.empty[Inlet[_], GraphStageLogic] var inOwners = SMap.empty[Inlet[_], GraphStageLogic]
var outOwners = SMap.empty[Outlet[_], GraphStageLogic] var outOwners = SMap.empty[Outlet[_], GraphStageLogic]
val logics = Array.ofDim[GraphStageLogic](upstreams.length + stages.length + downstreams.length) val logics = new Array[GraphStageLogic](upstreams.length + stages.length + downstreams.length)
var idx = 0 var idx = 0
while (idx < upstreams.length) { while (idx < upstreams.length) {
@ -132,7 +132,7 @@ object GraphInterpreterSpecKit {
inOwners: SMap[Inlet[_], GraphStageLogic], inOwners: SMap[Inlet[_], GraphStageLogic],
outOwners: SMap[Outlet[_], GraphStageLogic]): Array[Connection] = { outOwners: SMap[Outlet[_], GraphStageLogic]): Array[Connection] = {
val connections = Array.ofDim[Connection](connectedPorts.size) val connections = new Array[Connection](connectedPorts.size)
connectedPorts.zipWithIndex.foreach { connectedPorts.zipWithIndex.foreach {
case ((outlet, inlet), idx) case ((outlet, inlet), idx)

View file

@ -161,7 +161,7 @@ class FramingSpec extends StreamSpec {
} }
} }
ByteString(Array.ofDim[Byte](fieldOffset)) ++ header ++ payload ByteString(new Array[Byte](fieldOffset)) ++ header ++ payload
} }
"work with various byte orders, frame lengths and offsets" taggedAs LongRunningTest in { "work with various byte orders, frame lengths and offsets" taggedAs LongRunningTest in {

View file

@ -198,7 +198,7 @@ class SinkSpec extends StreamSpec with DefaultTimeout with ScalaFutures {
} }
def supplier(): Supplier[Array[Int]] = new Supplier[Array[Int]] { def supplier(): Supplier[Array[Int]] = new Supplier[Array[Int]] {
override def get(): Array[Int] = Array.ofDim(1) override def get(): Array[Int] = new Array(1)
} }
def accumulator(): BiConsumer[Array[Int], Int] = new BiConsumer[Array[Int], Int] { def accumulator(): BiConsumer[Array[Int], Int] = new BiConsumer[Array[Int], Int] {
override def accept(a: Array[Int], b: Int): Unit = a(0) = intIdentity.applyAsInt(b) override def accept(a: Array[Int], b: Int): Unit = a(0) = intIdentity.applyAsInt(b)

View file

@ -176,7 +176,7 @@ class UnfoldResourceAsyncSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
"work with ByteString as well" in assertAllStagesStopped { "work with ByteString as well" in assertAllStagesStopped {
val chunkSize = 50 val chunkSize = 50
val buffer = Array.ofDim[Char](chunkSize) val buffer = new Array[Char](chunkSize)
val p = Source.unfoldResourceAsync[ByteString, Reader]( val p = Source.unfoldResourceAsync[ByteString, Reader](
open, open,
reader { reader {

View file

@ -112,7 +112,7 @@ class UnfoldResourceSourceSpec extends StreamSpec(UnboundedMailboxConfig) {
"work with ByteString as well" in assertAllStagesStopped { "work with ByteString as well" in assertAllStagesStopped {
val chunkSize = 50 val chunkSize = 50
val buffer = Array.ofDim[Char](chunkSize) val buffer = new Array[Char](chunkSize)
val p = Source.unfoldResource[ByteString, Reader]( val p = Source.unfoldResource[ByteString, Reader](
() new BufferedReader(new FileReader(manyLinesFile)), () new BufferedReader(new FileReader(manyLinesFile)),
reader { reader {