From 7bd11eb9f3743cda9088453613bbacb0cdb6ea4a Mon Sep 17 00:00:00 2001 From: Bruno Woltzenlogel Paleo Date: Sat, 7 Apr 2018 11:07:48 +1000 Subject: [PATCH] Replace Array[Byte] by ByteString and Seq[Byte] --- src/main/scala/scorex/ObjectGenerators.scala | 11 ++-- .../scala/scorex/core/NodeViewHolder.scala | 19 ++---- .../scala/scorex/core/NodeViewModifier.scala | 2 +- .../core/api/http/NodeViewApiRoute.scala | 5 +- src/main/scala/scorex/core/app/Version.scala | 8 ++- .../scala/scorex/core/consensus/History.scala | 2 +- src/main/scala/scorex/core/core.scala | 8 +-- .../scorex/core/network/DeliveryTracker.scala | 65 ++++++++----------- .../scala/scorex/core/network/Handshake.scala | 22 ++++--- .../core/network/NodeViewSynchronizer.scala | 6 +- .../core/network/PeerConnectionHandler.scala | 3 +- .../network/message/BasicMessagesRepo.scala | 56 ++++++++-------- .../scorex/core/network/message/Message.scala | 24 ++++--- .../serialization/BytesSerializable.scala | 2 +- .../core/serialization/Serializer.scala | 6 +- .../core/transaction/BoxTransaction.scala | 11 ++-- .../scorex/core/transaction/Transaction.scala | 4 +- .../account/PublicKeyNoncedBox.scala | 5 +- .../PublicKey25519Proposition.scala | 6 +- .../scorex/core/transaction/proof/Proof.scala | 2 +- .../transaction/proof/Signature25519.scala | 8 +-- .../core/transaction/state/SecretHolder.scala | 6 +- .../core/transaction/wallet/Wallet.scala | 37 ++++++----- src/main/scala/scorex/core/utils/utils.scala | 1 + 24 files changed, 164 insertions(+), 155 deletions(-) diff --git a/src/main/scala/scorex/ObjectGenerators.scala b/src/main/scala/scorex/ObjectGenerators.scala index b161f2505..cb43f7289 100644 --- a/src/main/scala/scorex/ObjectGenerators.scala +++ b/src/main/scala/scorex/ObjectGenerators.scala @@ -2,6 +2,7 @@ package scorex import java.net.{InetAddress, InetSocketAddress} +import akka.util.ByteString import org.scalacheck.{Arbitrary, Gen} import scorex.core.app.Version import scorex.core.network.message.BasicMsgDataTypes._ @@ -18,8 +19,8 @@ trait ObjectGenerators { lazy val smallInt: Gen[Int] = Gen.choose(0, 20) - lazy val nonEmptyBytesGen: Gen[Array[Byte]] = Gen.nonEmptyListOf(Arbitrary.arbitrary[Byte]) - .map(_.toArray).suchThat(_.length > 0) + lazy val nonEmptyBytesGen: Gen[ByteString] = Gen.nonEmptyListOf(Arbitrary.arbitrary[Byte]) + .map(ByteString(_:_*)).suchThat(_.nonEmpty) def genBoundedBytes(minSize: Int, maxSize: Int): Gen[Array[Byte]] = { Gen.choose(minSize, maxSize) flatMap { sz => Gen.listOfN(sz, Arbitrary.arbitrary[Byte]).map(_.toArray) } @@ -33,7 +34,7 @@ trait ObjectGenerators { lazy val modifierIdGen: Gen[ModifierId] = Gen.listOfN(NodeViewModifier.ModifierIdSize, Arbitrary.arbitrary[Byte]) - .map(id => ModifierId @@ id.toArray) + .map(id => ModifierId @@ Seq(id:_*)) lazy val modifierTypeIdGen: Gen[ModifierTypeId] = Arbitrary.arbitrary[Byte].map(t => ModifierTypeId @@ t) @@ -42,14 +43,14 @@ trait ObjectGenerators { modifierIds: Seq[ModifierId] <- Gen.nonEmptyListOf(modifierIdGen) if modifierIds.nonEmpty } yield modifierTypeId -> modifierIds - lazy val modifierWithIdGen: Gen[(ModifierId, Array[Byte])] = for { + lazy val modifierWithIdGen: Gen[(ModifierId, ByteString)] = for { id <- modifierIdGen mod <- nonEmptyBytesGen } yield id -> mod lazy val modifiersGen: Gen[ModifiersData] = for { modifierTypeId: ModifierTypeId <- modifierTypeIdGen - modifiers: Map[ModifierId, Array[Byte]] <- Gen.nonEmptyMap(modifierWithIdGen).suchThat(_.nonEmpty) + modifiers: Map[ModifierId, ByteString] <- Gen.nonEmptyMap(modifierWithIdGen).suchThat(_.nonEmpty) } yield modifierTypeId -> modifiers diff --git a/src/main/scala/scorex/core/NodeViewHolder.scala b/src/main/scala/scorex/core/NodeViewHolder.scala index c865fea39..6adc53a3d 100644 --- a/src/main/scala/scorex/core/NodeViewHolder.scala +++ b/src/main/scala/scorex/core/NodeViewHolder.scala @@ -90,16 +90,11 @@ trait NodeViewHolder[P <: Proposition, TX <: Transaction[P], PMOD <: PersistentN */ val networkChunkSize: Int - - protected type MapKey = scala.collection.mutable.WrappedArray.ofByte - - protected def key(id: ModifierId): MapKey = new mutable.WrappedArray.ofByte(id) - /** * Cache for modifiers. If modifiers are coming out-of-order, they are to be stored in this cache. */ //todo: make configurable limited size - private val modifiersCache = mutable.Map[MapKey, PMOD]() + private val modifiersCache = mutable.Map[ModifierId, PMOD]() protected def txModify(tx: TX): Unit = { //todo: async validation? @@ -315,7 +310,7 @@ trait NodeViewHolder[P <: Proposition, TX <: Transaction[P], PMOD <: PersistentN case typeId: ModifierTypeId if typeId == Transaction.ModifierTypeId => memoryPool().notIn(modifierIds) case _ => - modifierIds.filterNot(mid => history().contains(mid) || modifiersCache.contains(key(mid))) + modifierIds.filterNot(mid => history().contains(mid) || modifiersCache.contains(mid)) } sender() ! RequestFromLocal(peer, modifierTypeId, ids) @@ -329,14 +324,14 @@ trait NodeViewHolder[P <: Proposition, TX <: Transaction[P], PMOD <: PersistentN txModify(tx) case pmod: PMOD@unchecked => - if (history().contains(pmod) || modifiersCache.contains(key(pmod.id))) { + if (history().contains(pmod) || modifiersCache.contains(pmod.id)) { log.warn(s"Received modifier ${pmod.encodedId} that is already in history") } else { - modifiersCache.put(key(pmod.id), pmod) + modifiersCache.put(pmod.id, pmod) } } - log.debug(s"Cache before(${modifiersCache.size}): ${modifiersCache.keySet.map(_.array).map(Base58.encode).mkString(",")}") + log.debug(s"Cache before(${modifiersCache.size}): ${modifiersCache.keySet.map(id => Base58.encode(id.toArray)).mkString(",")}") var t: Option[PMOD] = None do { @@ -352,7 +347,7 @@ trait NodeViewHolder[P <: Proposition, TX <: Transaction[P], PMOD <: PersistentN t.foreach(pmodModify) } while (t.isDefined) - log.debug(s"Cache after(${modifiersCache.size}): ${modifiersCache.keySet.map(_.array).map(Base58.encode).mkString(",")}") + log.debug(s"Cache after(${modifiersCache.size}): ${modifiersCache.keySet.map(id => Base58.encode(id.toArray)).mkString(",")}") } } @@ -398,7 +393,7 @@ object NodeViewHolder { // Moved from NodeViewSynchronizer as this was only received here case class CompareViews(source: ConnectedPeer, modifierTypeId: ModifierTypeId, modifierIds: Seq[ModifierId]) - case class ModifiersFromRemote(source: ConnectedPeer, modifierTypeId: ModifierTypeId, remoteObjects: Seq[Array[Byte]]) + case class ModifiersFromRemote(source: ConnectedPeer, modifierTypeId: ModifierTypeId, remoteObjects: Seq[Seq[Byte]]) case class LocallyGeneratedTransaction[P <: Proposition, TX <: Transaction[P]](tx: TX) case class LocallyGeneratedModifier[PMOD <: PersistentNodeViewModifier](pmod: PMOD) diff --git a/src/main/scala/scorex/core/NodeViewModifier.scala b/src/main/scala/scorex/core/NodeViewModifier.scala index ab5559fe6..199af014e 100644 --- a/src/main/scala/scorex/core/NodeViewModifier.scala +++ b/src/main/scala/scorex/core/NodeViewModifier.scala @@ -16,7 +16,7 @@ sealed trait NodeViewModifier extends BytesSerializable { //todo: check statically or dynamically output size def id: ModifierId - def encodedId: String = Base58.encode(id) + def encodedId: String = Base58.encode(id.toArray) override def equals(obj: scala.Any): Boolean = obj match { case that: NodeViewModifier => (that.id sameElements id) && (that.modifierTypeId == modifierTypeId) diff --git a/src/main/scala/scorex/core/api/http/NodeViewApiRoute.scala b/src/main/scala/scorex/core/api/http/NodeViewApiRoute.scala index 09e1b6b58..51b911803 100644 --- a/src/main/scala/scorex/core/api/http/NodeViewApiRoute.scala +++ b/src/main/scala/scorex/core/api/http/NodeViewApiRoute.scala @@ -4,6 +4,7 @@ import akka.actor.{ActorRef, ActorRefFactory} import akka.http.scaladsl.model.StatusCodes import akka.http.scaladsl.server.Route import akka.pattern.ask +import akka.util.ByteString import io.circe.syntax._ import scorex.core.NodeViewHolder.CurrentView import scorex.core.consensus.History @@ -61,7 +62,7 @@ case class NodeViewApiRoute[P <: Proposition, TX <: Transaction[P]] Base58.decode(encodedId) match { case Failure(e) => complete(ApiError.notExists) case Success(rawId) => - val id = ModifierId @@ rawId + val id: ModifierId = ModifierId @@ rawId.toSeq def f(v: CurrentView[HIS, MS, VL, MP]): Option[PM] = v.history.modifierById(id) @@ -95,7 +96,7 @@ case class NodeViewApiRoute[P <: Proposition, TX <: Transaction[P]] def openSurface: Route = (get & path("openSurface")) { withOpenSurface { os => - complete(SuccessApiResponse(os.ids.map(Base58.encode).asJson)) + complete(SuccessApiResponse(os.ids.map(id => Base58.encode(id.toArray)).asJson)) } } diff --git a/src/main/scala/scorex/core/app/Version.scala b/src/main/scala/scorex/core/app/Version.scala index 73dd46a38..4a7a00097 100644 --- a/src/main/scala/scorex/core/app/Version.scala +++ b/src/main/scala/scorex/core/app/Version.scala @@ -1,6 +1,8 @@ package scorex.core.app +import akka.util.ByteString import scorex.core.serialization.{BytesSerializable, Serializer} + import scala.util.Try object Version { @@ -18,10 +20,10 @@ case class Version(firstDigit: Byte, secondDigit: Byte, thirdDigit: Byte) extend object ApplicationVersionSerializer extends Serializer[Version] { val SerializedVersionLength = 3 - override def toBytes(obj: Version): Array[Byte] = - Array(obj.firstDigit, obj.secondDigit, obj.thirdDigit) + override def toBytes(obj: Version): Seq[Byte] = + ByteString(obj.firstDigit, obj.secondDigit, obj.thirdDigit) - override def parseBytes(bytes: Array[Byte]): Try[Version] = Try { + override def parseBytes(bytes: Seq[Byte]): Try[Version] = Try { Version( bytes(0), bytes(1), diff --git a/src/main/scala/scorex/core/consensus/History.scala b/src/main/scala/scorex/core/consensus/History.scala index 0e55d8049..b4b1d4b35 100644 --- a/src/main/scala/scorex/core/consensus/History.scala +++ b/src/main/scala/scorex/core/consensus/History.scala @@ -70,7 +70,7 @@ object History { lazy val chainSwitchingNeeded: Boolean = toRemove.nonEmpty override def toString: String = { - s"ProgressInfo(BranchPoint: ${branchPoint.map(Base58.encode)}, " + + s"ProgressInfo(BranchPoint: ${branchPoint.map(id => Base58.encode(id.toArray))}, " + s" to remove: ${toRemove.map(_.encodedId)}, to apply: ${toApply.map(_.encodedId)})" } } diff --git a/src/main/scala/scorex/core/core.scala b/src/main/scala/scorex/core/core.scala index 91d517a5f..9869f4a67 100644 --- a/src/main/scala/scorex/core/core.scala +++ b/src/main/scala/scorex/core/core.scala @@ -9,9 +9,9 @@ package object core { //TODO implement ModifierTypeId as a trait object ModifierTypeId extends TaggedType[Byte] - object ModifierId extends TaggedType[Array[Byte]] + object ModifierId extends TaggedType[Seq[Byte]] - object VersionTag extends TaggedType[Array[Byte]] + object VersionTag extends TaggedType[Seq[Byte]] type ModifierTypeId = ModifierTypeId.Type @@ -20,8 +20,8 @@ package object core { type VersionTag = VersionTag.Type def idsToString(ids: Seq[(ModifierTypeId, ModifierId)]): String = (ids.headOption, ids.lastOption) match { - case (Some(f), Some(l)) if f._2 sameElements l._2 => s"[(${f._1},${Base58.encode(f._2)})]" - case (Some(f), Some(l)) => s"[(${f._1},${Base58.encode(f._2)})..(${l._1},${Base58.encode(l._2)})]" + case (Some(f), Some(l)) if f._2 == l._2 => s"[(${f._1},${Base58.encode(f._2.toArray)})]" + case (Some(f), Some(l)) => s"[(${f._1},${Base58.encode(f._2.toArray)})..(${l._1},${Base58.encode(l._2.toArray)})]" case _ => "[]" } diff --git a/src/main/scala/scorex/core/network/DeliveryTracker.scala b/src/main/scala/scorex/core/network/DeliveryTracker.scala index b6ad21ac9..34a056082 100644 --- a/src/main/scala/scorex/core/network/DeliveryTracker.scala +++ b/src/main/scala/scorex/core/network/DeliveryTracker.scala @@ -20,76 +20,67 @@ class DeliveryTracker(context: ActorContext, maxDeliveryChecks: Int, nvsRef: ActorRef) extends ScorexLogging { - protected type ModifierIdAsKey = scala.collection.mutable.WrappedArray.ofByte - - protected def key(id: ModifierId): ModifierIdAsKey = new mutable.WrappedArray.ofByte(id) - // todo: Do we need to keep track of ModifierTypeIds? Maybe we could ignore them? // when a remote peer is asked a modifier, we add the expected data to `expecting` // when a remote peer delivers expected data, it is removed from `expecting` and added to `delivered`. // when a remote peer delivers unexpected data, it is added to `deliveredSpam`. - protected val expecting = mutable.Set[(ModifierTypeId, ModifierIdAsKey, ConnectedPeer)]() - protected val delivered = mutable.Map[ModifierIdAsKey, ConnectedPeer]() - protected val deliveredSpam = mutable.Map[ModifierIdAsKey, ConnectedPeer]() + protected val expecting = mutable.Set[(ModifierTypeId, ModifierId, ConnectedPeer)]() + protected val delivered = mutable.Map[ModifierId, ConnectedPeer]() + protected val deliveredSpam = mutable.Map[ModifierId, ConnectedPeer]() - protected val cancellables = mutable.Map[(ModifierIdAsKey, ConnectedPeer), Cancellable]() - protected val checksCounter = mutable.Map[(ModifierIdAsKey, ConnectedPeer), Int]() + protected val cancellables = mutable.Map[(ModifierId, ConnectedPeer), Cancellable]() + protected val checksCounter = mutable.Map[(ModifierId, ConnectedPeer), Int]() def expect(cp: ConnectedPeer, mtid: ModifierTypeId, mids: Seq[ModifierId])(implicit ec: ExecutionContext): Unit = tryWithLogging { - for (mid <- mids) expect(cp, mtid, mid) - } - - protected def expect(cp: ConnectedPeer, mtid: ModifierTypeId, mid: ModifierId)(implicit ec: ExecutionContext): Unit = { - val cancellable = context.system.scheduler.scheduleOnce(deliveryTimeout, - nvsRef, - CheckDelivery(cp, mtid, mid)) - val midAsKey = key(mid) - expecting += ((mtid, midAsKey, cp)) - cancellables((midAsKey, cp)) = cancellable + for (mid <- mids) { + val cancellable = context.system.scheduler.scheduleOnce(deliveryTimeout, + nvsRef, + CheckDelivery(cp, mtid, mid)) + expecting += ((mtid, mid, cp)) + cancellables((mid, cp)) = cancellable + } } // stops expecting, and expects again if the number of checks does not exceed the maximum def reexpect(cp: ConnectedPeer, mtid: ModifierTypeId, mid: ModifierId)(implicit ec: ExecutionContext): Unit = tryWithLogging { stopExpecting(cp, mtid, mid) - val midAsKey = key(mid) - val checks = checksCounter.getOrElseUpdate((midAsKey, cp), 0) + 1 - checksCounter((midAsKey, cp)) = checks - if (checks < maxDeliveryChecks) expect(cp, mtid, mid) - else checksCounter -= ((midAsKey, cp)) + val checks = checksCounter.getOrElseUpdate((mid, cp), 0) + 1 + checksCounter((mid, cp)) = checks + if (checks < maxDeliveryChecks) expect(cp, mtid, Seq(mid)) + else checksCounter -= ((mid, cp)) } protected def stopExpecting(cp: ConnectedPeer, mtid: ModifierTypeId, mid: ModifierId): Unit = { - val midAsKey = key(mid) - expecting -= ((mtid, midAsKey, cp)) - cancellables((midAsKey, cp)).cancel() - cancellables -= ((midAsKey, cp)) + expecting -= ((mtid, mid, cp)) + cancellables((mid, cp)).cancel() + cancellables -= ((mid, cp)) } protected def isExpecting(mtid: ModifierTypeId, mid: ModifierId, cp: ConnectedPeer): Boolean = - expecting.exists(e => (mtid == e._1) && (mid sameElements e._2.array) && cp == e._3) + expecting.exists(e => (mtid == e._1) && (mid == e._2) && cp == e._3) def receive(mtid: ModifierTypeId, mid: ModifierId, cp: ConnectedPeer): Unit = tryWithLogging { if (isExpecting(mtid, mid, cp)) { - val eo = expecting.find(e => (mtid == e._1) && (mid sameElements e._2) && cp == e._3) + val eo = expecting.find(e => (mtid == e._1) && (mid == e._2) && cp == e._3) for (e <- eo) expecting -= e - delivered(key(mid)) = cp - val cancellableKey = (key(mid), cp) + delivered(mid) = cp + val cancellableKey = (mid, cp) for (c <- cancellables.get(cancellableKey)) c.cancel() cancellables -= cancellableKey } else { - deliveredSpam(key(mid)) = cp + deliveredSpam(mid) = cp } } - def delete(mid: ModifierId): Unit = tryWithLogging(delivered -= key(mid)) + def delete(mid: ModifierId): Unit = tryWithLogging(delivered -= mid) - def deleteSpam(mids: Seq[ModifierId]): Unit = for (id <- mids) tryWithLogging(deliveredSpam -= key(id)) + def deleteSpam(mids: Seq[ModifierId]): Unit = for (id <- mids) tryWithLogging(deliveredSpam -= id) - def isSpam(mid: ModifierId): Boolean = deliveredSpam contains key(mid) + def isSpam(mid: ModifierId): Boolean = deliveredSpam contains mid - def peerWhoDelivered(mid: ModifierId): Option[ConnectedPeer] = delivered.get(key(mid)) + def peerWhoDelivered(mid: ModifierId): Option[ConnectedPeer] = delivered.get(mid) protected def tryWithLogging(fn: => Unit): Unit = { Try(fn).recoverWith { diff --git a/src/main/scala/scorex/core/network/Handshake.scala b/src/main/scala/scorex/core/network/Handshake.scala index 45df92ed4..ff2eb96a0 100644 --- a/src/main/scala/scorex/core/network/Handshake.scala +++ b/src/main/scala/scorex/core/network/Handshake.scala @@ -2,8 +2,9 @@ package scorex.core.network import java.net.{InetAddress, InetSocketAddress} +import akka.util.ByteString import com.google.common.primitives.{Ints, Longs} -import scorex.core.app.{Version, ApplicationVersionSerializer} +import scorex.core.app.{ApplicationVersionSerializer, Version} import scorex.core.serialization.{BytesSerializable, Serializer} import scala.util.Try @@ -25,7 +26,7 @@ case class Handshake(applicationName: String, object HandshakeSerializer extends Serializer[Handshake] { - override def toBytes(obj: Handshake): Array[Byte] = { + override def toBytes(obj: Handshake): Seq[Byte] = { val anb = obj.applicationName.getBytes val fab = obj.declaredAddress.map { isa => @@ -34,7 +35,7 @@ object HandshakeSerializer extends Serializer[Handshake] { val nodeNameBytes = obj.nodeName.getBytes - Array(anb.size.toByte) ++ anb ++ + ByteString(anb.size.toByte) ++ anb ++ obj.protocolVersion.bytes ++ Array(nodeNameBytes.size.toByte) ++ nodeNameBytes ++ Ints.toByteArray(fab.length) ++ fab ++ @@ -42,14 +43,15 @@ object HandshakeSerializer extends Serializer[Handshake] { } - override def parseBytes(bytes: Array[Byte]): Try[Handshake] = Try { + @SuppressWarnings(Array("org.wartremover.warts.TraversableOps")) + override def parseBytes(bytes: Seq[Byte]): Try[Handshake] = Try { var position = 0 val appNameSize = bytes.head require(appNameSize > 0) position += 1 - val an = new String(bytes.slice(position, position + appNameSize)) + val an = new String(bytes.slice(position, position + appNameSize).toArray) // fixme: use `toString`? position += appNameSize val av = ApplicationVersionSerializer.parseBytes( @@ -59,23 +61,23 @@ object HandshakeSerializer extends Serializer[Handshake] { val nodeNameSize = bytes.slice(position, position + 1).head position += 1 - val nodeName = new String(bytes.slice(position, position + nodeNameSize)) + val nodeName = new String(bytes.slice(position, position + nodeNameSize).toArray) // fixme position += nodeNameSize - val fas = Ints.fromByteArray(bytes.slice(position, position + 4)) + val fas = Ints.fromByteArray(bytes.slice(position, position + 4).toArray) position += 4 val isaOpt = if (fas > 0) { val fa = bytes.slice(position, position + fas - 4) position += fas - 4 - val port = Ints.fromByteArray(bytes.slice(position, position + 4)) + val port = Ints.fromByteArray(bytes.slice(position, position + 4).toArray) position += 4 - Some(new InetSocketAddress(InetAddress.getByAddress(fa), port)) + Some(new InetSocketAddress(InetAddress.getByAddress(fa.toArray), port)) } else None - val time = Longs.fromByteArray(bytes.slice(position, position + 8)) + val time = Longs.fromByteArray(bytes.slice(position, position + 8).toArray) Handshake(an, av, nodeName, isaOpt, time) } diff --git a/src/main/scala/scorex/core/network/NodeViewSynchronizer.scala b/src/main/scala/scorex/core/network/NodeViewSynchronizer.scala index 03e50c03a..1057956b7 100644 --- a/src/main/scala/scorex/core/network/NodeViewSynchronizer.scala +++ b/src/main/scala/scorex/core/network/NodeViewSynchronizer.scala @@ -211,7 +211,7 @@ MR <: MempoolReader[TX]](networkControllerRef: ActorRef, val modifiers = data._2 log.info(s"Got modifiers of type $typeId from remote connected peer: $remote") - log.trace(s"Received modifier ids ${data._2.keySet.map(Base58.encode).mkString(",")}") + log.trace(s"Received modifier ids ${data._2.keySet.map(id => Base58.encode(id.toArray)).mkString(",")}") for ((id, _) <- modifiers) deliveryTracker.receive(typeId, id, remote) @@ -219,7 +219,7 @@ MR <: MempoolReader[TX]](networkControllerRef: ActorRef, if (spam.nonEmpty) { log.info(s"Spam attempt: peer $remote has sent a non-requested modifiers of type $typeId with ids" + - s": ${spam.keys.map(Base58.encode)}") + s": ${spam.keys.map(id => Base58.encode(id.toArray))}") penalizeSpammingPeer(remote) val mids = spam.keys.toSeq deliveryTracker.deleteSpam(mids) @@ -252,7 +252,7 @@ MR <: MempoolReader[TX]](networkControllerRef: ActorRef, deliveryTracker.delete(modifierId) } else { - log.info(s"Peer $peer has not delivered asked modifier ${Base58.encode(modifierId)} on time") + log.info(s"Peer $peer has not delivered asked modifier ${Base58.encode(modifierId.toArray)} on time") penalizeNonDeliveringPeer(peer) deliveryTracker.reexpect(peer, modifierTypeId, modifierId) } diff --git a/src/main/scala/scorex/core/network/PeerConnectionHandler.scala b/src/main/scala/scorex/core/network/PeerConnectionHandler.scala index 9bb4bc01c..db5db2245 100644 --- a/src/main/scala/scorex/core/network/PeerConnectionHandler.scala +++ b/src/main/scala/scorex/core/network/PeerConnectionHandler.scala @@ -106,7 +106,8 @@ class PeerConnectionHandler(val settings: NetworkSettings, Version(settings.appVersion), settings.nodeName, ownSocketAddress, timeProvider.time()).bytes - connection ! Tcp.Write(ByteString(hb)) + val bs: ByteString = ByteString(hb:_*) + connection ! Tcp.Write(bs) log.info(s"Handshake sent to $remote") handshakeSent = true if (handshakeGot && handshakeSent) self ! HandshakeDone diff --git a/src/main/scala/scorex/core/network/message/BasicMessagesRepo.scala b/src/main/scala/scorex/core/network/message/BasicMessagesRepo.scala index dd245c919..01c46638c 100644 --- a/src/main/scala/scorex/core/network/message/BasicMessagesRepo.scala +++ b/src/main/scala/scorex/core/network/message/BasicMessagesRepo.scala @@ -4,30 +4,30 @@ package scorex.core.network.message import java.net.{InetAddress, InetSocketAddress} import java.util +import akka.util.ByteString import com.google.common.primitives.{Bytes, Ints} import scorex.core.{ModifierId, ModifierTypeId, NodeViewModifier} import scorex.core.consensus.SyncInfo import scorex.core.network.message.Message.{MessageCode, _} -import scorex.core.settings.NetworkSettings import scala.util.Try object BasicMsgDataTypes { type InvData = (ModifierTypeId, Seq[ModifierId]) - type ModifiersData = (ModifierTypeId, Map[ModifierId, Array[Byte]]) + type ModifiersData = (ModifierTypeId, Map[ModifierId, Seq[Byte]]) } import scorex.core.network.message.BasicMsgDataTypes._ -class SyncInfoMessageSpec[SI <: SyncInfo](deserializer: Array[Byte] => Try[SI]) extends MessageSpec[SI] { +class SyncInfoMessageSpec[SI <: SyncInfo](deserializer: Seq[Byte] => Try[SI]) extends MessageSpec[SI] { override val messageCode: MessageCode = 65: Byte override val messageName: String = "Sync" - override def parseBytes(bytes: Array[Byte]): Try[SI] = deserializer(bytes) + override def parseBytes(bytes: Seq[Byte]): Try[SI] = deserializer(bytes) - override def toBytes(data: SI): Array[Byte] = data.bytes + override def toBytes(data: SI): Seq[Byte] = data.bytes } object InvSpec { @@ -40,9 +40,10 @@ class InvSpec(maxInvObjects: Int) extends MessageSpec[InvData] { override val messageCode = MessageCode override val messageName: String = MessageName - override def parseBytes(bytes: Array[Byte]): Try[InvData] = Try { + @SuppressWarnings(Array("org.wartremover.warts.TraversableOps")) + override def parseBytes(bytes: Seq[Byte]): Try[InvData] = Try { val typeId = ModifierTypeId @@ bytes.head - val count = Ints.fromByteArray(bytes.slice(1, 5)) + val count = Ints.fromByteArray(bytes.slice(1, 5).toArray) require(count > 0, "empty inv list") require(count <= maxInvObjects, s"more invs than $maxInvObjects in a message") @@ -54,12 +55,12 @@ class InvSpec(maxInvObjects: Int) extends MessageSpec[InvData] { typeId -> elems } - override def toBytes(data: InvData): Array[Byte] = { + override def toBytes(data: InvData): Seq[Byte] = { require(data._2.nonEmpty, "empty inv list") require(data._2.size <= maxInvObjects, s"more invs than $maxInvObjects in a message") data._2.foreach(e => require(e.length == NodeViewModifier.ModifierIdSize)) - Bytes.concat(Array(data._1), Ints.toByteArray(data._2.size), scorex.core.utils.concatBytes(data._2)) + ByteString(data._1) ++ Ints.toByteArray(data._2.size) ++ data._2.flatten } } @@ -76,10 +77,10 @@ class RequestModifierSpec(maxInvObjects: Int) private val invSpec = new InvSpec(maxInvObjects) - override def toBytes(typeAndId: InvData): Array[Byte] = + override def toBytes(typeAndId: InvData): Seq[Byte] = invSpec.toBytes(typeAndId) - override def parseBytes(bytes: Array[Byte]): Try[InvData] = + override def parseBytes(bytes: Seq[Byte]): Try[InvData] = invSpec.parseBytes(bytes) } @@ -89,15 +90,16 @@ object ModifiersSpec extends MessageSpec[ModifiersData] { override val messageCode: MessageCode = 33: Byte override val messageName: String = "Modifier" - override def parseBytes(bytes: Array[Byte]): Try[ModifiersData] = Try { + @SuppressWarnings(Array("org.wartremover.warts.TraversableOps")) + override def parseBytes(bytes: Seq[Byte]): Try[ModifiersData] = Try { val typeId = ModifierTypeId @@ bytes.head - val count = Ints.fromByteArray(bytes.slice(1, 5)) + val count = Ints.fromByteArray(bytes.slice(1, 5).toArray) val objBytes = bytes.slice(5, bytes.length) - val (_, seq) = (0 until count).foldLeft(0 -> Seq[(ModifierId, Array[Byte])]()) { + val (_, seq) = (0 until count).foldLeft(0 -> Seq[(ModifierId, Seq[Byte])]()) { case ((pos, collected), _) => val id = ModifierId @@ objBytes.slice(pos, pos + NodeViewModifier.ModifierIdSize) - val objBytesCnt = Ints.fromByteArray(objBytes.slice(pos + NodeViewModifier.ModifierIdSize, pos + NodeViewModifier.ModifierIdSize + 4)) + val objBytesCnt = Ints.fromByteArray(objBytes.slice(pos + NodeViewModifier.ModifierIdSize, pos + NodeViewModifier.ModifierIdSize + 4).toArray) val obj = objBytes.slice(pos + NodeViewModifier.ModifierIdSize + 4, pos + NodeViewModifier.ModifierIdSize + 4 + objBytesCnt) (pos + NodeViewModifier.ModifierIdSize + 4 + objBytesCnt) -> (collected :+ (id -> obj)) @@ -105,13 +107,13 @@ object ModifiersSpec extends MessageSpec[ModifiersData] { typeId -> seq.toMap } - override def toBytes(data: ModifiersData): Array[Byte] = { + override def toBytes(data: ModifiersData): Seq[Byte] = { require(data._2.nonEmpty, "empty modifiers list") val typeId = data._1 val modifiers = data._2 - Array(typeId) ++ Ints.toByteArray(modifiers.size) ++ modifiers.map { case (id, modifier) => + ByteString(typeId) ++ Ints.toByteArray(modifiers.size) ++ modifiers.map { case (id, modifier) => id ++ Ints.toByteArray(modifier.length) ++ modifier - }.fold(Array[Byte]())(_ ++ _) + }.fold(ByteString())(_ ++ _) } } @@ -120,10 +122,10 @@ object GetPeersSpec extends MessageSpec[Unit] { override val messageName: String = "GetPeers message" - override def parseBytes(bytes: Array[Byte]): Try[Unit] = + override def parseBytes(bytes: Seq[Byte]): Try[Unit] = Try(require(bytes.isEmpty, "Non-empty data for GetPeers")) - override def toBytes(data: Unit): Array[Byte] = Array() + override def toBytes(data: Unit): Seq[Byte] = ByteString() } object PeersSpec extends MessageSpec[Seq[InetSocketAddress]] { @@ -135,27 +137,27 @@ object PeersSpec extends MessageSpec[Seq[InetSocketAddress]] { override val messageName: String = "Peers message" - override def parseBytes(bytes: Array[Byte]): Try[Seq[InetSocketAddress]] = Try { - val lengthBytes = util.Arrays.copyOfRange(bytes, 0, DataLength) + override def parseBytes(bytes: Seq[Byte]): Try[Seq[InetSocketAddress]] = Try { + val lengthBytes = util.Arrays.copyOfRange(bytes.toArray, 0, DataLength) val length = Ints.fromByteArray(lengthBytes) require(bytes.length == DataLength + (length * (AddressLength + PortLength)), "Data does not match length") (0 until length).map { i => val position = lengthBytes.length + (i * (AddressLength + PortLength)) - val addressBytes = util.Arrays.copyOfRange(bytes, position, position + AddressLength) + val addressBytes = util.Arrays.copyOfRange(bytes.toArray, position, position + AddressLength) val address = InetAddress.getByAddress(addressBytes) - val portBytes = util.Arrays.copyOfRange(bytes, position + AddressLength, position + AddressLength + PortLength) + val portBytes = util.Arrays.copyOfRange(bytes.toArray, position + AddressLength, position + AddressLength + PortLength) new InetSocketAddress(address, Ints.fromByteArray(portBytes)) } } - override def toBytes(peers: Seq[InetSocketAddress]): Array[Byte] = { + override def toBytes(peers: Seq[InetSocketAddress]): Seq[Byte] = { val length = peers.size val lengthBytes = Ints.toByteArray(length) - peers.foldLeft(lengthBytes) { case (bs, peer) => - Bytes.concat(bs, peer.getAddress.getAddress, Ints.toByteArray(peer.getPort)) + peers.foldLeft(ByteString(lengthBytes)) { (bs, peer) => + bs ++ peer.getAddress.getAddress ++ Ints.toByteArray(peer.getPort) } } } \ No newline at end of file diff --git a/src/main/scala/scorex/core/network/message/Message.scala b/src/main/scala/scorex/core/network/message/Message.scala index 1df1ad23c..53fbdd63f 100644 --- a/src/main/scala/scorex/core/network/message/Message.scala +++ b/src/main/scala/scorex/core/network/message/Message.scala @@ -1,14 +1,15 @@ package scorex.core.network.message -import com.google.common.primitives.{Bytes, Ints} +import com.google.common.primitives.Ints import scorex.core.network.ConnectedPeer import scorex.core.serialization.{BytesSerializable, Serializer} import scorex.crypto.hash.Blake2b256 import scala.util.{Success, Try} +import akka.util.ByteString case class Message[Content](spec: MessageSpec[Content], - input: Either[Array[Byte], Content], + input: Either[Seq[Byte], Content], source: Option[ConnectedPeer]) extends BytesSerializable { lazy val dataBytes = input match { @@ -32,23 +33,26 @@ class MessageSerializer[Content] extends Serializer[Message[Content]] { import Message.{ChecksumLength, MAGIC} - override def toBytes(obj: Message[Content]): Array[Byte] = { + override def toBytes(obj: Message[Content]): Seq[Byte] = { val dataWithChecksum = if (obj.dataLength > 0) { - val checksum = Blake2b256.hash(obj.dataBytes).take(ChecksumLength) - Bytes.concat(checksum, obj.dataBytes) - } else obj.dataBytes //empty array - - MAGIC ++ Array(obj.spec.messageCode) ++ Ints.toByteArray(obj.dataLength) ++ dataWithChecksum + val checksum = Blake2b256.hash(obj.dataBytes.toArray).take(ChecksumLength) + ByteString(checksum) ++ obj.dataBytes + } else obj.dataBytes // empty + + MAGIC ++ + ByteString(obj.spec.messageCode) ++ + ByteString(Ints.toByteArray(obj.dataLength)) ++ + dataWithChecksum } //TODO move MessageHandler.parseBytes here - override def parseBytes(bytes: Array[Byte]): Try[Message[Content]] = ??? + override def parseBytes(bytes: Seq[Byte]): Try[Message[Content]] = ??? } object Message { type MessageCode = Byte - val MAGIC = Array(0x12: Byte, 0x34: Byte, 0x56: Byte, 0x78: Byte) + val MAGIC = ByteString(0x12: Byte, 0x34: Byte, 0x56: Byte, 0x78: Byte) val MagicLength = MAGIC.length diff --git a/src/main/scala/scorex/core/serialization/BytesSerializable.scala b/src/main/scala/scorex/core/serialization/BytesSerializable.scala index 12ff349b7..f2bcf32c1 100644 --- a/src/main/scala/scorex/core/serialization/BytesSerializable.scala +++ b/src/main/scala/scorex/core/serialization/BytesSerializable.scala @@ -4,7 +4,7 @@ trait BytesSerializable extends Serializable { type M >: this.type <: BytesSerializable - lazy val bytes: Array[Byte] = serializer.toBytes(this) + lazy val bytes: Seq[Byte] = serializer.toBytes(this) def serializer: Serializer[M] } diff --git a/src/main/scala/scorex/core/serialization/Serializer.scala b/src/main/scala/scorex/core/serialization/Serializer.scala index af46ac5a6..183016d4e 100644 --- a/src/main/scala/scorex/core/serialization/Serializer.scala +++ b/src/main/scala/scorex/core/serialization/Serializer.scala @@ -1,9 +1,11 @@ package scorex.core.serialization +// import akka.util.ByteString + import scala.util.Try trait Serializer[M] { - def toBytes(obj: M): Array[Byte] + def toBytes(obj: M): Seq[Byte] - def parseBytes(bytes: Array[Byte]): Try[M] + def parseBytes(bytes: Seq[Byte]): Try[M] } diff --git a/src/main/scala/scorex/core/transaction/BoxTransaction.scala b/src/main/scala/scorex/core/transaction/BoxTransaction.scala index 260162933..6c3fdeb8d 100644 --- a/src/main/scala/scorex/core/transaction/BoxTransaction.scala +++ b/src/main/scala/scorex/core/transaction/BoxTransaction.scala @@ -1,5 +1,6 @@ package scorex.core.transaction +import akka.util.ByteString import com.google.common.primitives.{Bytes, Longs} import scorex.core.transaction.box.proposition.Proposition import scorex.core.transaction.box.{Box, BoxUnlocker} @@ -14,9 +15,9 @@ abstract class BoxTransaction[P <: Proposition, BX <: Box[P]] extends Transactio val timestamp: Long - override lazy val messageToSign: Array[Byte] = - Bytes.concat(if (newBoxes.nonEmpty) scorex.core.utils.concatBytes(newBoxes.map(_.bytes)) else Array[Byte](), - scorex.core.utils.concatFixLengthBytes(unlockers.map(_.closedBoxId)), - Longs.toByteArray(timestamp), - Longs.toByteArray(fee)) + override lazy val messageToSign: Seq[Byte] = + newBoxes.foldLeft(ByteString()) { (acc, box) => acc ++ box.bytes } ++ + scorex.core.utils.concatFixLengthBytes(unlockers.map(_.closedBoxId)) ++ + Longs.toByteArray(timestamp) ++ + Longs.toByteArray(fee) } diff --git a/src/main/scala/scorex/core/transaction/Transaction.scala b/src/main/scala/scorex/core/transaction/Transaction.scala index a5e8c225d..8a09ff9f7 100644 --- a/src/main/scala/scorex/core/transaction/Transaction.scala +++ b/src/main/scala/scorex/core/transaction/Transaction.scala @@ -12,9 +12,9 @@ import scorex.crypto.hash.Blake2b256 abstract class Transaction[P <: Proposition] extends EphemerealNodeViewModifier { override val modifierTypeId: ModifierTypeId = Transaction.ModifierTypeId - val messageToSign: Array[Byte] + val messageToSign: Seq[Byte] - override lazy val id: ModifierId = ModifierId @@ Blake2b256(messageToSign) + override lazy val id: ModifierId = ModifierId @@ Blake2b256(messageToSign.toArray).toSeq } diff --git a/src/main/scala/scorex/core/transaction/account/PublicKeyNoncedBox.scala b/src/main/scala/scorex/core/transaction/account/PublicKeyNoncedBox.scala index 9e40245b8..3bf5d55e1 100644 --- a/src/main/scala/scorex/core/transaction/account/PublicKeyNoncedBox.scala +++ b/src/main/scala/scorex/core/transaction/account/PublicKeyNoncedBox.scala @@ -1,5 +1,6 @@ package scorex.core.transaction.account +import akka.util.ByteString import com.google.common.primitives.Longs import scorex.core.ModifierId import scorex.core.transaction.box.Box @@ -10,7 +11,7 @@ import scorex.crypto.hash.Blake2b256 trait PublicKeyNoncedBox[PKP <: PublicKey25519Proposition] extends Box[PKP] { val nonce: Long - lazy val id: ADKey = ADKey @@ PublicKeyNoncedBox.idFromBox(proposition, nonce) + lazy val id: ADKey = ADKey @@ PublicKeyNoncedBox.idFromBox(proposition, nonce).toArray lazy val publicKey: PKP = proposition @@ -24,5 +25,5 @@ trait PublicKeyNoncedBox[PKP <: PublicKey25519Proposition] extends Box[PKP] { object PublicKeyNoncedBox { def idFromBox[PKP <: PublicKey25519Proposition](prop: PKP, nonce: Long): ModifierId = - ModifierId @@ Blake2b256(prop.pubKeyBytes ++ Longs.toByteArray(nonce)) + ModifierId @@ (prop.pubKeyBytes ++ Longs.toByteArray(nonce)).toSeq } \ No newline at end of file diff --git a/src/main/scala/scorex/core/transaction/box/proposition/PublicKey25519Proposition.scala b/src/main/scala/scorex/core/transaction/box/proposition/PublicKey25519Proposition.scala index 6a3b777dd..c0c125ac8 100644 --- a/src/main/scala/scorex/core/transaction/box/proposition/PublicKey25519Proposition.scala +++ b/src/main/scala/scorex/core/transaction/box/proposition/PublicKey25519Proposition.scala @@ -37,10 +37,10 @@ case class PublicKey25519Proposition(pubKeyBytes: PublicKey) extends ProofOfKnow } object PublicKey25519PropositionSerializer extends Serializer[PublicKey25519Proposition] { - override def toBytes(obj: PublicKey25519Proposition): Array[Byte] = obj.pubKeyBytes + override def toBytes(obj: PublicKey25519Proposition): Seq[Byte] = obj.pubKeyBytes - override def parseBytes(bytes: Array[Byte]): Try[PublicKey25519Proposition] = - Try(PublicKey25519Proposition(PublicKey @@ bytes)) + override def parseBytes(bytes: Seq[Byte]): Try[PublicKey25519Proposition] = + Try(PublicKey25519Proposition(PublicKey @@ bytes.toArray)) } object PublicKey25519Proposition { diff --git a/src/main/scala/scorex/core/transaction/proof/Proof.scala b/src/main/scala/scorex/core/transaction/proof/Proof.scala index fdf4b669a..c0a01be2f 100644 --- a/src/main/scala/scorex/core/transaction/proof/Proof.scala +++ b/src/main/scala/scorex/core/transaction/proof/Proof.scala @@ -12,7 +12,7 @@ import scorex.core.transaction.state.Secret */ trait Proof[P <: Proposition] extends BytesSerializable { - def isValid(proposition: P, message: Array[Byte]): Boolean + def isValid(proposition: P, message: Seq[Byte]): Boolean } trait ProofOfKnowledge[S <: Secret, P <: ProofOfKnowledgeProposition[S]] extends Proof[P] diff --git a/src/main/scala/scorex/core/transaction/proof/Signature25519.scala b/src/main/scala/scorex/core/transaction/proof/Signature25519.scala index a428237d3..3d0d7d2ee 100644 --- a/src/main/scala/scorex/core/transaction/proof/Signature25519.scala +++ b/src/main/scala/scorex/core/transaction/proof/Signature25519.scala @@ -15,8 +15,8 @@ case class Signature25519(signature: Signature) extends ProofOfKnowledge[Private require(signature.isEmpty || signature.length == Curve25519.SignatureLength, s"${signature.length} != ${Curve25519.SignatureLength}") - override def isValid(proposition: PublicKey25519Proposition, message: Array[Byte]): Boolean = - Curve25519.verify(signature, message, proposition.pubKeyBytes) + override def isValid(proposition: PublicKey25519Proposition, message: Seq[Byte]): Boolean = + Curve25519.verify(signature, message.toArray, proposition.pubKeyBytes) override type M = Signature25519 @@ -26,9 +26,9 @@ case class Signature25519(signature: Signature) extends ProofOfKnowledge[Private } object Signature25519Serializer extends Serializer[Signature25519] { - override def toBytes(obj: Signature25519): Array[Byte] = obj.signature + override def toBytes(obj: Signature25519): Seq[Byte] = obj.signature - override def parseBytes(bytes: Array[Byte]): Try[Signature25519] = Try(Signature25519(Signature @@ bytes)) + override def parseBytes(bytes: Seq[Byte]): Try[Signature25519] = Try(Signature25519(Signature @@ bytes.toArray)) } object Signature25519 { diff --git a/src/main/scala/scorex/core/transaction/state/SecretHolder.scala b/src/main/scala/scorex/core/transaction/state/SecretHolder.scala index 68a47c309..c6894ea5f 100644 --- a/src/main/scala/scorex/core/transaction/state/SecretHolder.scala +++ b/src/main/scala/scorex/core/transaction/state/SecretHolder.scala @@ -51,10 +51,10 @@ case class PrivateKey25519(privKeyBytes: PrivateKey, publicKeyBytes: PublicKey) } object PrivateKey25519Serializer extends Serializer[PrivateKey25519] { - override def toBytes(obj: PrivateKey25519): Array[Byte] = Bytes.concat(obj.privKeyBytes, obj.publicKeyBytes) + override def toBytes(obj: PrivateKey25519): Seq[Byte] = obj.privKeyBytes ++ obj.publicKeyBytes - override def parseBytes(bytes: Array[Byte]): Try[PrivateKey25519] = Try { - PrivateKey25519(PrivateKey @@ bytes.slice(0, 32), PublicKey @@ bytes.slice(32, 64)) + override def parseBytes(bytes: Seq[Byte]): Try[PrivateKey25519] = Try { + PrivateKey25519(PrivateKey @@ bytes.slice(0, 32).toArray, PublicKey @@ bytes.slice(32, 64).toArray) } } diff --git a/src/main/scala/scorex/core/transaction/wallet/Wallet.scala b/src/main/scala/scorex/core/transaction/wallet/Wallet.scala index 8f80e4ad5..821dbad61 100644 --- a/src/main/scala/scorex/core/transaction/wallet/Wallet.scala +++ b/src/main/scala/scorex/core/transaction/wallet/Wallet.scala @@ -1,5 +1,6 @@ package scorex.core.transaction.wallet +import akka.util.ByteString import com.google.common.primitives.{Bytes, Ints, Longs} import scorex.core.serialization.{BytesSerializable, Serializer} import scorex.core.transaction.Transaction @@ -12,25 +13,25 @@ import scorex.crypto.encode.Base58 import scala.util.Try //TODO why do we need transactionId and createdAt -case class WalletBox[P <: Proposition, B <: Box[P]](box: B, transactionId: Array[Byte], createdAt: Long) +case class WalletBox[P <: Proposition, B <: Box[P]](box: B, transactionId: Seq[Byte], createdAt: Long) (subclassDeser: Serializer[B]) extends BytesSerializable { override type M = WalletBox[P, B] override def serializer: Serializer[WalletBox[P, B]] = new WalletBoxSerializer(subclassDeser) - override def toString: String = s"WalletBox($box, ${Base58.encode(transactionId)}, $createdAt)" + override def toString: String = s"WalletBox($box, ${Base58.encode(transactionId.toArray)}, $createdAt)" } class WalletBoxSerializer[P <: Proposition, B <: Box[P]](subclassDeser: Serializer[B]) extends Serializer[WalletBox[P, B]] { - override def toBytes(box: WalletBox[P, B]): Array[Byte] = { - Bytes.concat(box.transactionId, Longs.toByteArray(box.createdAt), box.box.bytes) + override def toBytes(box: WalletBox[P, B]): Seq[Byte] = { + box.transactionId ++ Longs.toByteArray(box.createdAt) ++ box.box.bytes } - override def parseBytes(bytes: Array[Byte]): Try[WalletBox[P, B]] = Try { + override def parseBytes(bytes: Seq[Byte]): Try[WalletBox[P, B]] = Try { val txId = bytes.slice(0, NodeViewModifier.ModifierIdSize) val createdAt = Longs.fromByteArray( - bytes.slice(NodeViewModifier.ModifierIdSize, NodeViewModifier.ModifierIdSize + 8)) + bytes.slice(NodeViewModifier.ModifierIdSize, NodeViewModifier.ModifierIdSize + 8).toArray) val boxB = bytes.slice(NodeViewModifier.ModifierIdSize + 8, bytes.length) val box: B = subclassDeser.parseBytes(boxB).get WalletBox[P, B](box, txId, createdAt)(subclassDeser) @@ -43,19 +44,20 @@ case class WalletTransaction[P <: Proposition, TX <: Transaction[P]](proposition createdAt: Long) object WalletTransaction { - def parse[P <: Proposition, TX <: Transaction[P]](bytes: Array[Byte]) - (propDeserializer: Array[Byte] => Try[P], - txDeserializer: Array[Byte] => Try[TX] + def parse[P <: Proposition, TX <: Transaction[P]](bytes: Seq[Byte]) + (propDeserializer: Seq[Byte] => Try[P], + txDeserializer: Seq[Byte] => Try[TX] ): Try[WalletTransaction[P, TX]] = Try { - val propLength = Ints.fromByteArray(bytes.slice(0, 4)) + val propLength = Ints.fromByteArray(bytes.slice(0, 4).toArray) var pos = 4 val propTry = propDeserializer(bytes.slice(pos, pos + propLength)) pos = pos + propLength - val txLength = Ints.fromByteArray(bytes.slice(pos, pos + 4)) + val txLength = Ints.fromByteArray(bytes.slice(pos, pos + 4).toArray) val txTry = txDeserializer(bytes.slice(pos, pos + txLength)) pos = pos + txLength + @SuppressWarnings(Array("org.wartremover.warts.TraversableOps")) val blockIdOpt: Option[ModifierId] = if (bytes.slice(pos, pos + 1).head == 0) { pos = pos + 1 @@ -67,19 +69,22 @@ object WalletTransaction { o } - val createdAt = Longs.fromByteArray(bytes.slice(pos, pos + 8)) + val createdAt = Longs.fromByteArray(bytes.slice(pos, pos + 8).toArray) WalletTransaction[P, TX](propTry.get, txTry.get, blockIdOpt, createdAt) } - def bytes[P <: Proposition, TX <: Transaction[P]](wt: WalletTransaction[P, TX]): Array[Byte] = { + def bytes[P <: Proposition, TX <: Transaction[P]](wt: WalletTransaction[P, TX]): Seq[Byte] = { val propBytes = wt.proposition.bytes val txBytes = wt.tx.bytes - val bIdBytes = wt.blockId.map(id => Array(1: Byte) ++ id).getOrElse(Array(0: Byte)) + val bIdBytes = wt.blockId.map(id => ByteString(1: Byte) ++ id).getOrElse(ByteString(0: Byte)) - Bytes.concat(Ints.toByteArray(propBytes.length), propBytes, Ints.toByteArray(txBytes.length), txBytes, bIdBytes, - Longs.toByteArray(wt.createdAt)) + ByteString(Ints.toByteArray(propBytes.length)) ++ + propBytes ++ + Ints.toByteArray(txBytes.length) ++ + txBytes ++ bIdBytes ++ + Longs.toByteArray(wt.createdAt) } } diff --git a/src/main/scala/scorex/core/utils/utils.scala b/src/main/scala/scorex/core/utils/utils.scala index 8fd6b672e..7d1582b9d 100644 --- a/src/main/scala/scorex/core/utils/utils.scala +++ b/src/main/scala/scorex/core/utils/utils.scala @@ -44,6 +44,7 @@ package object utils { r } + @deprecated("","") def concatBytes(seq: Traversable[Array[Byte]]): Array[Byte] = { val length: Int = seq.map(_.length).sum val result: Array[Byte] = new Array[Byte](length)