1
0
mirror of https://github.com/ACINQ/eclair.git synced 2024-11-20 02:27:32 +01:00

TLV improvements and full spec compatibility (#1069)

This PR adds support for truncated integers as defined in the spec.
The test vectors are updated to include all test vectors from rusty's spec PR.
It also provides many changes to the tlv and tlv stream classes:

- The tlv trait doesn't need a type field, the codec should handle that
- A TLV stream should be scoped to a specific subtrait of tlv
- Stream validation is done inside the codec instead of the tlv stream: it makes it more convenient for application layers to create tlv streams and manipulate them
This commit is contained in:
Bastien Teinturier 2019-07-11 17:25:14 +02:00 committed by GitHub
parent 5f4a2ebf3e
commit 1621e393dd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 349 additions and 141 deletions

View File

@ -18,14 +18,15 @@ package fr.acinq.eclair.wire
import java.net.{Inet4Address, Inet6Address, InetAddress}
import fr.acinq.bitcoin.{ByteVector32, ByteVector64}
import fr.acinq.bitcoin.Crypto.{PrivateKey, PublicKey}
import fr.acinq.bitcoin.{ByteVector32, ByteVector64}
import fr.acinq.eclair.{ShortChannelId, UInt64}
import org.apache.commons.codec.binary.Base32
import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound}
import scodec.bits.{BitVector, ByteVector}
import scodec.codecs._
import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound}
import scala.Ordering.Implicits._
import scala.util.Try
/**
@ -56,14 +57,15 @@ object CommonCodecs {
val uint64L: Codec[UInt64] = bytes(8).xmap(b => UInt64(b.reverse), a => a.toByteVector.padLeft(8).reverse)
/**
* We impose a minimal encoding on varint values to ensure that signed hashes can be reproduced easily.
* We impose a minimal encoding on some values (such as varint and truncated int) to ensure that signed hashes can be
* re-computed correctly.
* If a value could be encoded with less bytes, it's considered invalid and results in a failed decoding attempt.
*
* @param codec the integer codec (depends on the value).
* @param codec the value codec (depends on the value).
* @param min the minimal value that should be encoded.
*/
def uint64min(codec: Codec[UInt64], min: UInt64): Codec[UInt64] = codec.exmap({
case i if i < min => Attempt.failure(Err("varint was not minimally encoded"))
def minimalvalue[A : Ordering](codec: Codec[A], min: A): Codec[A] = codec.exmap({
case i if i < min => Attempt.failure(Err("value was not minimally encoded"))
case i => Attempt.successful(i)
}, Attempt.successful)
@ -71,9 +73,9 @@ object CommonCodecs {
// See https://bitcoin.org/en/developer-reference#compactsize-unsigned-integers for reference.
val varint: Codec[UInt64] = discriminatorWithDefault(
discriminated[UInt64].by(uint8L)
.\(0xff) { case i if i >= UInt64(0x100000000L) => i }(uint64min(uint64L, UInt64(0x100000000L)))
.\(0xfe) { case i if i >= UInt64(0x10000) => i }(uint64min(uint32L.xmap(UInt64(_), _.toBigInt.toLong), UInt64(0x10000)))
.\(0xfd) { case i if i >= UInt64(0xfd) => i }(uint64min(uint16L.xmap(UInt64(_), _.toBigInt.toInt), UInt64(0xfd))),
.\(0xff) { case i if i >= UInt64(0x100000000L) => i }(minimalvalue(uint64L, UInt64(0x100000000L)))
.\(0xfe) { case i if i >= UInt64(0x10000) => i }(minimalvalue(uint32L.xmap(UInt64(_), _.toBigInt.toLong), UInt64(0x10000)))
.\(0xfd) { case i if i >= UInt64(0xfd) => i }(minimalvalue(uint16L.xmap(UInt64(_), _.toBigInt.toInt), UInt64(0xfd))),
uint8L.xmap(UInt64(_), _.toBigInt.toInt)
)

View File

@ -16,11 +16,11 @@
package fr.acinq.eclair.wire
import fr.acinq.eclair.UInt64
import fr.acinq.eclair.UInt64.Conversions._
import fr.acinq.eclair.wire.CommonCodecs._
import scodec.{Attempt, Codec}
import scodec.codecs._
import scala.util.Try
import scodec.{Attempt, Codec, Err}
/**
* Created by t-bast on 20/06/2019.
@ -28,31 +28,94 @@ import scala.util.Try
object TlvCodecs {
private val genericTlv: Codec[GenericTlv] = (("type" | varint) :: variableSizeBytesLong(varintoverflow, bytes)).as[GenericTlv]
private def variableSizeUInt64(size: Int, min: UInt64): Codec[UInt64] = minimalvalue(bytes(size).xmap(UInt64(_), _.toByteVector.takeRight(size)), min)
private def tlvFallback(codec: Codec[Tlv]): Codec[Tlv] = discriminatorFallback(genericTlv, codec).xmap({
case Left(l) => l
case Right(r) => r
}, {
case g: GenericTlv => Left(g)
case o => Right(o)
})
/**
* Length-prefixed truncated uint64 (1 to 9 bytes unsigned integer).
*/
val tu64: Codec[UInt64] = discriminated[UInt64].by(uint8)
.\(0x00) { case i if i < 0x01 => i }(variableSizeUInt64(0, 0x00))
.\(0x01) { case i if i < 0x0100 => i }(variableSizeUInt64(1, 0x01))
.\(0x02) { case i if i < 0x010000 => i }(variableSizeUInt64(2, 0x0100))
.\(0x03) { case i if i < 0x01000000 => i }(variableSizeUInt64(3, 0x010000))
.\(0x04) { case i if i < 0x0100000000L => i }(variableSizeUInt64(4, 0x01000000))
.\(0x05) { case i if i < 0x010000000000L => i }(variableSizeUInt64(5, 0x0100000000L))
.\(0x06) { case i if i < 0x01000000000000L => i }(variableSizeUInt64(6, 0x010000000000L))
.\(0x07) { case i if i < 0x0100000000000000L => i }(variableSizeUInt64(7, 0x01000000000000L))
.\(0x08) { case i if i <= UInt64.MaxValue => i }(variableSizeUInt64(8, 0x0100000000000000L))
/**
* Length-prefixed truncated uint32 (1 to 5 bytes unsigned integer).
*/
val tu32: Codec[Long] = tu64.exmap({
case i if i > 0xffffffffL => Attempt.Failure(Err("tu32 overflow"))
case i => Attempt.Successful(i.toBigInt.toLong)
}, l => Attempt.Successful(l))
/**
* Length-prefixed truncated uint16 (1 to 3 bytes unsigned integer).
*/
val tu16: Codec[Int] = tu32.exmap({
case i if i > 0xffff => Attempt.Failure(Err("tu16 overflow"))
case i => Attempt.Successful(i.toInt)
}, l => Attempt.Successful(l))
private def validateGenericTlv(g: GenericTlv): Attempt[GenericTlv] = {
if (g.tag.toBigInt % 2 == 0) {
Attempt.Failure(Err("unknown even tlv type"))
} else {
Attempt.Successful(g)
}
}
private val genericTlv: Codec[GenericTlv] = (("tag" | varint) :: variableSizeBytesLong(varintoverflow, bytes)).as[GenericTlv].exmap(validateGenericTlv, validateGenericTlv)
private def tag[T <: Tlv](codec: DiscriminatorCodec[T, UInt64], record: Either[GenericTlv, T]): UInt64 = record match {
case Left(generic) => generic.tag
case Right(tlv) => tag(codec, tlv)
}
private def tag[T <: Tlv](codec: DiscriminatorCodec[T, UInt64], record: T): UInt64 =
codec.encode(record).flatMap(bits => varint.decode(bits)).require.value
private def validateStream[T <: Tlv](codec: DiscriminatorCodec[T, UInt64], records: List[Either[GenericTlv, T]]): Attempt[TlvStream[T]] = {
val tags = records.map(r => tag(codec, r))
if (tags.length != tags.distinct.length) {
Attempt.Failure(Err("tlv streams must not contain duplicate records"))
} else if (tags != tags.sorted) {
Attempt.Failure(Err("tlv records must be ordered by monotonically-increasing types"))
} else {
Attempt.Successful(TlvStream(records.collect { case Right(tlv) => tlv }, records.collect { case Left(generic) => generic }))
}
}
/**
* A tlv stream codec relies on an underlying tlv codec.
* This allows tlv streams to have different namespaces, increasing the total number of tlv types available.
*
* @param codec codec used for the tlv records contained in the stream.
* @tparam T stream namespace.
*/
def tlvStream(codec: Codec[Tlv]): Codec[TlvStream] = list(tlvFallback(codec)).exmap(
records => Attempt.fromTry(Try(TlvStream(records))),
stream => Attempt.successful(stream.records.toList)
def tlvStream[T <: Tlv](codec: DiscriminatorCodec[T, UInt64]): Codec[TlvStream[T]] = list(discriminatorFallback(genericTlv, codec)).exmap(
records => validateStream(codec, records),
(stream: TlvStream[T]) => {
val records = (stream.records.map(Right(_)) ++ stream.unknown.map(Left(_))).toList
val tags = records.map(r => tag(codec, r))
if (tags.length != tags.distinct.length) {
Attempt.Failure(Err("tlv streams must not contain duplicate records"))
} else {
Attempt.Successful(tags.zip(records).sortBy(_._1).map(_._2))
}
}
)
/**
* When used inside a message, a tlv stream needs to specify its length.
* When used inside a message, most of the time a tlv stream needs to specify its length.
* Note that some messages will have an independent length field and won't need this codec.
*
* @param codec codec used for the tlv records contained in the stream.
* @tparam T stream namespace.
*/
def lengthPrefixedTlvStream(codec: Codec[Tlv]): Codec[TlvStream] = variableSizeBytesLong(CommonCodecs.varintoverflow, tlvStream(codec))
def lengthPrefixedTlvStream[T <: Tlv](codec: DiscriminatorCodec[T, UInt64]): Codec[TlvStream[T]] = variableSizeBytesLong(CommonCodecs.varintoverflow, tlvStream(codec))
}

View File

@ -19,45 +19,36 @@ package fr.acinq.eclair.wire
import fr.acinq.eclair.UInt64
import scodec.bits.ByteVector
import scala.annotation.tailrec
/**
* Created by t-bast on 20/06/2019.
*/
// @formatter:off
trait Tlv {
val `type`: UInt64
}
trait Tlv
sealed trait OnionTlv extends Tlv
// @formatter:on
/**
* Generic tlv type we fallback to if we don't understand the incoming type.
* Generic tlv type we fallback to if we don't understand the incoming tlv.
*
* @param `type` tlv type.
* @param value tlv value (length is implicit, and encoded as a varint).
* @param tag tlv tag.
* @param value tlv value (length is implicit, and encoded as a varint).
*/
case class GenericTlv(`type`: UInt64, value: ByteVector) extends Tlv
case class GenericTlv(tag: UInt64, value: ByteVector) extends Tlv
/**
* A tlv stream is a collection of tlv records.
* A tlv stream is part of a given namespace that dictates how to parse the tlv records.
* That namespace is indicated by a trait extending the top-level tlv trait.
* A tlv stream is constrained to a specific tlv namespace that dictates how to parse the tlv records.
* That namespace is provided by a trait extending the top-level tlv trait.
*
* @param records tlv records.
* @param records known tlv records.
* @param unknown unknown tlv records.
* @tparam T the stream namespace is a trait extending the top-level tlv trait.
*/
case class TlvStream(records: Seq[Tlv]) {
case class TlvStream[T <: Tlv](records: Traversable[T], unknown: Traversable[GenericTlv] = Nil)
records.foldLeft(Option.empty[Tlv]) {
case (None, record) =>
require(!record.isInstanceOf[GenericTlv] || record.`type`.toBigInt % 2 != 0, "tlv streams must not contain unknown even tlv types")
Some(record)
case (Some(previousRecord), record) =>
require(record.`type` != previousRecord.`type`, "tlv streams must not contain duplicate records")
require(record.`type` > previousRecord.`type`, "tlv records must be ordered by monotonically-increasing types")
require(!record.isInstanceOf[GenericTlv] || record.`type`.toBigInt % 2 != 0, "tlv streams must not contain unknown even tlv types")
Some(record)
}
object TlvStream {
def apply[T <: Tlv](records: T*): TlvStream[T] = TlvStream(records, Nil)
}

View File

@ -20,8 +20,8 @@ import java.net.{Inet4Address, Inet6Address, InetAddress}
import com.google.common.net.InetAddresses
import fr.acinq.bitcoin.Crypto.PrivateKey
import fr.acinq.eclair.{UInt64, randomBytes32}
import fr.acinq.eclair.wire.CommonCodecs._
import fr.acinq.eclair.{UInt64, randomBytes32}
import org.scalatest.FunSuite
import scodec.bits.{BitVector, HexStringSyntax}

View File

@ -17,14 +17,14 @@
package fr.acinq.eclair.wire
import fr.acinq.bitcoin.Crypto.PublicKey
import fr.acinq.eclair.{ShortChannelId, UInt64}
import fr.acinq.eclair.UInt64.Conversions._
import fr.acinq.eclair.wire.CommonCodecs.{publicKey, shortchannelid, uint64, varint}
import fr.acinq.eclair.wire.TlvCodecs._
import fr.acinq.eclair.{ShortChannelId, UInt64}
import org.scalatest.FunSuite
import scodec.Codec
import scodec.bits.HexStringSyntax
import scodec.codecs._
import scodec.Codec
/**
* Created by t-bast on 20/06/2019.
@ -34,117 +34,263 @@ class TlvCodecsSpec extends FunSuite {
import TlvCodecsSpec._
test("encode/decode tlv") {
test("encode/decode truncated uint16") {
val testCases = Seq(
(hex"01 08 000000000000002a", TestType1(42)),
(hex"02 08 0000000000000226", TestType2(ShortChannelId(550))),
(hex"03 31 02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619 0000000000000231 0000000000000451", TestType3(PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 561, 1105))
(hex"00", 0),
(hex"01 01", 1),
(hex"01 2a", 42),
(hex"01 ff", 255),
(hex"02 0100", 256),
(hex"02 0231", 561),
(hex"02 ffff", 65535)
)
for ((bin, expected) <- testCases) {
val decoded = testTlvCodec.decode(bin.bits).require.value.asInstanceOf[Tlv]
val decoded = tu16.decode(bin.bits).require.value
assert(decoded === expected)
val encoded = testTlvCodec.encode(expected).require.bytes
val encoded = tu16.encode(expected).require.bytes
assert(encoded === bin)
}
}
test("decode invalid tlv") {
test("encode/decode truncated uint32") {
val testCases = Seq(
hex"fd02", // type truncated
hex"fd022a", // truncated after type
hex"fd0100", // not minimally encoded type
hex"2a fd02", // length truncated
hex"2a fd0226", // truncated after length
hex"2a fe01010000", // not minimally encoded length
hex"2a fd2602 0231", // value truncated
hex"02 01 2a", // short channel id too short
hex"02 09 010101010101010101", // short channel id length too big
hex"2a ff0000000000000080" // invalid length (too big to fit inside a long)
(hex"00", 0L),
(hex"01 01", 1L),
(hex"01 2a", 42L),
(hex"01 ff", 255L),
(hex"02 0100", 256L),
(hex"02 0231", 561L),
(hex"02 ffff", 65535L),
(hex"03 010000", 65536L),
(hex"03 ffffff", 16777215L),
(hex"04 01000000", 16777216L),
(hex"04 01020304", 16909060L),
(hex"04 ffffffff", 4294967295L)
)
for (testCase <- testCases) {
assert(testTlvCodec.decode(testCase.bits).isFailure)
for ((bin, expected) <- testCases) {
val decoded = tu32.decode(bin.bits).require.value
assert(decoded === expected)
val encoded = tu32.encode(expected).require.bytes
assert(encoded === bin)
}
}
test("encode/decode truncated uint64") {
val testCases = Seq(
(hex"00", UInt64(0)),
(hex"01 01", UInt64(1)),
(hex"01 2a", UInt64(42)),
(hex"01 ff", UInt64(255)),
(hex"02 0100", UInt64(256)),
(hex"02 0231", UInt64(561)),
(hex"02 ffff", UInt64(65535)),
(hex"03 010000", UInt64(65536)),
(hex"03 ffffff", UInt64(16777215)),
(hex"04 01000000", UInt64(16777216)),
(hex"04 01020304", UInt64(16909060)),
(hex"04 ffffffff", UInt64(4294967295L)),
(hex"05 0100000000", UInt64(4294967296L)),
(hex"05 0102030405", UInt64(4328719365L)),
(hex"05 ffffffffff", UInt64(1099511627775L)),
(hex"06 010000000000", UInt64(1099511627776L)),
(hex"06 010203040506", UInt64(1108152157446L)),
(hex"06 ffffffffffff", UInt64(281474976710655L)),
(hex"07 01000000000000", UInt64(281474976710656L)),
(hex"07 01020304050607", UInt64(283686952306183L)),
(hex"07 ffffffffffffff", UInt64(72057594037927935L)),
(hex"08 0100000000000000", UInt64(72057594037927936L)),
(hex"08 0102030405060708", UInt64(72623859790382856L)),
(hex"08 ffffffffffffffff", UInt64.MaxValue)
)
for ((bin, expected) <- testCases) {
val decoded = tu64.decode(bin.bits).require.value
assert(decoded === expected)
val encoded = tu64.encode(expected).require.bytes
assert(encoded === bin)
}
}
test("decode invalid truncated integers") {
val testCases = Seq(
(tu16, hex"01 00"), // not minimal
(tu16, hex"02 0001"), // not minimal
(tu16, hex"03 ffffff"), // length too big
(tu32, hex"01 00"), // not minimal
(tu32, hex"02 0001"), // not minimal
(tu32, hex"03 000100"), // not minimal
(tu32, hex"04 00010000"), // not minimal
(tu32, hex"05 ffffffffff"), // length too big
(tu64, hex"01 00"), // not minimal
(tu64, hex"02 0001"), // not minimal
(tu64, hex"03 000100"), // not minimal
(tu64, hex"04 00010000"), // not minimal
(tu64, hex"05 0001000000"), // not minimal
(tu64, hex"06 000100000000"), // not minimal
(tu64, hex"07 00010000000000"), // not minimal
(tu64, hex"08 0001000000000000"), // not minimal
(tu64, hex"09 ffffffffffffffffff") // length too big
)
for ((codec, bin) <- testCases) {
assert(codec.decode(bin.bits).isFailure, bin)
}
}
test("encode/decode tlv stream") {
val testCases = Seq(
(hex"", TlvStream[TestTlv]()),
(hex"21 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(33, hex"")))),
(hex"fd0102 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(513, hex"")))),
(hex"fdfd00 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(253, hex"")))),
(hex"fdff00 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(255, hex"")))),
(hex"fe01000002 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(33554433, hex"")))),
(hex"ff0100000000000002 00", TlvStream[TestTlv](Nil, Seq(GenericTlv(144115188075855873L, hex"")))),
(hex"01 00", TlvStream[TestTlv](TestType1(0))),
(hex"01 01 01", TlvStream[TestTlv](TestType1(1))),
(hex"01 01 2a", TlvStream[TestTlv](TestType1(42))),
(hex"01 02 0100", TlvStream[TestTlv](TestType1(256))),
(hex"01 03 010000", TlvStream[TestTlv](TestType1(65536))),
(hex"01 04 01000000", TlvStream[TestTlv](TestType1(16777216))),
(hex"01 05 0100000000", TlvStream[TestTlv](TestType1(4294967296L))),
(hex"01 06 010000000000", TlvStream[TestTlv](TestType1(1099511627776L))),
(hex"01 07 01000000000000", TlvStream[TestTlv](TestType1(281474976710656L))),
(hex"01 08 0100000000000000", TlvStream[TestTlv](TestType1(72057594037927936L))),
(hex"02 08 0000000000000226", TlvStream[TestTlv](TestType2(ShortChannelId(550)))),
(hex"03 31 023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb 0000000000000231 0000000000000451", TlvStream[TestTlv](TestType3(PublicKey(hex"023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb"), 561, 1105))),
(hex"fdfe00 02 0226", TlvStream[TestTlv](TestType254(550))),
(hex"01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451", TlvStream[TestTlv](TestType1(561), TestType2(ShortChannelId(1105)), TestType3(PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 561, 1105))),
(hex"01020231 0b020451 fdfe0002002a", TlvStream[TestTlv](Seq(TestType1(561), TestType254(42)), Seq(GenericTlv(11, hex"0451"))))
)
for ((bin, expected) <- testCases) {
val decoded = testTlvStreamCodec.decode(bin.bits).require.value
assert(decoded === expected)
val encoded = testTlvStreamCodec.encode(expected).require.bytes
assert(encoded === bin)
}
}
test("decode invalid tlv stream") {
val testCases = Seq(
hex"0108000000000000002a 02", // valid tlv record followed by invalid tlv record (only type, length and value are missing)
hex"02080000000000000226 0108000000000000002a", // valid tlv records but invalid ordering
// Type truncated.
hex"fd",
hex"fd01",
// Not minimally encoded type.
hex"fd0100 00",
// Missing length.
hex"fd0101",
// Length truncated.
hex"0f fd",
hex"0f fd02",
// Not minimally encoded length.
hex"0f fd0100 00",
hex"0f fe01000000 00",
// Missing value.
hex"0f fd0226",
// Value truncated.
hex"0f fd0102 000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
// Unknown even type.
hex"12 00",
hex"0a 00",
hex"fd0201 00",
hex"fe02000001 00",
hex"01020101 0a0101",
hex"ff0200000000000001 00",
// Invalid TestTlv1.
hex"01 01 00", // not minimally-encoded
hex"01 02 0001", // not minimally-encoded
hex"01 03 000100", // not minimally-encoded
hex"01 04 00010000", // not minimally-encoded
hex"01 05 0001000000", // not minimally-encoded
hex"01 06 000100000000", // not minimally-encoded
hex"01 07 00010000000000", // not minimally-encoded
hex"01 08 0001000000000000", // not minimally-encoded
// Invalid TestTlv2.
hex"02 07 01010101010101", // invalid length
hex"02 09 010101010101010101", // invalid length
// Invalid TestTlv3.
hex"03 21 023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb", // invalid length
hex"03 29 023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb0000000000000001", // invalid length
hex"03 30 023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb000000000000000100000000000001", // invalid length
hex"03 32 023da092f6980e58d2c037173180e9a465476026ee50f96695963e8efe436f54eb0000000000000001000000000000000001", // invalid length
// Invalid TestTlv254.
hex"fdfe00 00", // invalid length
hex"fdfe00 01 01", // invalid length
hex"fdfe00 03 010101", // invalid length
// Invalid multi-record streams.
hex"01012a 02", // valid tlv record followed by invalid tlv record (length missing)
hex"01012a 0208", // valid tlv record followed by invalid tlv record (value missing)
hex"01012a 020801010101", // valid tlv record followed by invalid tlv record (value truncated)
hex"02080000000000000226 01012a", // valid tlv records but invalid ordering
hex"1f00 0f012a", // valid tlv records but invalid ordering
hex"02080000000000000231 02080000000000000451", // duplicate tlv type
hex"0108000000000000002a 2a0101", // unknown even type
hex"0a080000000000000231 0b0400000451" // valid tlv records but from different namespace
hex"01012a 0b020231 0b020451", // duplicate tlv type
hex"1f00 1f012a", // duplicate tlv type
hex"01012a 0a020231 0b020451" // valid tlv records but from different namespace
)
for (testCase <- testCases) {
assert(tlvStream(testTlvCodec).decode(testCase.bits).isFailure, testCase)
assert(testTlvStreamCodec.decode(testCase.bits).isFailure, testCase)
}
}
test("create invalid tlv stream") {
assertThrows[IllegalArgumentException](TlvStream(Seq(GenericTlv(42, hex"2a")))) // unknown even type
assertThrows[IllegalArgumentException](TlvStream(Seq(TestType1(561), TestType2(ShortChannelId(1105)), GenericTlv(42, hex"2a")))) // unknown even type
assertThrows[IllegalArgumentException](TlvStream(Seq(TestType1(561), TestType1(1105)))) // duplicate type
assertThrows[IllegalArgumentException](TlvStream(Seq(TestType2(ShortChannelId(1105)), TestType1(561)))) // invalid ordering
}
test("encoded/decode empty tlv stream") {
assert(tlvStream(testTlvCodec).decode(hex"".bits).require.value === TlvStream(Nil))
assert(tlvStream(testTlvCodec).encode(TlvStream(Nil)).require.bytes === hex"")
}
test("encode/decode tlv stream") {
val bin = hex"01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451"
val expected = Seq(
TestType1(561),
TestType2(ShortChannelId(1105)),
TestType3(PublicKey(hex"02eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f283686619"), 561, 1105)
)
val decoded = tlvStream(testTlvCodec).decode(bin.bits).require.value
assert(decoded === TlvStream(expected))
val encoded = tlvStream(testTlvCodec).encode(TlvStream(expected)).require.bytes
assert(encoded === bin)
}
test("encode/decode tlv stream with unknown odd type") {
val bin = hex"01080000000000000231 0b0400000451 0d02002a"
val expected = Seq(
TestType1(561),
GenericTlv(11, hex"00000451"),
TestType13(42)
)
val decoded = tlvStream(testTlvCodec).decode(bin.bits).require.value
assert(decoded === TlvStream(expected))
val encoded = tlvStream(testTlvCodec).encode(TlvStream(expected)).require.bytes
assert(encoded === bin)
}
test("encode/decode length-prefixed tlv stream") {
val codec = lengthPrefixedTlvStream(testTlvCodec)
val testCases = Seq(
hex"47 01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
hex"fd5301 01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451 ff6543210987654321 fd0001 10101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010010101010101"
hex"41 01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
hex"fd4d01 01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451 ff6543210987654321 fd0001 10101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010101010010101010101"
)
for (testCase <- testCases) {
assert(codec.encode(codec.decode(testCase.bits).require.value).require.bytes === testCase)
assert(lengthPrefixedTestTlvStreamCodec.encode(lengthPrefixedTestTlvStreamCodec.decode(testCase.bits).require.value).require.bytes === testCase)
}
}
test("decode invalid length-prefixed tlv stream") {
val testCases = Seq(
hex"48 01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
hex"46 01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
hex"01080000000000000231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451"
// Length too big.
hex"42 01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
// Length too short.
hex"40 01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
// Missing length.
hex"01020231 02080000000000000451 033102eec7245d6b7d2ccb30380bfbe2a3648cd7a942653f5aa340edcea1f28368661900000000000002310000000000000451",
// Valid length but duplicate types.
hex"14 02080000000000000231 02080000000000000451",
// Valid length but invalid ordering.
hex"0e 02080000000000000451 01020231",
// Valid length but unknown even type.
hex"02 0a 00"
)
for (testCase <- testCases) {
assert(lengthPrefixedTlvStream(testTlvCodec).decode(testCase.bits).isFailure)
assert(lengthPrefixedTestTlvStreamCodec.decode(testCase.bits).isFailure)
}
}
test("encode unordered tlv stream (codec should sort appropriately)") {
val stream = TlvStream[TestTlv](Seq(TestType254(42), TestType1(42)), Seq(GenericTlv(13, hex"2a"), GenericTlv(11, hex"2b")))
assert(testTlvStreamCodec.encode(stream).require.toByteVector === hex"01012a 0b012b 0d012a fdfe0002002a")
assert(lengthPrefixedTestTlvStreamCodec.encode(stream).require.toByteVector === hex"0f 01012a 0b012b 0d012a fdfe0002002a")
}
test("encode invalid tlv stream") {
val testCases = Seq(
// Unknown even type.
TlvStream[TestTlv](Nil, Seq(GenericTlv(42, hex"2a"))),
TlvStream[TestTlv](Seq(TestType1(561), TestType2(ShortChannelId(1105))), Seq(GenericTlv(42, hex"2a"))),
// Duplicate type.
TlvStream[TestTlv](TestType1(561), TestType1(1105)),
TlvStream[TestTlv](Seq(TestType1(561)), Seq(GenericTlv(1, hex"0451")))
)
for (stream <- testCases) {
assert(testTlvStreamCodec.encode(stream).isFailure, stream)
assert(lengthPrefixedTestTlvStreamCodec.encode(stream).isFailure, stream)
}
}
@ -152,32 +298,38 @@ class TlvCodecsSpec extends FunSuite {
object TlvCodecsSpec {
// See https://github.com/lightningnetwork/lightning-rfc/blob/master/01-messaging.md#appendix-a-type-length-value-test-vectors
// @formatter:off
sealed trait TestTlv extends Tlv
case class TestType1(uintValue: UInt64) extends TestTlv { override val `type` = UInt64(1) }
case class TestType2(shortChannelId: ShortChannelId) extends TestTlv { override val `type` = UInt64(2) }
case class TestType3(nodeId: PublicKey, value1: UInt64, value2: UInt64) extends TestTlv { override val `type` = UInt64(3) }
case class TestType13(intValue: Int) extends TestTlv { override val `type` = UInt64(13) }
case class TestType1(uintValue: UInt64) extends TestTlv
case class TestType2(shortChannelId: ShortChannelId) extends TestTlv
case class TestType3(nodeId: PublicKey, value1: UInt64, value2: UInt64) extends TestTlv
case class TestType254(intValue: Int) extends TestTlv
val testCodec1: Codec[TestType1] = (("length" | constant(hex"08")) :: ("value" | uint64)).as[TestType1]
val testCodec2: Codec[TestType2] = (("length" | constant(hex"08")) :: ("short_channel_id" | shortchannelid)).as[TestType2]
val testCodec3: Codec[TestType3] = (("length" | constant(hex"31")) :: ("node_id" | publicKey) :: ("value_1" | uint64) :: ("value_2" | uint64)).as[TestType3]
val testCodec13: Codec[TestType13] = (("length" | constant(hex"02")) :: ("value" | uint16)).as[TestType13]
val testTlvCodec = discriminated[Tlv].by(varint)
private val testCodec1: Codec[TestType1] = ("value" | tu64).as[TestType1]
private val testCodec2: Codec[TestType2] = (("length" | constant(hex"08")) :: ("short_channel_id" | shortchannelid)).as[TestType2]
private val testCodec3: Codec[TestType3] = (("length" | constant(hex"31")) :: ("node_id" | publicKey) :: ("value_1" | uint64) :: ("value_2" | uint64)).as[TestType3]
private val testCodec254: Codec[TestType254] = (("length" | constant(hex"02")) :: ("value" | uint16)).as[TestType254]
private val testTlvCodec = discriminated[TestTlv].by(varint)
.typecase(1, testCodec1)
.typecase(2, testCodec2)
.typecase(3, testCodec3)
.typecase(13, testCodec13)
.typecase(254, testCodec254)
val testTlvStreamCodec = tlvStream(testTlvCodec)
val lengthPrefixedTestTlvStreamCodec = lengthPrefixedTlvStream(testTlvCodec)
sealed trait OtherTlv extends Tlv
case class OtherType1(uintValue: UInt64) extends OtherTlv { override val `type` = UInt64(10) }
case class OtherType2(smallValue: Long) extends OtherTlv { override val `type` = UInt64(11) }
case class OtherType1(uintValue: UInt64) extends OtherTlv
case class OtherType2(smallValue: Long) extends OtherTlv
val otherCodec1: Codec[OtherType1] = (("length" | constant(hex"08")) :: ("value" | uint64)).as[OtherType1]
val otherCodec2: Codec[OtherType2] = (("length" | constant(hex"04")) :: ("value" | uint32)).as[OtherType2]
val otherTlvCodec = discriminated[Tlv].by(varint)
val otherCodec1: Codec[OtherType1] = ("value" | tu64).as[OtherType1]
val otherCodec2: Codec[OtherType2] = ("value" | tu32).as[OtherType2]
val otherTlvStreamCodec = tlvStream(discriminated[OtherTlv].by(varint)
.typecase(10, otherCodec1)
.typecase(11, otherCodec2)
.typecase(11, otherCodec2))
// @formatter:on
}