From 230505c6b57043ab2c9c7eb4b8c4e1c21577708f Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 3 Sep 2021 08:11:29 -0400 Subject: [PATCH 01/33] Initial import of time series from scodec-protocols --- core/shared/src/main/scala/fs2/Scan.scala | 181 +++++++++ .../scala/fs2/timeseries/TimeSeriesPipe.scala | 39 ++ .../fs2/timeseries/TimeSeriesValue.scala | 39 ++ .../main/scala/fs2/timeseries/TimeStamp.scala | 40 ++ .../scala/fs2/timeseries/TimeStamped.scala | 353 ++++++++++++++++++ .../main/scala/fs2/timeseries/package.scala | 48 +++ 6 files changed, 700 insertions(+) create mode 100644 core/shared/src/main/scala/fs2/Scan.scala create mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala create mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala create mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala create mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala create mode 100644 core/shared/src/main/scala/fs2/timeseries/package.scala diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala new file mode 100644 index 0000000000..35dff512b1 --- /dev/null +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -0,0 +1,181 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD +package fs2 + +import cats.data.AndThen + +final class Scan[S, -I, +O]( + val initial: S, + private val transform_ : AndThen[(S, I), (S, Chunk[O])], + private val onComplete_ : AndThen[S, Chunk[O]] +) { + + def transform(s: S, i: I): (S, Chunk[O]) = transform_((s, i)) + + def transformAccumulate(s: S, c: Chunk[I]): (S, Chunk[O]) = + // c.traverse(i => State(transform(_, i))).map(_.flatten).run(s).value + c.foldLeft(s -> Chunk.empty[O]) { case ((s, acc), i) => + val (s2, os) = transform(s, i) + (s2, acc ++ os) + } + + def onComplete(s: S): Chunk[O] = onComplete_(s) + + def toPipe[F[_]]: Pipe[F, I, O] = + _.pull + .scanChunks(initial)(transformAccumulate) + .flatMap(state => Pull.output(onComplete(state))) + .stream + + def step(i: I): (Scan[S, I, O], Chunk[O]) = { + val (s, os) = transform(initial, i) + (new Scan(s, transform_, onComplete_), os) + } + + def andThen[S2, O2](t: Scan[S2, O, O2]): Scan[(S, S2), I, O2] = + Scan[(S, S2), I, O2]((initial, t.initial))( + { case ((s, s2), i) => + val (sp, os) = transform(s, i) + val (s2p, out) = t.transformAccumulate(s2, os) + ((sp, s2p), out) + }, + { case (s, s2) => + val (s3, out) = t.transformAccumulate(s2, onComplete(s)) + out ++ t.onComplete(s3) + } + ) + + def map[O2](f: O => O2): Scan[S, I, O2] = + new Scan( + initial, + transform_.andThen[(S, Chunk[O2])] { case (s, os) => (s, os.map(f)) }, + onComplete_.andThen(_.map(f)) + ) + + def contramap[I2](f: I2 => I): Scan[S, I2, O] = + new Scan( + initial, + AndThen[(S, I2), (S, I)] { case (s, i2) => (s, f(i2)) }.andThen(transform_), + onComplete_ + ) + + def imapState[S2](g: S => S2)(f: S2 => S): Scan[S2, I, O] = + Scan[S2, I, O](g(initial))( + { (s2, i) => + val (s3, os) = transform(f(s2), i) + (g(s3), os) + }, + AndThen(f).andThen(onComplete_) + ) + + def lens[I2, O2](get: I2 => I, set: (I2, O) => O2): Scan[S, I2, O2] = + Scan[S, I2, O2](initial)( + { (s, i2) => + val (s2, os) = transform(s, get(i2)) + (s2, os.map(s => set(i2, s))) + }, + _ => Chunk.empty + ) + + def first[A]: Scan[S, (I, A), (O, A)] = + lens(_._1, (t, o) => (o, t._2)) + + def second[A]: Scan[S, (A, I), (A, O)] = + lens(_._2, (t, o) => (t._1, o)) + + def semilens[I2, O2](extract: I2 => Either[O2, I], inject: (I2, O) => O2): Scan[S, I2, O2] = + Scan[S, I2, O2](initial)( + (s, i2) => + extract(i2).fold( + o2 => s -> Chunk.singleton(o2), + i => { + val (s2, os) = transform(s, i) + (s2, os.map(o => inject(i2, o))) + } + ), + _ => Chunk.empty + ) + + def semipass[I2, O2 >: O](extract: I2 => Either[O2, I]): Scan[S, I2, O2] = + semilens(extract, (_, o) => o) + + def left[A]: Scan[S, Either[I, A], Either[O, A]] = + semilens(_.fold(i => Right(i), a => Left(Right(a))), (_, o) => Left(o)) + + def right[A]: Scan[S, Either[A, I], Either[A, O]] = + semilens(_.fold(a => Left(Left(a)), i => Right(i)), (_, o) => Right(o)) + + def or[S2, I2, O2 >: O](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], O2] = + Scan[(S, S2), Either[I, I2], O2]((initial, t.initial))( + { case ((s, s2), e) => + e match { + case Left(i) => + val (sp, os) = transform(s, i) + ((sp, s2), os) + case Right(i2) => + val (s2p, o2s) = t.transform(s2, i2) + ((s, s2p), o2s) + } + }, + { case (s, s2) => onComplete(s) ++ t.onComplete(s2) } + ) + + def either[S2, I2, O2](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], Either[O, O2]] = + Scan[(S, S2), Either[I, I2], Either[O, O2]]((initial, t.initial))( + { case ((s, s2), e) => + e match { + case Left(i) => + val (sp, os) = transform(s, i) + ((sp, s2), os.map(Left(_))) + case Right(i2) => + val (s2p, o2s) = t.transform(s2, i2) + ((s, s2p), o2s.map(Right(_))) + } + }, + { case (s, s2) => onComplete(s).map(Left(_)) ++ t.onComplete(s2).map(Right(_)) } + ) +} + +object Scan { + // def apply[S, I, O](initial: S, transform: AndThen[(S, I), (S, Chunk[O])], onComplete: AndThen[S, Chunk[O]]): Scan[S,I,O] = + // new Scan[S,I,O](initial, transform, onComplete) + + def apply[S, I, O]( + initial: S + )(transform: (S, I) => (S, Chunk[O]), onComplete: S => Chunk[O]): Scan[S, I, O] = + new Scan(initial, AndThen { case (s, i) => transform(s, i) }, AndThen(onComplete)) + + def stateful[S, I, O](initial: S)(transform: (S, I) => (S, Chunk[O])): Scan[S, I, O] = + apply(initial)(transform, _ => Chunk.empty) + + def stateful1[S, I, O](initial: S)(f: (S, I) => (S, O)): Scan[S, I, O] = + stateful[S, I, O](initial) { (s, i) => + val (s2, o) = f(s, i); s2 -> Chunk.singleton(o) + } + + def stateless[I, O](f: I => Chunk[O]): Scan[Unit, I, O] = + stateful[Unit, I, O](())((u, i) => (u, f(i))) + + def lift[I, O](f: I => O): Scan[Unit, I, O] = + stateless(i => Chunk.singleton(f(i))) +} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala new file mode 100644 index 0000000000..8d9dbc1981 --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD +package fs2.timeseries + +object TimeSeriesPipe { + + def lift[F[_], A, B](f: A => B): TimeSeriesPipe[F, A, B] = + _.map(_.map(_.map(f))) + + def drainRight[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], L] = _.collect { + case tick @ TimeStamped(ts, None) => tick.asInstanceOf[TimeSeriesValue[L]] + case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) + } + + def drainLeft[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], R] = _.collect { + case tick @ TimeStamped(ts, None) => tick.asInstanceOf[TimeSeriesValue[R]] + case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) + } +} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala new file mode 100644 index 0000000000..4b6d60295a --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD +package fs2.timeseries + +import cats.Functor +import cats.effect.kernel.Clock + +object TimeSeriesValue { + def apply[A](time: TimeStamp, value: A): TimeSeriesValue[A] = TimeStamped(time, Some(value)) + def tick(time: TimeStamp): TimeSeriesValue[Nothing] = TimeStamped(time, None) + + def unsafeNow[A](value: A): TimeSeriesValue[A] = TimeStamped.unsafeNow(Some(value)) + def now[F[_]: Functor: Clock, A](value: A): F[TimeSeriesValue[A]] = TimeStamped.now(Some(value)) + + def unsafeNowTick: TimeSeriesValue[Nothing] = TimeStamped.unsafeNow(None) + def nowTick[F[_]: Functor: Clock]: F[TimeSeriesValue[Nothing]] = TimeStamped.now(None) + + def lift[A](t: TimeStamped[A]): TimeSeriesValue[A] = t.map(Some.apply) +} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala new file mode 100644 index 0000000000..cb88d01be2 --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD +package fs2.timeseries + +import cats.Functor +import cats.syntax.all._ +import cats.effect.kernel.Clock + +final case class TimeStamp(toEpochMilli: Long) extends Ordered[TimeStamp] { + def +(millis: Long): TimeStamp = TimeStamp(toEpochMilli + millis) + def isBefore(that: TimeStamp): Boolean = toEpochMilli < that.toEpochMilli + def compare(that: TimeStamp): Int = toEpochMilli.compareTo(that.toEpochMilli) +} + +object TimeStamp { + def unsafeNow(): TimeStamp = TimeStamp(System.currentTimeMillis()) + def now[F[_]: Functor: Clock]: F[TimeStamp] = Clock[F].realTime.map(d => TimeStamp(d.toMillis)) + +// implicit val ordering: Ordering[TimeStamp] = (x, y) => x.toEpochMilli.compare(y.toEpochMilli) +} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala new file mode 100644 index 0000000000..129fd1f0a2 --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -0,0 +1,353 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD +package fs2 +package timeseries + +import scala.concurrent.duration._ + +import cats.{Functor, Monoid} +import cats.data.Chain +import cats.syntax.all._ +import cats.effect.kernel.{Clock, Temporal} + +/** Wrapper that associates a time with a value. */ +case class TimeStamped[+A](time: TimeStamp, value: A) { + def map[B](f: A => B): TimeStamped[B] = copy(value = f(value)) + def mapTime(f: TimeStamp => TimeStamp): TimeStamped[A] = copy(time = f(time)) + + def toTimeSeriesValue: TimeSeriesValue[A] = map(Some.apply) +} + +object TimeStamped { + + def unsafeNow[A](a: A): TimeStamped[A] = TimeStamped(TimeStamp.unsafeNow(), a) + + def now[F[_]: Functor: Clock, A](a: A): F[TimeStamped[A]] = + TimeStamp.now[F].map(t => TimeStamped(t, a)) + + /** Orders values by timestamp -- values with the same timestamp are considered equal. */ + def timeBasedOrdering[A]: Ordering[TimeStamped[A]] = new Ordering[TimeStamped[A]] { + def compare(x: TimeStamped[A], y: TimeStamped[A]) = x.time.compareTo(y.time) + } + + /** Orders values by timestamp, then by value. */ + implicit def ordering[A](implicit A: Ordering[A]): Ordering[TimeStamped[A]] = + new Ordering[TimeStamped[A]] { + def compare(x: TimeStamped[A], y: TimeStamped[A]) = x.time.compareTo(y.time) match { + case 0 => A.compare(x.value, y.value) + case other => other + } + } + + /** Combinator that converts a `Scan[A, B]` in to a `Scan[TimeStamped[A], TimeStamped[B]]` such that + * timestamps are preserved on elements that flow through the stream. + */ + def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeStamped[I], TimeStamped[O]] = + t.lens(_.value, (tsi, o) => tsi.copy(value = o)) + + /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + * `TimeStamped[B]` where `B` is an accumulated feature of `A` over a second. + * + * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated + * using `perSecondRate(_.size * 8)`, which yields a stream of the emitted bits per second. + * + * @param f function which extracts a feature of `A` + */ + def perSecondRate[A, B: Monoid]( + f: A => B + ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[B]] = + rate(1.second)(f) + + /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + * `TimeStamped[B Either A]` where `B` is an accumulated feature of `A` over a second. + * + * Every incoming `A` is echoed to the output. + * + * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated + * using `perSecondRate(_.size * 8)`, which yields a stream of the emitted bits per second. + * + * @param f function which extracts a feature of `A` + */ + def withPerSecondRate[A, B: Monoid]( + f: A => B + ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]] = + withRate(1.second)(f) + + /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + * `TimeStamped[B]` where `B` is an accumulated feature of `A` over a specified time period. + * + * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated + * using `rate(1.0)(_.size * 8)`, which yields a stream of the emitted bits per second. + * + * @param over time period over which to calculate + * @param f function which extracts a feature of `A` + */ + def rate[A, B: Monoid]( + over: FiniteDuration + )(f: A => B): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[B]] = { + val t = withRate(over)(f) + Scan(t.initial)( + (s, tsa) => { + val (s2, out) = t.transform(s, tsa) + (s2, out.collect { case TimeStamped(ts, Left(b)) => TimeStamped(ts, b) }) + }, + s => t.onComplete(s).collect { case TimeStamped(ts, Left(b)) => TimeStamped(ts, b) } + ) + } + + /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + * `TimeStamped[Either[B, A]]` where `B` is an accumulated feature of `A` over a specified time period. + * + * Every incoming `A` is echoed to the output. + * + * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated + * using `rate(1.0)(_.size * 8)`, which yields a stream of the emitted bits per second. + * + * @param over time period over which to calculate + * @param f function which extracts a feature of `A` + */ + def withRate[A, B](over: FiniteDuration)(f: A => B)(implicit + B: Monoid[B] + ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]] = { + val overMillis = over.toMillis + Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]](None -> B.empty)( + { case ((end, acc), tsa) => + end match { + case Some(end) => + if (tsa.time.isBefore(end)) + (Some(end) -> B.combine(acc, f(tsa.value)), Chunk(tsa.map(Right.apply))) + else { + val bldr = List.newBuilder[TimeStamped[Either[B, A]]] + var e2 = end + var acc2 = acc + while (!tsa.time.isBefore(e2)) { + bldr += TimeStamped(e2, Left(acc2)) + acc2 = B.empty + e2 = e2 + overMillis + } + bldr += (tsa.map(Right.apply)) + ((Some(e2), f(tsa.value)), Chunk.seq(bldr.result())) + } + case None => ((Some(tsa.time + overMillis), f(tsa.value)), Chunk(tsa.map(Right.apply))) + } + }, + { + case (Some(end), acc) => Chunk(TimeStamped(end, Left(acc))) + case (None, _) => Chunk.empty + } + ) + } + + /** Returns a stream that is the throttled version of the source stream. + * + * Given two adjacent items from the source stream, `a` and `b`, where `a` is emitted + * first and `b` is emitted second, their time delta is `b.time - a.time`. + * + * This function creates a stream that emits values at wall clock times such that + * the time delta between any two adjacent values is proportional to their time delta + * in the source stream. + * + * The `throttlingFactor` is a scaling factor that determines how much source time a unit + * of wall clock time is worth. A value of 1.0 causes the output stream to emit + * values spaced in wall clock time equal to their time deltas. A value of 2.0 + * emits values at twice the speed of wall clock time. + * + * This is particularly useful when timestamped data can be read in bulk (e.g., from a capture file) + * but should be "played back" at real time speeds. + */ + def throttle[F[_]: Temporal, A]( + throttlingFactor: Double, + tickResolution: FiniteDuration = 100.milliseconds + ): Pipe[F, TimeStamped[A], TimeStamped[A]] = { + + val ticksPerSecond = 1.second.toMillis / tickResolution.toMillis + + def doThrottle: Pipe2[F, TimeStamped[A], Unit, TimeStamped[A]] = { + + type PullFromSourceOrTicks = + (Stream[F, TimeStamped[A]], Stream[F, Unit]) => Pull[F, TimeStamped[A], Unit] + + def takeUpto( + chunk: Chunk[TimeStamped[A]], + upto: TimeStamp + ): (Chunk[TimeStamped[A]], Chunk[TimeStamped[A]]) = { + val uptoMillis = upto.toEpochMilli + val toTake = chunk.indexWhere(_.time.toEpochMilli > uptoMillis).getOrElse(chunk.size) + chunk.splitAt(toTake) + } + + def read(upto: TimeStamp): PullFromSourceOrTicks = { (src, ticks) => + src.pull.uncons.flatMap { + case Some((chunk, tl)) => + if (chunk.isEmpty) read(upto)(tl, ticks) + else { + val (toOutput, pending) = takeUpto(chunk, upto) + if (pending.isEmpty) Pull.output(toOutput) >> read(upto)(tl, ticks) + else Pull.output(toOutput) >> awaitTick(upto, pending)(tl, ticks) + } + case None => Pull.done + } + } + + def awaitTick(upto: TimeStamp, pending: Chunk[TimeStamped[A]]): PullFromSourceOrTicks = { + (src, ticks) => + ticks.pull.uncons1.flatMap { + case Some((_, tl)) => + val newUpto = upto + ((1000 / ticksPerSecond) * throttlingFactor).toLong + val (toOutput, stillPending) = takeUpto(pending, newUpto) + if (stillPending.isEmpty) { + Pull.output(toOutput) >> read(newUpto)(src, tl) + } else { + Pull.output(toOutput) >> awaitTick(newUpto, stillPending)(src, tl) + } + case None => Pull.done + } + } + + (src, ticks) => + src.pull.uncons1.flatMap { + case Some((tsa, tl)) => Pull.output1(tsa) >> read(tsa.time)(tl, ticks) + case None => Pull.done + }.stream + } + + source => (source.through2(Stream.awakeEvery[F](tickResolution).as(())))(doThrottle) + } + + /** Stream transducer that filters the specified timestamped values to ensure + * the output time stamps are always increasing in time. Other values are + * dropped. + */ + def increasing[F[_], A]: Pipe[F, TimeStamped[A], TimeStamped[A]] = + increasingW.andThen(_.collect { case Right(out) => out }) + + /** Stream transducer that filters the specified timestamped values to ensure + * the output time stamps are always increasing in time. The increasing values + * are emitted as output of the writer, while out of order values are written + * to the writer side of the writer. + */ + def increasingW[F[_], A]: Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = + _.scanChunks(Long.MinValue) { (last, chunk) => + chunk.mapAccumulate(last) { (last, tsa) => + val now = tsa.time.toEpochMilli + if (last <= now) (now, Right(tsa)) else (last, Left(tsa)) + } + } + + /** Stream transducer that reorders a stream of timestamped values that are mostly ordered, + * using a time based buffer of the specified duration. See [[attemptReorderLocally]] for details. + * + * The resulting stream is guaranteed to always emit values in time increasing order. + * Values may be dropped from the source stream if they were not successfully reordered. + */ + def reorderLocally[F[_], A](over: FiniteDuration): Pipe[F, TimeStamped[A], TimeStamped[A]] = + reorderLocallyW(over).andThen(_.collect { case Right(tsa) => tsa }) + + /** Stream transducer that reorders a stream of timestamped values that are mostly ordered, + * using a time based buffer of the specified duration. See [[attemptReorderLocally]] for details. + * + * The resulting stream is guaranteed to always emit output values in time increasing order. + * Any values that could not be reordered due to insufficient buffer space are emitted on the writer (left) + * side. + */ + def reorderLocallyW[F[_], A]( + over: FiniteDuration + ): Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = + attemptReorderLocally(over).andThen(increasingW) + + /** Stream transducer that reorders timestamped values over a specified duration. + * + * Values are kept in an internal buffer. Upon receiving a new value, any buffered + * values that are timestamped with `value.time - over` are emitted. Other values, + * and the new value, are kept in the buffer. + * + * This is useful for ordering mostly ordered streams, where values + * may be out of order with close neighbors but are strictly less than values + * that come much later in the stream. + * + * An example of such a structure is the result of merging streams of values generated + * with `TimeStamped.now`. + * + * Caution: this transducer should only be used on streams that are mostly ordered. + * In the worst case, if the source is in reverse order, all values in the source + * will be accumulated in to the buffer until the source halts, and then the + * values will be emitted in order. + */ + def attemptReorderLocally[F[_], A]( + over: FiniteDuration + ): Pipe[F, TimeStamped[A], TimeStamped[A]] = { + import scala.collection.immutable.SortedMap + val overMillis = over.toMillis + + def outputMapValues(m: SortedMap[Long, Chain[TimeStamped[A]]]) = + Pull.output( + Chunk.seq( + m.foldLeft(Chain.empty[TimeStamped[A]]) { case (acc, (_, tss)) => acc ++ tss }.toList + ) + ) + + def go( + buffered: SortedMap[Long, Chain[TimeStamped[A]]], + s: Stream[F, TimeStamped[A]] + ): Pull[F, TimeStamped[A], Unit] = + s.pull.uncons.flatMap { + case Some((hd, tl)) => + val all = Chain.fromSeq(hd.toList).foldLeft(buffered) { (acc, tsa) => + val k = tsa.time.toEpochMilli + acc.updated(k, acc.getOrElse(k, Chain.empty) :+ tsa) + } + if (all.isEmpty) go(buffered, tl) + else { + val until = all.last._1 - overMillis + val (toOutput, toBuffer) = all.span { case (x, _) => x <= until } + outputMapValues(toOutput) >> go(toBuffer, tl) + } + case None => + outputMapValues(buffered) + } + + in => go(SortedMap.empty, in).stream + } + + def left[S, I, O, A]( + t: Scan[S, TimeStamped[I], TimeStamped[O]] + ): Scan[S, TimeStamped[Either[I, A]], TimeStamped[Either[O, A]]] = + t.semilens( + { + case TimeStamped(t, Left(i)) => Right(TimeStamped(t, i)) + case TimeStamped(t, Right(a)) => Left(TimeStamped(t, Right(a))) + }, + (_, tso) => tso.map(Left(_)) + ) + + def right[S, I, O, A]( + t: Scan[S, TimeStamped[I], TimeStamped[O]] + ): Scan[S, TimeStamped[Either[A, I]], TimeStamped[Either[A, O]]] = + t.semilens( + { + case TimeStamped(t, Right(i)) => Right(TimeStamped(t, i)) + case TimeStamped(t, Left(a)) => Left(TimeStamped(t, Left(a))) + }, + (_, tso) => tso.map(Right(_)) + ) +} diff --git a/core/shared/src/main/scala/fs2/timeseries/package.scala b/core/shared/src/main/scala/fs2/timeseries/package.scala new file mode 100644 index 0000000000..6e59542db8 --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/package.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 + +package object timeseries { + + /** A single value in a `TimeSeries`. Provides a timestamp along with either a value of type `A` or + * a clock tick (represented by a none). + */ + type TimeSeriesValue[+A] = TimeStamped[Option[A]] + + /** A stream of timestamped values or clock ticks. + * + * Values are represented as right values in a `TimeStamped[Option[A]]`, whereas + * clock ticks are represented as nones. This encoding allows for an indication + * of time passage with no observed values. + * + * Generally, time series appear in increasing order, and many combinators that work with + * time series will rely on that. For streams that are globally ordered, but not locally ordered, + * i.e., near adjacent values might be out of order but values at great distance from each other + * are ordered, consider using `TimeStamped.reorderLocally` to adjust. + */ + type TimeSeries[F[_], +A] = Stream[F, TimeSeriesValue[A]] + + /** Alias for a pipe on a time series. */ + type TimeSeriesPipe[F[_], -A, +B] = Stream[F, TimeSeriesValue[A]] => Stream[F, TimeSeriesValue[B]] +} From 014e274177d3ccb336de63bc30f2434b7a39bf45 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 3 Sep 2021 10:32:58 -0400 Subject: [PATCH 02/33] Port tests --- .../scala/fs2/timeseries/TimeSeries.scala | 142 +++++++++++ .../scala/fs2/timeseries/TimeSeriesPipe.scala | 4 +- .../main/scala/fs2/timeseries/TimeStamp.scala | 7 +- .../scala/fs2/timeseries/TimeStamped.scala | 7 + .../fs2/timeseries/TimeSeriesPipeSuite.scala | 52 ++++ .../fs2/timeseries/TimeSeriesSuite.scala | 62 +++++ .../fs2/timeseries/TimeStampedSuite.scala | 224 ++++++++++++++++++ 7 files changed, 494 insertions(+), 4 deletions(-) create mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala create mode 100644 core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala create mode 100644 core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala create mode 100644 core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala new file mode 100644 index 0000000000..0015bf9bff --- /dev/null +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -0,0 +1,142 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package timeseries + +import scala.concurrent.duration._ + +import cats.effect.Temporal + +/** Companion for [[TimeSeries]]. */ +object TimeSeries { + + /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ + def apply[F[_]: Temporal, A]( + source: Stream[F, TimeStamped[A]], + tickPeriod: FiniteDuration = 1.second, + reorderOver: FiniteDuration = 100.milliseconds + ): TimeSeries[F, A] = { + val src: TimeSeries[F, A] = source.map(tsa => tsa.map(Some(_): Option[A])) + val ticks: TimeSeries[F, Nothing] = timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) + src.merge(ticks).through(TimeStamped.reorderLocally(reorderOver)) + } + + /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ + def lift[F[_]: Temporal, A]( + source: Stream[F, A], + tickPeriod: FiniteDuration = 1.second, + reorderOver: FiniteDuration = 100.milliseconds + ): TimeSeries[F, A] = + apply(source.map(TimeStamped.unsafeNow), tickPeriod, reorderOver) + + /** Stream of time ticks spaced by `tickPeriod`. */ + private def timeTicks[F[_]: Temporal](tickPeriod: FiniteDuration): Stream[F, TimeStamped[Unit]] = + Stream.awakeEvery[F](tickPeriod).map(_ => TimeStamped.unsafeNow(())) + + /** Stream transducer that converts a stream of timestamped values with monotonically increasing timestamps in + * to a stream of timestamped ticks or values, where a tick is emitted every `tickPeriod`. + * Ticks are emitted between values from the source stream. + */ + def interpolateTicks[A]( + tickPeriod: FiniteDuration = 1.second + ): Pipe[Pure, TimeStamped[A], TimeSeriesValue[A]] = { + val tickPeriodMillis = tickPeriod.toMillis + def go( + nextTick: TimeStamp, + s: Stream[Pure, TimeStamped[A]] + ): Pull[Pure, TimeSeriesValue[A], Unit] = { + def tickTime(x: Int) = nextTick + (x * tickPeriodMillis) + s.pull.uncons.flatMap { + case Some((hd, tl)) => + hd.indexWhere(_.time.toEpochMilli >= nextTick.toEpochMilli) match { + case None => + if (hd.isEmpty) Pull.pure(()) + else Pull.output(hd.map(_.toTimeSeriesValue)) >> go(nextTick, tl) + case Some(idx) => + val (prefix, suffix) = hd.splitAt(idx) + val out = + if (prefix.isEmpty) Pull.pure(()) else Pull.output(prefix.map(_.toTimeSeriesValue)) + // we know suffix is non-empty and suffix.head has a time >= next tick time + val next = suffix(0) + val tickCount = + ((next.time.toEpochMilli - nextTick.toEpochMilli) / tickPeriodMillis + 1).toInt + val tickTimes = (0 until tickCount).map(tickTime) + val ticks = tickTimes.map(TimeSeriesValue.tick) + val rest = Pull.output(Chunk.seq(ticks)) >> go(tickTime(tickCount), tl.cons(suffix)) + out >> rest + } + case None => Pull.done + } + } + in => + in.pull.uncons1.flatMap { + case Some((hd, tl)) => + Pull.output1(hd.toTimeSeriesValue) >> go(hd.time + tickPeriodMillis, tl) + case None => Pull.done + }.stream + } + + /** Combinator that converts a `Scan[I, O]` in to a `Scan[TimeSeriesValue[I], TimeSeriesValue[O]]` such that + * timestamps are preserved on elements that flow through the stream. + */ + def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]] = + preserveTicks(TimeStamped.preserve(t)) + + /** Combinator that converts a `Scan[TimeStamped[A], TimeStamped[B]]` in to a `Scan[TimeSeriesValue[A], TimeSeriesValue[B]]` such that + * timestamps are preserved on elements that flow through the stream. + */ + def preserveTicks[S, I, O]( + t: Scan[S, TimeStamped[I], TimeStamped[O]] + ): Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]] = + t.semilens( + tsi => + tsi.value + .map(v => Right(TimeStamped(tsi.time, v))) + .getOrElse(Left(TimeSeriesValue.tick(tsi.time))), + (_, tso) => tso.map(Some(_)) + ) + + /** Combinator that combines a `Scan[TimeSeriesValue[L],O]` and a `Scan[TimeSeriesValue[R],O]` in to a `Scan[TimeSeriesVlaue[Either[L,R],O]]`. + */ + def choice[LS, L, RS, R, O]( + l: Scan[LS, TimeSeriesValue[L], O], + r: Scan[RS, TimeSeriesValue[R], O] + ): Scan[(LS, RS), TimeSeriesValue[Either[L, R]], O] = + Scan[(LS, RS), TimeSeriesValue[Either[L, R]], O]((l.initial, r.initial))( + { case ((lState, rState), tsv) => + tsv match { + case TimeStamped(t, Some(Left(lValue))) => + val (s, out) = l.transform(lState, TimeStamped(t, Some(lValue))) + (s -> rState, out) + case TimeStamped(t, Some(Right(rValue))) => + val (s, out) = r.transform(rState, TimeStamped(t, Some(rValue))) + (lState -> s, out) + case TimeStamped(t, None) => + val (ls, lout) = l.transform(lState, TimeStamped(t, None)) + val (rs, rout) = r.transform(rState, TimeStamped(t, None)) + ((ls, rs), lout ++ rout) + } + }, + { case (lState, rState) => l.onComplete(lState) ++ r.onComplete(rState) } + ) +} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala index 8d9dbc1981..19e04a2201 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala @@ -28,12 +28,12 @@ object TimeSeriesPipe { _.map(_.map(_.map(f))) def drainRight[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], L] = _.collect { - case tick @ TimeStamped(ts, None) => tick.asInstanceOf[TimeSeriesValue[L]] + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeSeriesValue[L]] case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) } def drainLeft[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], R] = _.collect { - case tick @ TimeStamped(ts, None) => tick.asInstanceOf[TimeSeriesValue[R]] + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeSeriesValue[R]] case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) } } diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala index cb88d01be2..214561579b 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala @@ -22,19 +22,22 @@ // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2.timeseries +import scala.concurrent.duration.FiniteDuration + import cats.Functor import cats.syntax.all._ import cats.effect.kernel.Clock final case class TimeStamp(toEpochMilli: Long) extends Ordered[TimeStamp] { def +(millis: Long): TimeStamp = TimeStamp(toEpochMilli + millis) + def +(duration: FiniteDuration): TimeStamp = TimeStamp(toEpochMilli + duration.toMillis) def isBefore(that: TimeStamp): Boolean = toEpochMilli < that.toEpochMilli def compare(that: TimeStamp): Int = toEpochMilli.compareTo(that.toEpochMilli) } object TimeStamp { + def fromSeconds(seconds: Long): TimeStamp = apply(seconds * 1000L) + def fromMillis(millis: Long): TimeStamp = apply(millis) def unsafeNow(): TimeStamp = TimeStamp(System.currentTimeMillis()) def now[F[_]: Functor: Clock]: F[TimeStamp] = Clock[F].realTime.map(d => TimeStamp(d.toMillis)) - -// implicit val ordering: Ordering[TimeStamp] = (x, y) => x.toEpochMilli.compare(y.toEpochMilli) } diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index 129fd1f0a2..909a49d83c 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -350,4 +350,11 @@ object TimeStamped { }, (_, tso) => tso.map(Right(_)) ) + + object syntax { + implicit class AtSyntax[A](private val value: A) extends AnyVal { + def at(d: FiniteDuration): TimeStamped[A] = + TimeStamped(TimeStamp.fromMillis(d.toMillis), value) + } + } } diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala new file mode 100644 index 0000000000..1c437f4d40 --- /dev/null +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package timeseries + +import scala.concurrent.duration._ + +import TimeStamped.syntax._ + +class TimeSeriesTransducerTest extends Fs2Suite { + + test("support combining two transducers via an either") { + val add1: Scan[Unit, Int, Int] = Scan.lift(_ + 1) + val add2: Scan[Unit, Int, Int] = Scan.lift(_ + 2) + val x: Scan[Unit, Either[Int, Int], Int] = add1.or(add2).imapState(_._1)(u => (u, u)) + val source: TimeSeries[Pure, Either[Int, Int]] = + Stream( + Right(1).at(0.seconds), + Left(2).at(0.5.seconds), + Right(3).at(1.5.seconds) + ).through(TimeSeries.interpolateTicks()) + assertEquals( + source.through(TimeSeries.preserve(x).toPipe).toList, + List( + TimeSeriesValue(TimeStamp.fromMillis(0), 3), + TimeSeriesValue(TimeStamp.fromMillis(500), 3), + TimeSeriesValue.tick(TimeStamp.fromMillis(1000)), + TimeSeriesValue(TimeStamp.fromMillis(1500), 5) + ) + ) + } +} diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala new file mode 100644 index 0000000000..d88de21bee --- /dev/null +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package timeseries + +import scala.concurrent.duration._ +import TimeStamped.syntax._ + +class TimeSeriesSuite extends Fs2Suite { + + def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), value) + + test("interpolating time ticks in a timestamped stream") { + val events = Stream(ts(1), ts(2), ts(3)) + val withTicksDefault = events.through(TimeSeries.interpolateTicks()).toList + assertEquals( + withTicksDefault, + List( + Some(1).at(1.seconds), + None.at(2.seconds), + Some(2).at(2.seconds), + None.at(3.seconds), + Some(3).at(3.seconds) + ) + ) + val withTicks300ms = events.through(TimeSeries.interpolateTicks(300.millis)).toList + assertEquals( + withTicks300ms, + List( + Some(1).at(1.second), + None.at(1.3.seconds), + None.at(1.6.seconds), + None.at(1.9.seconds), + Some(2).at(2.seconds), + None.at(2.2.seconds), + None.at(2.5.seconds), + None.at(2.8.seconds), + Some(3).at(3.seconds) + ) + ) + } +} diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala new file mode 100644 index 0000000000..e06ff03430 --- /dev/null +++ b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala @@ -0,0 +1,224 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package timeseries + +import scala.concurrent.duration._ + +import cats.effect._ +import cats.syntax.all._ +import scodec.bits._ + +import munit.Location + +import TimeStamped.syntax._ + +class TimeStampedSuite extends Fs2Suite { + + def assertEqualsEpsilon(actual: Double, expected: Double, epsilon: Double)(implicit + loc: Location + ): Unit = + assert((actual >= expected - epsilon) && (actual <= expected + epsilon)) + + def assertEqualsEpsilon(actual: Long, expected: Long, epsilon: Long)(implicit + loc: Location + ): Unit = + assertEqualsEpsilon(actual.toDouble, expected.toDouble, epsilon.toDouble) + + group("rates") { + + test( + "emits accumulated feature values for each specified time period and emits a final value" + ) { + val data = Stream(1.at(0.seconds), 2.at(0.5.seconds), 1.at(1.seconds), 2.at(2.3.seconds)) + + assertEquals( + data.through(TimeStamped.rate(1.second)(identity[Int]).toPipe).toVector, + Vector(3.at(1.second), 1.at(2.seconds), 2.at(3.seconds)) + ) + + assertEquals( + data.through(TimeStamped.rate(2.seconds)(identity[Int]).toPipe).toVector, + Vector(4.at(2.seconds), 2.at(4.seconds)) + ) + } + + test("emits 0s when values are skipped over") { + val data = Stream(1.at(0.seconds), 2.at(3.3.seconds)) + assertEquals( + data.through(TimeStamped.rate(1.second)(identity[Int]).toPipe).toVector, + Vector(1.at(1.second), 0.at(2.seconds), 0.at(3.seconds), 2.at(4.seconds)) + ) + + assertEquals( + data.through(TimeStamped.withRate(1.second)(identity[Int]).toPipe).toVector, + Vector( + Right(1).at(0.seconds), + Left(1).at(1.second), + Left(0).at(2.seconds), + Left(0).at(3.seconds), + Right(2).at(3.3.seconds), + Left(2).at(4.seconds) + ) + ) + } + + test("supports calculation of an average bitrate") { + val data = Stream( + hex"deadbeef".at(0.seconds), + hex"deadbeef".at(1.seconds), + hex"deadbeef".at(1.5.seconds), + hex"deadbeef".at(2.5.seconds), + hex"deadbeef".at(2.6.seconds) + ) + + val bitsPerSecond = + data.through(TimeStamped.rate(1.second)((x: ByteVector) => x.size * 8L).toPipe) + + case class Average(samples: Int, value: Double) + val zero = Average(0, 0) + val combineAverages = (x: Average, y: Average) => { + val totalSamples = x.samples + y.samples + val avg = ((x.samples * x.value) + (y.samples * y.value)) / totalSamples + Average(totalSamples, avg) + } + + val avgBitrate = bitsPerSecond.toVector.foldLeft(zero) { (acc, bits) => + combineAverages(acc, Average(1, bits.value.toDouble)) + } + assertEqualsEpsilon(avgBitrate.value, 53.3, 1.0) + } + } + + group( + "support filtering a source of timestamped values such that output is monotonically increasing in time" + ) { + def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), ()) + val data = Stream(0, -2, -1, 1, 5, 3, 6).map(ts) + + test("supports dropping out-of-order values") { + val filtered = data.through(TimeStamped.increasing) + assertEquals(filtered.toList, List(ts(0), ts(1), ts(5), ts(6))) + } + + test("supports receiving out-of-order values") { + val filtered = data.through(TimeStamped.increasingW) + assertEquals( + filtered.toList, + List( + Right(ts(0)), + Left(ts(-2)), + Left(ts(-1)), + Right(ts(1)), + Right(ts(5)), + Left(ts(3)), + Right(ts(6)) + ) + ) + } + } + + group( + "support reordering timestamped values over a specified time buffer such that output is monotonically increasing in time" + ) { + def ts(value: Int) = TimeStamped(TimeStamp.fromMillis(value.toLong), value.toLong) + + val onTheSecond = Stream.emits(1 to 10).map(x => ts(x * 1000)) + val onTheQuarterPast = onTheSecond.map(_.mapTime(t => t + 250)) + + test("reorders when all out of order values lie within the buffer time") { + val inOrder = onTheSecond.interleave(onTheQuarterPast) + val outOfOrder = onTheQuarterPast.interleave(onTheSecond) + val reordered = outOfOrder.through(TimeStamped.reorderLocally(1.second)) + assertEquals(reordered.toList, inOrder.toList) + } + + test("drops values that appear outside the buffer time") { + // Create mostly ordered data with clumps of values around each second that are unordered + val events = Stream.emits(1 to 10).flatMap { x => + val local = (-10 to 10).map(y => ts((x * 1000) + (y * 10))) + Stream.emits(scala.util.Random.shuffle(local)) + } + val reordered200ms = events.through(TimeStamped.reorderLocally(200.milliseconds)) + assertEquals(reordered200ms.toList, events.toList.sorted) + + val reordered20ms = events.through(TimeStamped.reorderLocally(20.milliseconds)) + assert(reordered20ms.toList.size >= 10) + } + + test("emits values with the same timestamp in insertion order") { + val onTheSecondBumped = onTheSecond.map(_.map(_ + 1)) + val inOrder = (onTheSecond + .interleave(onTheQuarterPast)) + .interleave(onTheSecondBumped.interleave(onTheQuarterPast)) + val outOfOrder = (onTheQuarterPast + .interleave(onTheSecond)) + .interleave(onTheQuarterPast.interleave(onTheSecondBumped)) + val reordered = outOfOrder.through(TimeStamped.reorderLocally(1.second)) + assertEquals(reordered.toList, inOrder.toList) + } + } + + test("support throttling a time stamped source") { + def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), value.toLong) + val source = Stream(ts(0), ts(1), ts(2), ts(3), ts(4)).covary[IO] + def time[A](f: IO[A]): IO[Long] = + IO.delay(System.nanoTime()).flatMap { started => + f >> IO.delay(System.nanoTime() - started) + } + val realtime = source.through(TimeStamped.throttle[IO, Long](1.0)).compile.drain + val doubletime = source.through(TimeStamped.throttle[IO, Long](2.0)).compile.drain + + time(realtime).map { elapsed => + assertEqualsEpsilon(elapsed, 4.seconds.toNanos, 250.millis.toNanos) + } >> + time(doubletime).map { elapsed => + assertEqualsEpsilon(elapsed, 2.seconds.toNanos, 250.millis.toNanos) + } + } + + test( + "support lifting a Scan[S, TimeStamped[A], TimeStamped[B]] in to a Scan[S, TimeStamped[Either[A, C]], TimeStamped[Either[B, C]]]" + ) { + val source = Stream( + TimeStamped(TimeStamp.fromMillis(1), Left(1)), + TimeStamped(TimeStamp.fromMillis(2), Right(2)), + TimeStamped(TimeStamp.fromMillis(3), Right(3)), + TimeStamped(TimeStamp.fromMillis(4), Left(4)), + TimeStamped(TimeStamp.fromMillis(5), Left(5)), + TimeStamped(TimeStamp.fromMillis(6), Right(6)) + ) + val square: Scan[Unit, TimeStamped[Int], TimeStamped[Int]] = Scan.lift(_.map(x => x * x)) + assertEquals( + source.through(TimeStamped.left(square).toPipe).toVector, + Vector( + TimeStamped(TimeStamp.fromMillis(1), Left(1)), + TimeStamped(TimeStamp.fromMillis(2), Right(2)), + TimeStamped(TimeStamp.fromMillis(3), Right(3)), + TimeStamped(TimeStamp.fromMillis(4), Left(16)), + TimeStamped(TimeStamp.fromMillis(5), Left(25)), + TimeStamped(TimeStamp.fromMillis(6), Right(6)) + ) + ) + } +} From 77681e3e83bbe66e5f0b464a075eecf16955435b Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 3 Sep 2021 10:46:27 -0400 Subject: [PATCH 03/33] Mark flaky test --- core/shared/src/test/scala/fs2/StreamCombinatorsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/shared/src/test/scala/fs2/StreamCombinatorsSuite.scala b/core/shared/src/test/scala/fs2/StreamCombinatorsSuite.scala index bb5e0d57a1..5fe3c5b001 100644 --- a/core/shared/src/test/scala/fs2/StreamCombinatorsSuite.scala +++ b/core/shared/src/test/scala/fs2/StreamCombinatorsSuite.scala @@ -369,7 +369,7 @@ class StreamCombinatorsSuite extends Fs2Suite { .assertEquals(List(2, 4, 6, 8)) } - test("filters up to N items in parallel") { + test("filters up to N items in parallel".flaky) { val s = Stream.range(0, 100) val n = 5 From 5d817da2856fde2ae1d0b18ab42905a9a20a22da Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sat, 4 Sep 2021 13:25:06 -0400 Subject: [PATCH 04/33] Add ScalaDoc to Scan --- core/shared/src/main/scala/fs2/Scan.scala | 85 ++++++++++++++++--- .../scala/fs2/timeseries/TimeSeries.scala | 1 + 2 files changed, 74 insertions(+), 12 deletions(-) diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index 35dff512b1..8f03f6f568 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -22,49 +22,75 @@ // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 +import cats.{Contravariant, Functor} import cats.data.AndThen +/** A stateful transformation of the elements of a stream. + * + * A scan is primarily represented as a function `(S, I) => (S, Chunk[O])`. + * Scans also have an initial state value of type `S` and the ability to emit + * elements upon completion via a function `S => Chunk[O]`. + * + * A scan is built up incrementally via various combinators and then converted to + * a pipe via `.toPipe`. For example, `s.through(Scan.lift(identity).toPipe) == s`. + * + * A scan is much less powerful than a pull. Scans cannot evaluate effects or terminate + * early. These limitations allow combinators that are not possible on pulls though. + * For example, the [[first]] method converts a `Scan[S, I, O]` to a `Scan[S, (I, A), (O, A)]`. + * Critically, this method relies on the ability to feed a single `I` to the original scan + * and collect the resulting `O` values, pairing each `O` with the `A` that was paired with `I`. + */ final class Scan[S, -I, +O]( val initial: S, private val transform_ : AndThen[(S, I), (S, Chunk[O])], private val onComplete_ : AndThen[S, Chunk[O]] ) { + /** Transformation function. */ def transform(s: S, i: I): (S, Chunk[O]) = transform_((s, i)) + /** Chunk form of [[transform]]. */ def transformAccumulate(s: S, c: Chunk[I]): (S, Chunk[O]) = - // c.traverse(i => State(transform(_, i))).map(_.flatten).run(s).value + // Same as: c.traverse(i => State(transform(_, i))).map(_.flatten).run(s).value c.foldLeft(s -> Chunk.empty[O]) { case ((s, acc), i) => val (s2, os) = transform(s, i) (s2, acc ++ os) } + /** Completion function. */ def onComplete(s: S): Chunk[O] = onComplete_(s) + /** Converts this scan to a pipe. */ def toPipe[F[_]]: Pipe[F, I, O] = _.pull .scanChunks(initial)(transformAccumulate) .flatMap(state => Pull.output(onComplete(state))) .stream + /** Steps this scan by a single input, returning a new scan and the output elements computed from the input. */ def step(i: I): (Scan[S, I, O], Chunk[O]) = { val (s, os) = transform(initial, i) (new Scan(s, transform_, onComplete_), os) } - def andThen[S2, O2](t: Scan[S2, O, O2]): Scan[(S, S2), I, O2] = - Scan[(S, S2), I, O2]((initial, t.initial))( + /** Composes the supplied scan with this scan. + * + * The resulting scan maintains the state of each of the input scans independently. + */ + def andThen[S2, O2](that: Scan[S2, O, O2]): Scan[(S, S2), I, O2] = + Scan[(S, S2), I, O2]((initial, that.initial))( { case ((s, s2), i) => val (sp, os) = transform(s, i) - val (s2p, out) = t.transformAccumulate(s2, os) + val (s2p, out) = that.transformAccumulate(s2, os) ((sp, s2p), out) }, { case (s, s2) => - val (s3, out) = t.transformAccumulate(s2, onComplete(s)) - out ++ t.onComplete(s3) + val (s3, out) = that.transformAccumulate(s2, onComplete(s)) + out ++ that.onComplete(s3) } ) + /** Returns a new scan which transforms output values using the supplied function. */ def map[O2](f: O => O2): Scan[S, I, O2] = new Scan( initial, @@ -72,6 +98,7 @@ final class Scan[S, -I, +O]( onComplete_.andThen(_.map(f)) ) + /** Returns a new scan which transforms input values using the supplied function. */ def contramap[I2](f: I2 => I): Scan[S, I2, O] = new Scan( initial, @@ -79,6 +106,7 @@ final class Scan[S, -I, +O]( onComplete_ ) + /** Transforms the state type. */ def imapState[S2](g: S => S2)(f: S2 => S): Scan[S2, I, O] = Scan[S2, I, O](g(initial))( { (s2, i) => @@ -88,6 +116,12 @@ final class Scan[S, -I, +O]( AndThen(f).andThen(onComplete_) ) + /** Returns a new scan with transformed input and output types. + * + * Upon receiving an `I2`, `get` is invoked and the result is fed to the + * original scan. For each output value, `set` is invoked with the original + * `I2` input and the computed `O`, yielding a new output of type `O2`. + */ def lens[I2, O2](get: I2 => I, set: (I2, O) => O2): Scan[S, I2, O2] = Scan[S, I2, O2](initial)( { (s, i2) => @@ -97,12 +131,17 @@ final class Scan[S, -I, +O]( _ => Chunk.empty ) + /** Returns a scan that inputs/outputs pairs of elements, with `I` and `O` in the first element of the pair. */ def first[A]: Scan[S, (I, A), (O, A)] = lens(_._1, (t, o) => (o, t._2)) + /** Returns a scan that inputs/outputs pairs of elements, with `I` and `O` in the second element of the pair. */ def second[A]: Scan[S, (A, I), (A, O)] = lens(_._2, (t, o) => (t._1, o)) + /** Like [[lens]] but some elements are passed to the output (skipping the original scan) while other elements + * are lensed through the original scan. + */ def semilens[I2, O2](extract: I2 => Either[O2, I], inject: (I2, O) => O2): Scan[S, I2, O2] = Scan[S, I2, O2](initial)( (s, i2) => @@ -116,30 +155,44 @@ final class Scan[S, -I, +O]( _ => Chunk.empty ) + /** Like [[semilens]] but the elements of the original scan are output directly. */ def semipass[I2, O2 >: O](extract: I2 => Either[O2, I]): Scan[S, I2, O2] = semilens(extract, (_, o) => o) + /** Returns a scan that wraps the inputs/outputs with `Either`. + * Elements on the left pass through the original scan while elements on + * the right pass through directly. + */ def left[A]: Scan[S, Either[I, A], Either[O, A]] = semilens(_.fold(i => Right(i), a => Left(Right(a))), (_, o) => Left(o)) + /** Returns a scan that wraps the inputs/outputs with `Either`. + * Elements on the right pass through the original scan while elements on + * the left pass through directly. + */ def right[A]: Scan[S, Either[A, I], Either[A, O]] = semilens(_.fold(a => Left(Left(a)), i => Right(i)), (_, o) => Right(o)) - def or[S2, I2, O2 >: O](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], O2] = - Scan[(S, S2), Either[I, I2], O2]((initial, t.initial))( + /** Combines this scan with the supplied scan such that elements on the left + * are fed through this scan while elements on the right are fed through the + * suppplied scan. The outputs are joined together. + */ + def or[S2, I2, O2 >: O](that: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], O2] = + Scan[(S, S2), Either[I, I2], O2]((initial, that.initial))( { case ((s, s2), e) => e match { case Left(i) => val (sp, os) = transform(s, i) ((sp, s2), os) case Right(i2) => - val (s2p, o2s) = t.transform(s2, i2) + val (s2p, o2s) = that.transform(s2, i2) ((s, s2p), o2s) } }, - { case (s, s2) => onComplete(s) ++ t.onComplete(s2) } + { case (s, s2) => onComplete(s) ++ that.onComplete(s2) } ) + /** Like [[or]] but the output elements are kept separate. */ def either[S2, I2, O2](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], Either[O, O2]] = Scan[(S, S2), Either[I, I2], Either[O, O2]]((initial, t.initial))( { case ((s, s2), e) => @@ -157,8 +210,6 @@ final class Scan[S, -I, +O]( } object Scan { - // def apply[S, I, O](initial: S, transform: AndThen[(S, I), (S, Chunk[O])], onComplete: AndThen[S, Chunk[O]]): Scan[S,I,O] = - // new Scan[S,I,O](initial, transform, onComplete) def apply[S, I, O]( initial: S @@ -178,4 +229,14 @@ object Scan { def lift[I, O](f: I => O): Scan[Unit, I, O] = stateless(i => Chunk.singleton(f(i))) + + implicit def functor[S, I]: Functor[Scan[S, I, *]] = + new Functor[Scan[S, I, *]] { + def map[O, O2](s: Scan[S, I, O])(f: O => O2) = s.map(f) + } + + implicit def contravariant[S, O]: Contravariant[Scan[S, *, O]] = + new Contravariant[Scan[S, *, O]] { + def contramap[I, I2](s: Scan[S, I, O])(f: I2 => I) = s.contramap(f) + } } diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index 0015bf9bff..d57fd46ea3 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 From aace84a143a12f004b6d24083e63f089c873d0a8 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sun, 5 Sep 2021 10:58:35 -0400 Subject: [PATCH 05/33] Replace TimeStamp with FiniteDuration --- .../scala/fs2/timeseries/TimeSeries.scala | 11 ++-- .../fs2/timeseries/TimeSeriesValue.scala | 6 +- .../main/scala/fs2/timeseries/TimeStamp.scala | 43 -------------- .../scala/fs2/timeseries/TimeStamped.scala | 57 +++++++++---------- .../fs2/timeseries/TimeSeriesPipeSuite.scala | 9 +-- .../fs2/timeseries/TimeSeriesSuite.scala | 3 +- .../fs2/timeseries/TimeStampedSuite.scala | 33 +++++------ 7 files changed, 60 insertions(+), 102 deletions(-) delete mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index d57fd46ea3..23036c0e35 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -61,15 +61,14 @@ object TimeSeries { def interpolateTicks[A]( tickPeriod: FiniteDuration = 1.second ): Pipe[Pure, TimeStamped[A], TimeSeriesValue[A]] = { - val tickPeriodMillis = tickPeriod.toMillis def go( - nextTick: TimeStamp, + nextTick: FiniteDuration, s: Stream[Pure, TimeStamped[A]] ): Pull[Pure, TimeSeriesValue[A], Unit] = { - def tickTime(x: Int) = nextTick + (x * tickPeriodMillis) + def tickTime(x: Int) = nextTick + (x * tickPeriod) s.pull.uncons.flatMap { case Some((hd, tl)) => - hd.indexWhere(_.time.toEpochMilli >= nextTick.toEpochMilli) match { + hd.indexWhere(_.time >= nextTick) match { case None => if (hd.isEmpty) Pull.pure(()) else Pull.output(hd.map(_.toTimeSeriesValue)) >> go(nextTick, tl) @@ -80,7 +79,7 @@ object TimeSeries { // we know suffix is non-empty and suffix.head has a time >= next tick time val next = suffix(0) val tickCount = - ((next.time.toEpochMilli - nextTick.toEpochMilli) / tickPeriodMillis + 1).toInt + ((next.time.toMillis - nextTick.toMillis) / tickPeriod.toMillis + 1).toInt val tickTimes = (0 until tickCount).map(tickTime) val ticks = tickTimes.map(TimeSeriesValue.tick) val rest = Pull.output(Chunk.seq(ticks)) >> go(tickTime(tickCount), tl.cons(suffix)) @@ -92,7 +91,7 @@ object TimeSeries { in => in.pull.uncons1.flatMap { case Some((hd, tl)) => - Pull.output1(hd.toTimeSeriesValue) >> go(hd.time + tickPeriodMillis, tl) + Pull.output1(hd.toTimeSeriesValue) >> go(hd.time + tickPeriod, tl) case None => Pull.done }.stream } diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala index 4b6d60295a..b47d3d8738 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala @@ -22,12 +22,14 @@ // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2.timeseries +import scala.concurrent.duration._ + import cats.Functor import cats.effect.kernel.Clock object TimeSeriesValue { - def apply[A](time: TimeStamp, value: A): TimeSeriesValue[A] = TimeStamped(time, Some(value)) - def tick(time: TimeStamp): TimeSeriesValue[Nothing] = TimeStamped(time, None) + def apply[A](time: FiniteDuration, value: A): TimeSeriesValue[A] = TimeStamped(time, Some(value)) + def tick(time: FiniteDuration): TimeSeriesValue[Nothing] = TimeStamped(time, None) def unsafeNow[A](value: A): TimeSeriesValue[A] = TimeStamped.unsafeNow(Some(value)) def now[F[_]: Functor: Clock, A](value: A): F[TimeSeriesValue[A]] = TimeStamped.now(Some(value)) diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala deleted file mode 100644 index 214561579b..0000000000 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamp.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Copyright (c) 2013 Functional Streams for Scala - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of - * this software and associated documentation files (the "Software"), to deal in - * the Software without restriction, including without limitation the rights to - * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of - * the Software, and to permit persons to whom the Software is furnished to do so, - * subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all - * copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS - * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER - * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - */ - -// Adapted from scodec-protocols, licensed under 3-clause BSD -package fs2.timeseries - -import scala.concurrent.duration.FiniteDuration - -import cats.Functor -import cats.syntax.all._ -import cats.effect.kernel.Clock - -final case class TimeStamp(toEpochMilli: Long) extends Ordered[TimeStamp] { - def +(millis: Long): TimeStamp = TimeStamp(toEpochMilli + millis) - def +(duration: FiniteDuration): TimeStamp = TimeStamp(toEpochMilli + duration.toMillis) - def isBefore(that: TimeStamp): Boolean = toEpochMilli < that.toEpochMilli - def compare(that: TimeStamp): Int = toEpochMilli.compareTo(that.toEpochMilli) -} - -object TimeStamp { - def fromSeconds(seconds: Long): TimeStamp = apply(seconds * 1000L) - def fromMillis(millis: Long): TimeStamp = apply(millis) - def unsafeNow(): TimeStamp = TimeStamp(System.currentTimeMillis()) - def now[F[_]: Functor: Clock]: F[TimeStamp] = Clock[F].realTime.map(d => TimeStamp(d.toMillis)) -} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index 909a49d83c..b452778505 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -31,19 +31,19 @@ import cats.syntax.all._ import cats.effect.kernel.{Clock, Temporal} /** Wrapper that associates a time with a value. */ -case class TimeStamped[+A](time: TimeStamp, value: A) { +case class TimeStamped[+A](time: FiniteDuration, value: A) { def map[B](f: A => B): TimeStamped[B] = copy(value = f(value)) - def mapTime(f: TimeStamp => TimeStamp): TimeStamped[A] = copy(time = f(time)) + def mapTime(f: FiniteDuration => FiniteDuration): TimeStamped[A] = copy(time = f(time)) def toTimeSeriesValue: TimeSeriesValue[A] = map(Some.apply) } object TimeStamped { - def unsafeNow[A](a: A): TimeStamped[A] = TimeStamped(TimeStamp.unsafeNow(), a) + def unsafeNow[A](a: A): TimeStamped[A] = TimeStamped(System.currentTimeMillis().millis, a) def now[F[_]: Functor: Clock, A](a: A): F[TimeStamped[A]] = - TimeStamp.now[F].map(t => TimeStamped(t, a)) + Clock[F].realTime.map(TimeStamped(_, a)) /** Orders values by timestamp -- values with the same timestamp are considered equal. */ def timeBasedOrdering[A]: Ordering[TimeStamped[A]] = new Ordering[TimeStamped[A]] { @@ -75,7 +75,7 @@ object TimeStamped { */ def perSecondRate[A, B: Monoid]( f: A => B - ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[B]] = + ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[B]] = rate(1.second)(f) /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of @@ -90,7 +90,7 @@ object TimeStamped { */ def withPerSecondRate[A, B: Monoid]( f: A => B - ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]] = + ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]] = withRate(1.second)(f) /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of @@ -104,7 +104,7 @@ object TimeStamped { */ def rate[A, B: Monoid]( over: FiniteDuration - )(f: A => B): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[B]] = { + )(f: A => B): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[B]] = { val t = withRate(over)(f) Scan(t.initial)( (s, tsa) => { @@ -128,27 +128,26 @@ object TimeStamped { */ def withRate[A, B](over: FiniteDuration)(f: A => B)(implicit B: Monoid[B] - ): Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]] = { - val overMillis = over.toMillis - Scan[(Option[TimeStamp], B), TimeStamped[A], TimeStamped[Either[B, A]]](None -> B.empty)( + ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]] = { + Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]](None -> B.empty)( { case ((end, acc), tsa) => end match { case Some(end) => - if (tsa.time.isBefore(end)) + if (tsa.time < end) (Some(end) -> B.combine(acc, f(tsa.value)), Chunk(tsa.map(Right.apply))) else { val bldr = List.newBuilder[TimeStamped[Either[B, A]]] var e2 = end var acc2 = acc - while (!tsa.time.isBefore(e2)) { + while (tsa.time >= e2) { bldr += TimeStamped(e2, Left(acc2)) acc2 = B.empty - e2 = e2 + overMillis + e2 = e2 + over } bldr += (tsa.map(Right.apply)) ((Some(e2), f(tsa.value)), Chunk.seq(bldr.result())) } - case None => ((Some(tsa.time + overMillis), f(tsa.value)), Chunk(tsa.map(Right.apply))) + case None => ((Some(tsa.time + over), f(tsa.value)), Chunk(tsa.map(Right.apply))) } }, { @@ -189,14 +188,13 @@ object TimeStamped { def takeUpto( chunk: Chunk[TimeStamped[A]], - upto: TimeStamp + upto: FiniteDuration ): (Chunk[TimeStamped[A]], Chunk[TimeStamped[A]]) = { - val uptoMillis = upto.toEpochMilli - val toTake = chunk.indexWhere(_.time.toEpochMilli > uptoMillis).getOrElse(chunk.size) + val toTake = chunk.indexWhere(_.time > upto).getOrElse(chunk.size) chunk.splitAt(toTake) } - def read(upto: TimeStamp): PullFromSourceOrTicks = { (src, ticks) => + def read(upto: FiniteDuration): PullFromSourceOrTicks = { (src, ticks) => src.pull.uncons.flatMap { case Some((chunk, tl)) => if (chunk.isEmpty) read(upto)(tl, ticks) @@ -209,11 +207,11 @@ object TimeStamped { } } - def awaitTick(upto: TimeStamp, pending: Chunk[TimeStamped[A]]): PullFromSourceOrTicks = { + def awaitTick(upto: FiniteDuration, pending: Chunk[TimeStamped[A]]): PullFromSourceOrTicks = { (src, ticks) => ticks.pull.uncons1.flatMap { case Some((_, tl)) => - val newUpto = upto + ((1000 / ticksPerSecond) * throttlingFactor).toLong + val newUpto = upto + ((1000 / ticksPerSecond) * throttlingFactor).toLong.millis val (toOutput, stillPending) = takeUpto(pending, newUpto) if (stillPending.isEmpty) { Pull.output(toOutput) >> read(newUpto)(src, tl) @@ -247,9 +245,9 @@ object TimeStamped { * to the writer side of the writer. */ def increasingW[F[_], A]: Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = - _.scanChunks(Long.MinValue) { (last, chunk) => + _.scanChunks(Duration.MinusInf: Duration) { (last, chunk) => chunk.mapAccumulate(last) { (last, tsa) => - val now = tsa.time.toEpochMilli + val now = tsa.time if (last <= now) (now, Right(tsa)) else (last, Left(tsa)) } } @@ -297,28 +295,27 @@ object TimeStamped { over: FiniteDuration ): Pipe[F, TimeStamped[A], TimeStamped[A]] = { import scala.collection.immutable.SortedMap - val overMillis = over.toMillis - def outputMapValues(m: SortedMap[Long, Chain[TimeStamped[A]]]) = + def outputMapValues(m: SortedMap[FiniteDuration, Chain[TimeStamped[A]]]) = Pull.output( - Chunk.seq( - m.foldLeft(Chain.empty[TimeStamped[A]]) { case (acc, (_, tss)) => acc ++ tss }.toList + Chunk.chain( + m.foldLeft(Chain.empty[TimeStamped[A]]) { case (acc, (_, tss)) => acc ++ tss } ) ) def go( - buffered: SortedMap[Long, Chain[TimeStamped[A]]], + buffered: SortedMap[FiniteDuration, Chain[TimeStamped[A]]], s: Stream[F, TimeStamped[A]] ): Pull[F, TimeStamped[A], Unit] = s.pull.uncons.flatMap { case Some((hd, tl)) => val all = Chain.fromSeq(hd.toList).foldLeft(buffered) { (acc, tsa) => - val k = tsa.time.toEpochMilli + val k = tsa.time acc.updated(k, acc.getOrElse(k, Chain.empty) :+ tsa) } if (all.isEmpty) go(buffered, tl) else { - val until = all.last._1 - overMillis + val until = all.last._1 - over val (toOutput, toBuffer) = all.span { case (x, _) => x <= until } outputMapValues(toOutput) >> go(toBuffer, tl) } @@ -354,7 +351,7 @@ object TimeStamped { object syntax { implicit class AtSyntax[A](private val value: A) extends AnyVal { def at(d: FiniteDuration): TimeStamped[A] = - TimeStamped(TimeStamp.fromMillis(d.toMillis), value) + TimeStamped(d, value) } } } diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala index 1c437f4d40..c9abe71dca 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 @@ -42,10 +43,10 @@ class TimeSeriesTransducerTest extends Fs2Suite { assertEquals( source.through(TimeSeries.preserve(x).toPipe).toList, List( - TimeSeriesValue(TimeStamp.fromMillis(0), 3), - TimeSeriesValue(TimeStamp.fromMillis(500), 3), - TimeSeriesValue.tick(TimeStamp.fromMillis(1000)), - TimeSeriesValue(TimeStamp.fromMillis(1500), 5) + TimeSeriesValue(0.millis, 3), + TimeSeriesValue(500.millis, 3), + TimeSeriesValue.tick(1000.millis), + TimeSeriesValue(1500.millis, 5) ) ) } diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala index d88de21bee..30669bb5e4 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 @@ -28,7 +29,7 @@ import TimeStamped.syntax._ class TimeSeriesSuite extends Fs2Suite { - def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), value) + def ts(value: Int) = TimeStamped(value.seconds, value) test("interpolating time ticks in a timestamped stream") { val events = Stream(ts(1), ts(2), ts(3)) diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala index e06ff03430..515405c8df 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 @@ -113,7 +114,7 @@ class TimeStampedSuite extends Fs2Suite { group( "support filtering a source of timestamped values such that output is monotonically increasing in time" ) { - def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), ()) + def ts(value: Int) = TimeStamped(value.seconds, ()) val data = Stream(0, -2, -1, 1, 5, 3, 6).map(ts) test("supports dropping out-of-order values") { @@ -141,10 +142,10 @@ class TimeStampedSuite extends Fs2Suite { group( "support reordering timestamped values over a specified time buffer such that output is monotonically increasing in time" ) { - def ts(value: Int) = TimeStamped(TimeStamp.fromMillis(value.toLong), value.toLong) + def ts(value: Int) = TimeStamped(value.millis, value.toLong) val onTheSecond = Stream.emits(1 to 10).map(x => ts(x * 1000)) - val onTheQuarterPast = onTheSecond.map(_.mapTime(t => t + 250)) + val onTheQuarterPast = onTheSecond.map(_.mapTime(t => t + 250.millis)) test("reorders when all out of order values lie within the buffer time") { val inOrder = onTheSecond.interleave(onTheQuarterPast) @@ -180,7 +181,7 @@ class TimeStampedSuite extends Fs2Suite { } test("support throttling a time stamped source") { - def ts(value: Int) = TimeStamped(TimeStamp.fromSeconds(value.toLong), value.toLong) + def ts(value: Int) = TimeStamped(value.seconds, value.toLong) val source = Stream(ts(0), ts(1), ts(2), ts(3), ts(4)).covary[IO] def time[A](f: IO[A]): IO[Long] = IO.delay(System.nanoTime()).flatMap { started => @@ -201,23 +202,23 @@ class TimeStampedSuite extends Fs2Suite { "support lifting a Scan[S, TimeStamped[A], TimeStamped[B]] in to a Scan[S, TimeStamped[Either[A, C]], TimeStamped[Either[B, C]]]" ) { val source = Stream( - TimeStamped(TimeStamp.fromMillis(1), Left(1)), - TimeStamped(TimeStamp.fromMillis(2), Right(2)), - TimeStamped(TimeStamp.fromMillis(3), Right(3)), - TimeStamped(TimeStamp.fromMillis(4), Left(4)), - TimeStamped(TimeStamp.fromMillis(5), Left(5)), - TimeStamped(TimeStamp.fromMillis(6), Right(6)) + TimeStamped(1.millis, Left(1)), + TimeStamped(2.millis, Right(2)), + TimeStamped(3.millis, Right(3)), + TimeStamped(4.millis, Left(4)), + TimeStamped(5.millis, Left(5)), + TimeStamped(6.millis, Right(6)) ) val square: Scan[Unit, TimeStamped[Int], TimeStamped[Int]] = Scan.lift(_.map(x => x * x)) assertEquals( source.through(TimeStamped.left(square).toPipe).toVector, Vector( - TimeStamped(TimeStamp.fromMillis(1), Left(1)), - TimeStamped(TimeStamp.fromMillis(2), Right(2)), - TimeStamped(TimeStamp.fromMillis(3), Right(3)), - TimeStamped(TimeStamp.fromMillis(4), Left(16)), - TimeStamped(TimeStamp.fromMillis(5), Left(25)), - TimeStamped(TimeStamp.fromMillis(6), Right(6)) + TimeStamped(1.millis, Left(1)), + TimeStamped(2.millis, Right(2)), + TimeStamped(3.millis, Right(3)), + TimeStamped(4.millis, Left(16)), + TimeStamped(5.millis, Left(25)), + TimeStamped(6.millis, Right(6)) ) ) } From 73a3c8bc332d60112de2ea7ae0a6112df9fe4ba8 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sun, 5 Sep 2021 11:17:33 -0400 Subject: [PATCH 06/33] Scalafmt --- core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index b452778505..9cbb55c044 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -128,7 +128,7 @@ object TimeStamped { */ def withRate[A, B](over: FiniteDuration)(f: A => B)(implicit B: Monoid[B] - ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]] = { + ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]] = Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]](None -> B.empty)( { case ((end, acc), tsa) => end match { @@ -155,7 +155,6 @@ object TimeStamped { case (None, _) => Chunk.empty } ) - } /** Returns a stream that is the throttled version of the source stream. * From 36920308cb5b2b68a5f596edcdb28baaa4dd791b Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sun, 5 Sep 2021 13:54:40 -0400 Subject: [PATCH 07/33] Cleanup --- .scalafmt.conf | 3 +- .../scala/fs2/timeseries/TimeSeries.scala | 16 ++++---- .../scala/fs2/timeseries/TimeStamped.scala | 39 +++++++++---------- .../fs2/timeseries/TimeSeriesPipeSuite.scala | 2 +- .../fs2/timeseries/TimeSeriesSuite.scala | 32 +++++++-------- .../fs2/timeseries/TimeStampedSuite.scala | 38 +++++++++--------- 6 files changed, 65 insertions(+), 65 deletions(-) diff --git a/.scalafmt.conf b/.scalafmt.conf index cf0bfe4520..6240b2a9a3 100644 --- a/.scalafmt.conf +++ b/.scalafmt.conf @@ -28,4 +28,5 @@ rewrite.neverInfix.excludeFilters = [until have when size - theSameElementsAs] + theSameElementsAs + at] diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index 23036c0e35..e0115806e1 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -34,8 +34,8 @@ object TimeSeries { /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ def apply[F[_]: Temporal, A]( source: Stream[F, TimeStamped[A]], - tickPeriod: FiniteDuration = 1.second, - reorderOver: FiniteDuration = 100.milliseconds + tickPeriod: FiniteDuration, + reorderOver: FiniteDuration ): TimeSeries[F, A] = { val src: TimeSeries[F, A] = source.map(tsa => tsa.map(Some(_): Option[A])) val ticks: TimeSeries[F, Nothing] = timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) @@ -45,8 +45,8 @@ object TimeSeries { /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ def lift[F[_]: Temporal, A]( source: Stream[F, A], - tickPeriod: FiniteDuration = 1.second, - reorderOver: FiniteDuration = 100.milliseconds + tickPeriod: FiniteDuration, + reorderOver: FiniteDuration ): TimeSeries[F, A] = apply(source.map(TimeStamped.unsafeNow), tickPeriod, reorderOver) @@ -59,7 +59,7 @@ object TimeSeries { * Ticks are emitted between values from the source stream. */ def interpolateTicks[A]( - tickPeriod: FiniteDuration = 1.second + tickPeriod: FiniteDuration ): Pipe[Pure, TimeStamped[A], TimeSeriesValue[A]] = { def go( nextTick: FiniteDuration, @@ -96,13 +96,13 @@ object TimeSeries { }.stream } - /** Combinator that converts a `Scan[I, O]` in to a `Scan[TimeSeriesValue[I], TimeSeriesValue[O]]` such that + /** Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]]` such that * timestamps are preserved on elements that flow through the stream. */ def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]] = preserveTicks(TimeStamped.preserve(t)) - /** Combinator that converts a `Scan[TimeStamped[A], TimeStamped[B]]` in to a `Scan[TimeSeriesValue[A], TimeSeriesValue[B]]` such that + /** Combinator that converts a `Scan[S, TimeStamped[I], TimeStamped[O]]` in to a `Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]]` such that * timestamps are preserved on elements that flow through the stream. */ def preserveTicks[S, I, O]( @@ -116,7 +116,7 @@ object TimeSeries { (_, tso) => tso.map(Some(_)) ) - /** Combinator that combines a `Scan[TimeSeriesValue[L],O]` and a `Scan[TimeSeriesValue[R],O]` in to a `Scan[TimeSeriesVlaue[Either[L,R],O]]`. + /** Combinator that combines a `Scan[LS, TimeSeriesValue[L], O]` and a `Scan[RS, TimeSeriesValue[R], O]` in to a `Scan[(LS, RS), TimeSeriesVlaue[Either[L, R], O]]`. */ def choice[LS, L, RS, R, O]( l: Scan[LS, TimeSeriesValue[L], O], diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index 9cbb55c044..396f6966e6 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -65,7 +65,7 @@ object TimeStamped { def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeStamped[I], TimeStamped[O]] = t.lens(_.value, (tsi, o) => tsi.copy(value = o)) - /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + /** Scan that converts a stream of `TimeStamped[A]` in to a stream of * `TimeStamped[B]` where `B` is an accumulated feature of `A` over a second. * * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated @@ -78,7 +78,7 @@ object TimeStamped { ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[B]] = rate(1.second)(f) - /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + /** Scan that converts a stream of `TimeStamped[A]` in to a stream of * `TimeStamped[B Either A]` where `B` is an accumulated feature of `A` over a second. * * Every incoming `A` is echoed to the output. @@ -93,7 +93,7 @@ object TimeStamped { ): Scan[(Option[FiniteDuration], B), TimeStamped[A], TimeStamped[Either[B, A]]] = withRate(1.second)(f) - /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + /** Scan that converts a stream of `TimeStamped[A]` in to a stream of * `TimeStamped[B]` where `B` is an accumulated feature of `A` over a specified time period. * * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated @@ -115,7 +115,7 @@ object TimeStamped { ) } - /** Stream transducer that converts a stream of `TimeStamped[A]` in to a stream of + /** Scan that converts a stream of `TimeStamped[A]` in to a stream of * `TimeStamped[Either[B, A]]` where `B` is an accumulated feature of `A` over a specified time period. * * Every incoming `A` is echoed to the output. @@ -231,19 +231,18 @@ object TimeStamped { source => (source.through2(Stream.awakeEvery[F](tickResolution).as(())))(doThrottle) } - /** Stream transducer that filters the specified timestamped values to ensure + /** Scan that filters the specified timestamped values to ensure * the output time stamps are always increasing in time. Other values are * dropped. */ def increasing[F[_], A]: Pipe[F, TimeStamped[A], TimeStamped[A]] = - increasingW.andThen(_.collect { case Right(out) => out }) + increasingEither.andThen(_.collect { case Right(out) => out }) - /** Stream transducer that filters the specified timestamped values to ensure + /** Scan that filters the specified timestamped values to ensure * the output time stamps are always increasing in time. The increasing values - * are emitted as output of the writer, while out of order values are written - * to the writer side of the writer. + * are emitted wrapped in `Right`, while out of order values are emitted in `Left`. */ - def increasingW[F[_], A]: Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = + def increasingEither[F[_], A]: Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = _.scanChunks(Duration.MinusInf: Duration) { (last, chunk) => chunk.mapAccumulate(last) { (last, tsa) => val now = tsa.time @@ -251,28 +250,28 @@ object TimeStamped { } } - /** Stream transducer that reorders a stream of timestamped values that are mostly ordered, + /** Scan that reorders a stream of timestamped values that are mostly ordered, * using a time based buffer of the specified duration. See [[attemptReorderLocally]] for details. * * The resulting stream is guaranteed to always emit values in time increasing order. * Values may be dropped from the source stream if they were not successfully reordered. */ def reorderLocally[F[_], A](over: FiniteDuration): Pipe[F, TimeStamped[A], TimeStamped[A]] = - reorderLocallyW(over).andThen(_.collect { case Right(tsa) => tsa }) + reorderLocallyEither(over).andThen(_.collect { case Right(tsa) => tsa }) - /** Stream transducer that reorders a stream of timestamped values that are mostly ordered, + /** Scan that reorders a stream of timestamped values that are mostly ordered, * using a time based buffer of the specified duration. See [[attemptReorderLocally]] for details. * - * The resulting stream is guaranteed to always emit output values in time increasing order. - * Any values that could not be reordered due to insufficient buffer space are emitted on the writer (left) - * side. + * The resulting stream is guaranteed to always emit output values in time increasing order, + * wrapped in `Right`. Any values that could not be reordered due to insufficient buffer space + * are emitted wrapped in `Left`. */ - def reorderLocallyW[F[_], A]( + def reorderLocallyEither[F[_], A]( over: FiniteDuration ): Pipe[F, TimeStamped[A], Either[TimeStamped[A], TimeStamped[A]]] = - attemptReorderLocally(over).andThen(increasingW) + attemptReorderLocally(over).andThen(increasingEither) - /** Stream transducer that reorders timestamped values over a specified duration. + /** Scan that reorders timestamped values over a specified duration. * * Values are kept in an internal buffer. Upon receiving a new value, any buffered * values that are timestamped with `value.time - over` are emitted. Other values, @@ -285,7 +284,7 @@ object TimeStamped { * An example of such a structure is the result of merging streams of values generated * with `TimeStamped.now`. * - * Caution: this transducer should only be used on streams that are mostly ordered. + * Caution: this scan should only be used on streams that are mostly ordered. * In the worst case, if the source is in reverse order, all values in the source * will be accumulated in to the buffer until the source halts, and then the * values will be emitted in order. diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala index c9abe71dca..bab85c5a24 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala @@ -39,7 +39,7 @@ class TimeSeriesTransducerTest extends Fs2Suite { Right(1).at(0.seconds), Left(2).at(0.5.seconds), Right(3).at(1.5.seconds) - ).through(TimeSeries.interpolateTicks()) + ).through(TimeSeries.interpolateTicks(1.second)) assertEquals( source.through(TimeSeries.preserve(x).toPipe).toList, List( diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala index 30669bb5e4..c6e09ff459 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala @@ -33,30 +33,30 @@ class TimeSeriesSuite extends Fs2Suite { test("interpolating time ticks in a timestamped stream") { val events = Stream(ts(1), ts(2), ts(3)) - val withTicksDefault = events.through(TimeSeries.interpolateTicks()).toList + val withTicks1s = events.through(TimeSeries.interpolateTicks(1.second)).toList assertEquals( - withTicksDefault, + withTicks1s, List( - Some(1).at(1.seconds), - None.at(2.seconds), - Some(2).at(2.seconds), - None.at(3.seconds), - Some(3).at(3.seconds) + Some(1) at 1.seconds, + None at 2.seconds, + Some(2) at 2.seconds, + None at 3.seconds, + Some(3) at 3.seconds ) ) val withTicks300ms = events.through(TimeSeries.interpolateTicks(300.millis)).toList assertEquals( withTicks300ms, List( - Some(1).at(1.second), - None.at(1.3.seconds), - None.at(1.6.seconds), - None.at(1.9.seconds), - Some(2).at(2.seconds), - None.at(2.2.seconds), - None.at(2.5.seconds), - None.at(2.8.seconds), - Some(3).at(3.seconds) + Some(1) at 1.second, + None at 1.3.seconds, + None at 1.6.seconds, + None at 1.9.seconds, + Some(2) at 2.seconds, + None at 2.2.seconds, + None at 2.5.seconds, + None at 2.8.seconds, + Some(3) at 3.seconds ) ) } diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala index 515405c8df..164abbec1b 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala @@ -74,12 +74,12 @@ class TimeStampedSuite extends Fs2Suite { assertEquals( data.through(TimeStamped.withRate(1.second)(identity[Int]).toPipe).toVector, Vector( - Right(1).at(0.seconds), - Left(1).at(1.second), - Left(0).at(2.seconds), - Left(0).at(3.seconds), - Right(2).at(3.3.seconds), - Left(2).at(4.seconds) + Right(1) at 0.seconds, + Left(1) at 1.second, + Left(0) at 2.seconds, + Left(0) at 3.seconds, + Right(2) at 3.3.seconds, + Left(2) at 4.seconds ) ) } @@ -123,7 +123,7 @@ class TimeStampedSuite extends Fs2Suite { } test("supports receiving out-of-order values") { - val filtered = data.through(TimeStamped.increasingW) + val filtered = data.through(TimeStamped.increasingEither) assertEquals( filtered.toList, List( @@ -202,23 +202,23 @@ class TimeStampedSuite extends Fs2Suite { "support lifting a Scan[S, TimeStamped[A], TimeStamped[B]] in to a Scan[S, TimeStamped[Either[A, C]], TimeStamped[Either[B, C]]]" ) { val source = Stream( - TimeStamped(1.millis, Left(1)), - TimeStamped(2.millis, Right(2)), - TimeStamped(3.millis, Right(3)), - TimeStamped(4.millis, Left(4)), - TimeStamped(5.millis, Left(5)), - TimeStamped(6.millis, Right(6)) + Left(1) at 1.millis, + Right(2) at 2.millis, + Right(3) at 3.millis, + Left(4) at 4.millis, + Left(5) at 5.millis, + Right(6) at 6.millis ) val square: Scan[Unit, TimeStamped[Int], TimeStamped[Int]] = Scan.lift(_.map(x => x * x)) assertEquals( source.through(TimeStamped.left(square).toPipe).toVector, Vector( - TimeStamped(1.millis, Left(1)), - TimeStamped(2.millis, Right(2)), - TimeStamped(3.millis, Right(3)), - TimeStamped(4.millis, Left(16)), - TimeStamped(5.millis, Left(25)), - TimeStamped(6.millis, Right(6)) + Left(1) at 1.millis, + Right(2) at 2.millis, + Right(3) at 3.millis, + Left(16) at 4.millis, + Left(25) at 5.millis, + Right(6) at 6.millis ) ) } From 0b0ec7c0d537acef020de11635b19dd82673acfd Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sun, 5 Sep 2021 14:14:24 -0400 Subject: [PATCH 08/33] Change TimeStamped.throttle to use stepLeg --- .../scala/fs2/timeseries/TimeStamped.scala | 29 ++++++++++++------- .../fs2/timeseries/TimeStampedSuite.scala | 4 +-- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index 396f6966e6..1f254ef5bf 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -175,11 +175,13 @@ object TimeStamped { */ def throttle[F[_]: Temporal, A]( throttlingFactor: Double, - tickResolution: FiniteDuration = 100.milliseconds + tickResolution: FiniteDuration ): Pipe[F, TimeStamped[A], TimeStamped[A]] = { val ticksPerSecond = 1.second.toMillis / tickResolution.toMillis + require(ticksPerSecond > 0L, "tickResolution must be <= 1 second") + def doThrottle: Pipe2[F, TimeStamped[A], Unit, TimeStamped[A]] = { type PullFromSourceOrTicks = @@ -194,13 +196,14 @@ object TimeStamped { } def read(upto: FiniteDuration): PullFromSourceOrTicks = { (src, ticks) => - src.pull.uncons.flatMap { - case Some((chunk, tl)) => - if (chunk.isEmpty) read(upto)(tl, ticks) + src.pull.stepLeg.flatMap { + case Some(leg) => + val chunk = leg.head + if (chunk.isEmpty) read(upto)(leg.stream, ticks) else { val (toOutput, pending) = takeUpto(chunk, upto) - if (pending.isEmpty) Pull.output(toOutput) >> read(upto)(tl, ticks) - else Pull.output(toOutput) >> awaitTick(upto, pending)(tl, ticks) + if (pending.isEmpty) Pull.output(toOutput) >> read(upto)(leg.stream, ticks) + else Pull.output(toOutput) >> awaitTick(upto, pending)(leg.stream, ticks) } case None => Pull.done } @@ -208,8 +211,9 @@ object TimeStamped { def awaitTick(upto: FiniteDuration, pending: Chunk[TimeStamped[A]]): PullFromSourceOrTicks = { (src, ticks) => - ticks.pull.uncons1.flatMap { - case Some((_, tl)) => + ticks.pull.stepLeg.flatMap { + case Some(leg) => + val tl = leg.stream.cons(leg.head.drop(1)) val newUpto = upto + ((1000 / ticksPerSecond) * throttlingFactor).toLong.millis val (toOutput, stillPending) = takeUpto(pending, newUpto) if (stillPending.isEmpty) { @@ -222,9 +226,12 @@ object TimeStamped { } (src, ticks) => - src.pull.uncons1.flatMap { - case Some((tsa, tl)) => Pull.output1(tsa) >> read(tsa.time)(tl, ticks) - case None => Pull.done + src.pull.stepLeg.flatMap { + case Some(leg) => + val head = leg.head(0) + val tl = leg.head.drop(1) + Pull.output1(head) >> read(head.time)(leg.stream.cons(tl), ticks) + case None => Pull.done }.stream } diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala index 164abbec1b..397dc794c9 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala @@ -187,8 +187,8 @@ class TimeStampedSuite extends Fs2Suite { IO.delay(System.nanoTime()).flatMap { started => f >> IO.delay(System.nanoTime() - started) } - val realtime = source.through(TimeStamped.throttle[IO, Long](1.0)).compile.drain - val doubletime = source.through(TimeStamped.throttle[IO, Long](2.0)).compile.drain + val realtime = source.through(TimeStamped.throttle[IO, Long](1.0, 100.milliseconds)).compile.drain + val doubletime = source.through(TimeStamped.throttle[IO, Long](2.0, 100.milliseconds)).compile.drain time(realtime).map { elapsed => assertEqualsEpsilon(elapsed, 4.seconds.toNanos, 250.millis.toNanos) From 48be638cdaedae2f8f559559e95cf9d0cec7db86 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sun, 5 Sep 2021 14:14:36 -0400 Subject: [PATCH 09/33] Scalafmt --- .../src/test/scala/fs2/timeseries/TimeStampedSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala index 397dc794c9..3584a3e47e 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeStampedSuite.scala @@ -187,8 +187,10 @@ class TimeStampedSuite extends Fs2Suite { IO.delay(System.nanoTime()).flatMap { started => f >> IO.delay(System.nanoTime() - started) } - val realtime = source.through(TimeStamped.throttle[IO, Long](1.0, 100.milliseconds)).compile.drain - val doubletime = source.through(TimeStamped.throttle[IO, Long](2.0, 100.milliseconds)).compile.drain + val realtime = + source.through(TimeStamped.throttle[IO, Long](1.0, 100.milliseconds)).compile.drain + val doubletime = + source.through(TimeStamped.throttle[IO, Long](2.0, 100.milliseconds)).compile.drain time(realtime).map { elapsed => assertEqualsEpsilon(elapsed, 4.seconds.toNanos, 250.millis.toNanos) From 689d8b2e727322795080c34a85ca50fc8959f806 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Tue, 7 Sep 2021 07:42:53 -0400 Subject: [PATCH 10/33] Add scodec-stream --- build.sbt | 23 +- .../scala/fs2/interop/scodec/CodecError.scala | 30 ++ .../fs2/interop/scodec/StreamDecoder.scala | 261 ++++++++++++++++++ .../fs2/interop/scodec/StreamEncoder.scala | 195 +++++++++++++ .../fs2/interop/scodec/ListOfNSuite.scala | 51 ++++ .../fs2/interop/scodec/SpaceLeakTest.scala | 44 +++ .../fs2/interop/scodec/StreamCodecSuite.scala | 125 +++++++++ 7 files changed, 728 insertions(+), 1 deletion(-) create mode 100644 scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala create mode 100644 scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala create mode 100644 scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala create mode 100644 scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala create mode 100644 scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala create mode 100644 scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala diff --git a/build.sbt b/build.sbt index d8d56bc846..23b8d59c45 100644 --- a/build.sbt +++ b/build.sbt @@ -144,7 +144,7 @@ ThisBuild / mimaBinaryIssueFilters ++= Seq( lazy val root = project .in(file(".")) .enablePlugins(NoPublishPlugin, SonatypeCiReleasePlugin) - .aggregate(coreJVM, coreJS, io.jvm, node.js, io.js, reactiveStreams, benchmark) + .aggregate(coreJVM, coreJS, io.jvm, node.js, io.js, scodec.jvm, scodec.js, reactiveStreams, benchmark) lazy val rootJVM = project .in(file(".")) @@ -269,6 +269,27 @@ lazy val io = crossProject(JVMPlatform, JSPlatform) .dependsOn(core % "compile->compile;test->test") .jsConfigure(_.dependsOn(node.js)) +lazy val scodec = crossProject(JVMPlatform, JSPlatform) + .in(file("scodec")) + .enablePlugins(SbtOsgi) + .settings( + name := "fs2-scodec", + libraryDependencies += "org.scodec" %% "scodec-core" % (if (scalaVersion.value.startsWith("2.")) "1.11.8" else "2.0.0"), + OsgiKeys.exportPackage := Seq("fs2.interop.scodec.*"), + OsgiKeys.privatePackage := Seq(), + OsgiKeys.importPackage := { + val Some((major, minor)) = CrossVersion.partialVersion(scalaVersion.value) + Seq( + s"""scala.*;version="[$major.$minor,$major.${minor + 1})"""", + """fs2.*;version="${Bundle-Version}"""", + "*" + ) + }, + OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), + osgiSettings + ) + .dependsOn(core % "compile->compile;test->test") + lazy val reactiveStreams = project .in(file("reactive-streams")) .enablePlugins(SbtOsgi) diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala new file mode 100644 index 0000000000..b5fe20eedb --- /dev/null +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala @@ -0,0 +1,30 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import _root_.scodec.Err + +/** Lifts an `scodec.Err` in to an exception. */ +final case class CodecError(err: Err) extends Exception(err.messageWithContext) diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala new file mode 100644 index 0000000000..3759d08067 --- /dev/null +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala @@ -0,0 +1,261 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import _root_.scodec.{Attempt, DecodeResult, Decoder, Err, codecs} +import _root_.scodec.bits.BitVector + +/** Supports binary decoding of a stream that emits elements as they are decoded. + * + * The main purpose of using a `StreamDecoder` over a `scodec.Decoder` is mixing + * decoding with processing. For example, `scodec.codecs.vector(decoderA): Decoder[Vector[A]]` + * could be used to decode a bit stream but the decoded `Vector[A]` would not be + * emitted until the end of the bit stream. With `StreamDecoder.many(decoderA): StreamDecoder[A]`, + * each decoded `A` value is emitted as soon as it is decoded. + * + * The `StreamDecoder` companion has various constructors -- most importantly, `once` + * and `many`, that allow a `Decoder[A]` to be lifted to a `StreamDecoder[A]`. + * + * Given a `StreamDecoder[A]`, a bit stream can be decoded via the `decode` method or + * by calling a variant of `toPipe`. + */ +final class StreamDecoder[+A] private (private val step: StreamDecoder.Step[A]) { self => + + import StreamDecoder._ + + /** Converts this decoder to a `Pipe[F, BitVector, A]`. */ + def toPipe[F[_]: RaiseThrowable]: Pipe[F, BitVector, A] = decode(_) + + /** Converts this decoder to a `Pipe[F, Byte, A]`. */ + def toPipeByte[F[_]: RaiseThrowable]: Pipe[F, Byte, A] = + in => in.chunks.map(_.toBitVector).through(toPipe) + + /** Returns a `Stream[F, A]` given a `Stream[F, BitVector]`. */ + def decode[F[_]: RaiseThrowable](s: Stream[F, BitVector]): Stream[F, A] = + apply(s).void.stream + + /** Returns a `Pull[F, A, Option[Stream[F, BitVector]]]` given a `Stream[F, BitVector]`. + * The result of the returned pull is the remainder of the input stream that was not used in decoding. + */ + def apply[F[_]: RaiseThrowable]( + s: Stream[F, BitVector] + ): Pull[F, A, Option[Stream[F, BitVector]]] = + step match { + case Empty => Pull.pure(Some(s)) + case Result(a) => Pull.output1(a).as(Some(s)) + case Failed(cause) => Pull.raiseError(cause) + case Append(x, y) => + x(s).flatMap { + case None => Pull.pure(None) + case Some(rem) => y()(rem) + } + + case Decode(decoder, once, failOnErr) => + def loop( + carry: BitVector, + s: Stream[F, BitVector] + ): Pull[F, A, Option[Stream[F, BitVector]]] = + s.pull.uncons1.flatMap { + case Some((hd, tl)) => + val buffer = carry ++ hd + decoder(buffer) match { + case Attempt.Successful(DecodeResult(value, remainder)) => + val next = if (remainder.isEmpty) tl else tl.cons1(remainder) + val p = value(next) + if (once) p + else + p.flatMap { + case Some(next) => loop(BitVector.empty, next) + case None => Pull.pure(None) + } + case Attempt.Failure(_: Err.InsufficientBits) => + loop(buffer, tl) + case Attempt.Failure(comp: Err.Composite) + if comp.errs.exists(_.isInstanceOf[Err.InsufficientBits]) => + loop(buffer, tl) + case Attempt.Failure(e) => + if (failOnErr) Pull.raiseError(CodecError(e)) + else Pull.pure(Some(tl.cons1(buffer))) + } + case None => if (carry.isEmpty) Pull.pure(None) else Pull.pure(Some(Stream(carry))) + } + loop(BitVector.empty, s) + + case Isolate(bits, decoder) => + def loop( + carry: BitVector, + s: Stream[F, BitVector] + ): Pull[F, A, Option[Stream[F, BitVector]]] = + s.pull.uncons1.flatMap { + case Some((hd, tl)) => + val (buffer, remainder) = (carry ++ hd).splitAt(bits) + if (buffer.size == bits) + decoder[F](Stream(buffer)) >> Pull.pure(Some(tl.cons1(remainder))) + else loop(buffer, tl) + case None => if (carry.isEmpty) Pull.pure(None) else Pull.pure(Some(Stream(carry))) + } + loop(BitVector.empty, s) + } + + /** Creates a stream decoder that, upon decoding an `A`, applies it to the supplied function and decodes + * the next part of the input with the returned decoder. When that decoder finishes, the remainder of + * the input is returned to the original decoder for further decoding. + */ + def flatMap[B](f: A => StreamDecoder[B]): StreamDecoder[B] = + new StreamDecoder[B]( + self.step match { + case Empty => Empty + case Result(a) => f(a).step + case Failed(cause) => Failed(cause) + case Decode(g, once, failOnErr) => + Decode(in => g(in).map(_.map(_.flatMap(f))), once, failOnErr) + case Isolate(bits, decoder) => Isolate(bits, decoder.flatMap(f)) + case Append(x, y) => Append(x.flatMap(f), () => y().flatMap(f)) + } + ) + + /** Maps the supplied function over each output of this decoder. */ + def map[B](f: A => B): StreamDecoder[B] = flatMap(a => StreamDecoder.emit(f(a))) + + /** Creates a stream decoder that first decodes until this decoder finishes and then decodes + * using the supplied decoder. + * + * Note: this should not be used to write recursive decoders (e.g., `def ints: StreamDecoder[A] = once(int32) ++ ints`) + * if each incremental decoding step can fail with `InsufficientBits`. Otherwise, it decoding can get stuck in + * an infinite loop, where the remaining bits are fed to the recursive call. + */ + def ++[A2 >: A](that: => StreamDecoder[A2]): StreamDecoder[A2] = + new StreamDecoder(Append(this, () => that)) + + /** Alias for `StreamDecoder.isolate(bits)(this)`. */ + def isolate(bits: Long): StreamDecoder[A] = StreamDecoder.isolate(bits)(this) + + /** Converts this stream decoder to a `Decoder[Vector[A]]`. */ + def strict: Decoder[Vector[A]] = + new Decoder[Vector[A]] { + def decode(bits: BitVector): Attempt[DecodeResult[Vector[A]]] = { + type ET[X] = Either[Throwable, X] + self + .map(Left(_)) + .apply[Fallible](Stream(bits)) + .flatMap { remainder => + remainder + .map { r => + r.map(Right(_)).pull.echo + } + .getOrElse(Pull.done) + } + .stream + .compile[Fallible, ET, Either[A, BitVector]] + .fold((Vector.empty[A], BitVector.empty)) { case ((acc, rem), entry) => + entry match { + case Left(a) => (acc :+ a, rem) + case Right(r2) => (acc, rem ++ r2) + } + } + .fold( + { + case CodecError(e) => Attempt.failure(e) + case other => Attempt.failure(Err.General(other.getMessage, Nil)) + }, + { case (acc, rem) => Attempt.successful(DecodeResult(acc, rem)) } + ) + } + } +} + +object StreamDecoder { + private sealed trait Step[+A] + private case object Empty extends Step[Nothing] + private case class Result[A](value: A) extends Step[A] + private case class Failed(cause: Throwable) extends Step[Nothing] + private case class Decode[A]( + f: BitVector => Attempt[DecodeResult[StreamDecoder[A]]], + once: Boolean, + failOnErr: Boolean + ) extends Step[A] + private case class Isolate[A](bits: Long, decoder: StreamDecoder[A]) extends Step[A] + private case class Append[A](x: StreamDecoder[A], y: () => StreamDecoder[A]) extends Step[A] + + /** Stream decoder that emits no elements. */ + val empty: StreamDecoder[Nothing] = new StreamDecoder[Nothing](Empty) + + /** Stream decoder that emits a single `A` and consumes no bits from the input. */ + def emit[A](a: A): StreamDecoder[A] = new StreamDecoder[A](Result(a)) + + /** Stream decoder that emits the supplied `A` values and consumes no bits from the input. */ + def emits[A](as: Iterable[A]): StreamDecoder[A] = + as.foldLeft(empty: StreamDecoder[A])((acc, a) => acc ++ emit(a)) + + /** Creates a stream decoder that decodes one `A` using the supplied decoder. + * Input bits are buffered until the decoder is able to decode an `A`. + */ + def once[A](decoder: Decoder[A]): StreamDecoder[A] = + new StreamDecoder[A]( + Decode(in => decoder.decode(in).map(_.map(emit)), once = true, failOnErr = true) + ) + + /** Creates a stream decoder that repeatedly decodes `A` values using the supplied decoder. + */ + def many[A](decoder: Decoder[A]): StreamDecoder[A] = + new StreamDecoder[A]( + Decode(in => decoder.decode(in).map(_.map(emit)), once = false, failOnErr = true) + ) + + /** Creates a stream decoder that attempts to decode one `A` using the supplied decoder. + * Input bits are buffered until the decoder is able to decode an `A`. + * If decoding fails, the bits are not consumed and the stream decoder yields no values. + */ + def tryOnce[A](decoder: Decoder[A]): StreamDecoder[A] = + new StreamDecoder[A]( + Decode(in => decoder.decode(in).map(_.map(emit)), once = true, failOnErr = false) + ) + + /** Creates a stream decoder that repeatedly decodes `A` values until decoding fails. + * If decoding fails, the read bits are not consumed and the stream decoder terminates, + * having emitted any successfully decoded values earlier. + */ + def tryMany[A](decoder: Decoder[A]): StreamDecoder[A] = + new StreamDecoder[A]( + Decode(in => decoder.decode(in).map(_.map(emit)), once = false, failOnErr = false) + ) + + /** Creates a stream decoder that fails decoding with the specified exception. */ + def raiseError(cause: Throwable): StreamDecoder[Nothing] = new StreamDecoder(Failed(cause)) + + /** Creates a stream decoder that fails decoding with the specified error. */ + def raiseError(err: Err): StreamDecoder[Nothing] = raiseError(CodecError(err)) + + /** Creates a stream decoder that reads the specified number of bits and then decodes + * them with the supplied stream decoder. Any remainder from the inner stream decoder is + * discarded. + */ + def isolate[A](bits: Long)(decoder: StreamDecoder[A]): StreamDecoder[A] = + new StreamDecoder(Isolate(bits, decoder)) + + /** Creates a stream decoder that ignores the specified number of bits. */ + def ignore(bits: Long): StreamDecoder[Nothing] = + once(codecs.ignore(bits)).flatMap(_ => empty) +} diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala new file mode 100644 index 0000000000..eabe80354d --- /dev/null +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala @@ -0,0 +1,195 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-stream, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import _root_.scodec.bits.BitVector +import _root_.scodec.{Encoder, Err} + +/** A streaming encoding process, represented as a `Stream[Pure, A] => Pull[Pure, BitVector, Option[(Stream[Pure, A], StreamEncoder[A])]]`. + */ +final class StreamEncoder[A] private (private val step: StreamEncoder.Step[A]) { self => + + /** Encode the given sequence of `A` values to a `BitVector`, raising an exception + * in the event of an encoding error. + */ + def encodeAllValid(in: Seq[A]): BitVector = { + type ET[X] = Either[Throwable, X] + encode[Fallible](Stream.emits(in)) + .compile[Fallible, ET, BitVector] + .fold(BitVector.empty)(_ ++ _) + .fold(e => throw e, identity) + } + + /** Converts this encoder to a `Pipe[F, A, BitVector]`. */ + def toPipe[F[_]: RaiseThrowable]: Pipe[F, A, BitVector] = + in => encode(in) + + /** Converts this encoder to a `Pipe[F, A, Byte]`. */ + def toPipeByte[F[_]: RaiseThrowable]: Pipe[F, A, Byte] = + in => encode(in).flatMap(bits => Stream.chunk(Chunk.byteVector(bits.bytes))) + + /** Encodes the supplied stream of `A` values in to a stream of `BitVector`. */ + def encode[F[_]: RaiseThrowable](in: Stream[F, A]): Stream[F, BitVector] = + apply(in).void.stream + + private def apply[F[_]: RaiseThrowable]( + in: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = { + def loop( + s: Stream[F, A], + encoder: StreamEncoder[A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + encoder.step[F](s).flatMap { + case Some((s, next)) => loop(s, next) + case None => Pull.pure(None) + } + loop(in, this) + } + + private def or(other: StreamEncoder[A]): StreamEncoder[A] = + new StreamEncoder[A]( + new StreamEncoder.Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + self.step(s).flatMap { + case Some(x) => Pull.pure(Some(x)) + case None => other.step(s) + } + } + ) + + /** Creates a stream encoder that first encodes with this encoder and then when complete, + * encodes the remainder with the supplied encoder. + */ + def ++(that: => StreamEncoder[A]): StreamEncoder[A] = + new StreamEncoder[A]( + new StreamEncoder.Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + self.step(s).map(_.map { case (s1, next) => (s1, next.or(that)) }) + } + ) + + /** Encodes values as long as there are more inputs. */ + def repeat: StreamEncoder[A] = this ++ repeat + + /** Transform the input type of this `StreamEncoder`. */ + def xmapc[B](f: A => B)(g: B => A): StreamEncoder[B] = + new StreamEncoder[B]( + new StreamEncoder.Step[B] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, B] + ): Pull[F, BitVector, Option[(Stream[F, B], StreamEncoder[B])]] = + self.step(s.map(g)).map(_.map { case (s1, e1) => s1.map(f) -> e1.xmapc(f)(g) }) + } + ) +} + +object StreamEncoder { + + private trait Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] + } + + /** Creates a stream encoder that consumes no values and emits no bits. */ + def empty[A]: StreamEncoder[A] = + new StreamEncoder[A](new Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + Pull.pure(None) + }) + + /** Creates a stream encoder that encodes a single value of input using the supplied encoder. */ + def once[A](encoder: Encoder[A]): StreamEncoder[A] = + new StreamEncoder[A](new Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + s.pull.uncons1.flatMap { + case None => Pull.pure(None) + case Some((a, s1)) => + encoder + .encode(a) + .fold( + e => Pull.raiseError(CodecError(e)), + b => Pull.output1(b) + ) >> Pull.pure(Some(s1 -> empty)) + } + }) + + /** Creates a stream encoder that encodes all input values using the supplied encoder. */ + def many[A](encoder: Encoder[A]): StreamEncoder[A] = once(encoder).repeat + + /** Creates a stream encoder which encodes a single value, then halts. + * Unlike `once`, if an encoding failure occurs, the resulting stream is not terminated. + */ + def tryOnce[A](encoder: Encoder[A]): StreamEncoder[A] = + new StreamEncoder[A](new Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + s.pull.uncons1.flatMap { + case None => Pull.pure(None) + case Some((a, s1)) => + encoder + .encode(a) + .fold( + _ => Pull.pure(Some(s1.cons1(a) -> empty)), + b => Pull.output1(b) >> Pull.pure(Some(s1 -> empty)) + ) + } + }) + + /** Creates a stream encoder which encodes all input values, then halts. + * Unlike `many`, if an encoding failure occurs, the resulting stream is not terminated. + */ + def tryMany[A](encoder: Encoder[A]): StreamEncoder[A] = tryOnce(encoder).repeat + + /** Creates a stream encoder that emits the given `BitVector`, then halts. */ + def emit[A](bits: BitVector): StreamEncoder[A] = + new StreamEncoder[A](new Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + Pull.output1(bits) >> Pull.pure(Some(s -> empty[A])) + }) + + /** The encoder that consumes no input and halts with the given error. */ + def raiseError[A](err: Throwable): StreamEncoder[A] = + new StreamEncoder[A](new Step[A] { + def apply[F[_]: RaiseThrowable]( + s: Stream[F, A] + ): Pull[F, BitVector, Option[(Stream[F, A], StreamEncoder[A])]] = + Pull.raiseError(err) + }) + + /** The encoder that consumes no input and halts with the given error message. */ + def raiseError[A](err: Err): StreamEncoder[A] = raiseError(CodecError(err)) +} diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala new file mode 100644 index 0000000000..0c27be3164 --- /dev/null +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-stream, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import cats.effect.SyncIO +import _root_.scodec.codecs._ + +// test for https://github.com/scodec/scodec-stream/issues/58 +class ListOfNTest extends Fs2Suite { + + val codec = listOfN(uint16, uint16) + val pipe = StreamDecoder.many(codec).toPipeByte[SyncIO] + + val ints = (1 to 7).toList + val encodedBytes = Chunk.array(codec.encode(ints).require.toByteArray) + + test("non-split chunk") { + val source = Stream.chunk(encodedBytes) + val decodedList = source.through(pipe).compile.lastOrError.unsafeRunSync() + assertEquals(decodedList, ints) + } + + property("split chunk") { + val (splitChunk1, splitChunk2) = encodedBytes.splitAt(6) + val splitSource = Stream.chunk(splitChunk1) ++ Stream.chunk(splitChunk2) + val decodedList = splitSource.through(pipe).compile.lastOrError.unsafeRunSync() + assertEquals(decodedList, ints) + } +} diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala new file mode 100644 index 0000000000..49ff60bf56 --- /dev/null +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-stream, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import _root_.scodec.codecs._ + +class SpaceLeakTest extends Fs2Suite { + + test("head of stream not retained") { + // make sure that head of stream can be garbage collected + // as we go; this also checks for stack safety + val ints = variableSizeBytes(int32, vector(int32)) + val N = 400000L + val M = 5 + val chunk = (0 until M).toVector + val dec = StreamDecoder.many(ints) + val source = Stream(ints.encode(chunk).require).repeat + val actual = + source.through(dec.toPipe[Fallible]).take(N).flatMap(Stream.emits(_)).compile.foldMonoid + assert(actual == Right((0 until M).sum * N)) + } +} diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala new file mode 100644 index 0000000000..2baedfcc57 --- /dev/null +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala @@ -0,0 +1,125 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ +// Adapted from scodec-stream, licensed under 3-clause BSD + +package fs2 +package interop +package scodec + +import org.scalacheck._ +import Prop._ +import _root_.scodec.Err +import _root_.scodec.bits._ +import _root_.scodec.codecs +import _root_.scodec.codecs._ + +class StreamCodecSuite extends Fs2Suite { + + property("many/tryMany") { + Prop.forAll { (ints: List[Int]) => + val bits = vector(int32).encode(Vector.empty[Int] ++ ints).require + val bits2 = StreamEncoder.many(int32).encodeAllValid(ints) + bits == bits2 && + StreamDecoder.many(int32).decode[Fallible](Stream(bits)).toList == Right(ints) && + StreamDecoder.tryMany(int32).decode[Fallible](Stream(bits2)).toList == Right(ints) + } + } + + test("many/tryMany insufficient") { + val bits = hex"00000001 00000002 0000".bits + assert(StreamDecoder.many(int32).decode[Fallible](Stream(bits)).toList == Right(List(1, 2))) + assert(StreamDecoder.tryMany(int32).decode[Fallible](Stream(bits)).toList == Right(List(1, 2))) + } + + test("tryMany example") { + val bits = StreamEncoder.many(int32).encodeAllValid(Vector(1, 2, 3)) + assert( + StreamDecoder.tryMany(int32).decode[Fallible](Stream(bits)).toList == Right(List(1, 2, 3)) + ) + } + + test("many + flatMap + tryMany") { + val decoder = StreamDecoder.many(bits(4)).flatMap { _ => + StreamDecoder.tryMany( + bits(4).flatMap { b => + if (b == bin"0111") codecs.fail[BitVector](Err("")) + else codecs.provide(b) + } + ) + } + val actual = decoder + .decode[Fallible](Stream.emits(hex"1a bc d7 ab 7a bc".toArray.map(BitVector(_)))) + .compile + .fold(BitVector.empty)(_ ++ _) + assert(actual == Right(hex"abcdababc".bits.drop(4))) + } + + property("isolate") { + forAll { (ints: List[Int], _: Long) => + val bits = vector(int32).encode(ints.toVector).require + val d = + StreamDecoder.many(int32).isolate(bits.size).map(_ => 0) ++ + StreamDecoder.many(int32).isolate(bits.size).map(_ => 1) + val s = Stream(bits ++ bits) + d.decode[Fallible](s).toVector == Right( + Vector.fill(ints.size)(0) ++ Vector.fill(ints.size.toInt)(1) + ) + } + } + + def genChunkSize = Gen.choose(1L, 128L) + def genSmallListOfString = Gen.choose(0, 10).flatMap(n => Gen.listOfN(n, Gen.alphaStr)) + + property("list of fixed size strings") { + forAll(genSmallListOfString, genChunkSize) { (strings: List[String], chunkSize: Long) => + val bits = StreamEncoder.many(utf8_32).encodeAllValid(strings) + val chunks = Stream.emits(BitVector.GroupedOp(bits).grouped(chunkSize).toSeq).covary[Fallible] + chunks.through(StreamDecoder.many(utf8_32).toPipe).toList == Right(strings) + } + } + + def genSmallListOfInt = Gen.choose(0, 10).flatMap(n => Gen.listOfN(n, Arbitrary.arbitrary[Int])) + property("list of fixed size ints") { + forAll(genSmallListOfInt, genChunkSize) { (ints: List[Int], chunkSize: Long) => + val bits = StreamEncoder.many(int32).encodeAllValid(ints) + val chunks = Stream.emits(BitVector.GroupedOp(bits).grouped(chunkSize).toSeq).covary[Fallible] + chunks.through(StreamDecoder.many(int32).toPipe).toList == Right(ints) + } + } + + property("encode - emit") { + forAll { (toEmit: Int, ints: List[Int]) => + val bv: BitVector = int32.encode(toEmit).require + val e: StreamEncoder[Int] = StreamEncoder.emit[Int](bv) + e.encode(Stream.emits(ints).covary[Fallible]).compile.fold(BitVector.empty)(_ ++ _) == Right( + bv + ) + } + } + + test("encode - tryOnce") { + assert( + (StreamEncoder.tryOnce(codecs.fail[Int](Err("error"))) ++ StreamEncoder.many(int8)) + .encode(Stream(1, 2).covary[Fallible]) + .toList == Right(List(hex"01".bits, hex"02".bits)) + ) + } +} From cc14302731a0b7c709619662d8a3308ea5a291ee Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 08:45:35 -0400 Subject: [PATCH 11/33] Add scodec interop example --- build.sbt | 2 +- .../scala/fs2/interop/scodec/CodecError.scala | 1 + .../fs2/interop/scodec/StreamDecoder.scala | 1 + .../fs2/interop/scodec/StreamEncoder.scala | 1 + .../fs2/interop/scodec/ListOfNSuite.scala | 1 + .../fs2/interop/scodec/MpegExample.scala | 269 ++++++++++++++++++ .../fs2/interop/scodec/SpaceLeakTest.scala | 1 + .../fs2/interop/scodec/StreamCodecSuite.scala | 1 + 8 files changed, 276 insertions(+), 1 deletion(-) create mode 100644 scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala diff --git a/build.sbt b/build.sbt index 23b8d59c45..376dd72054 100644 --- a/build.sbt +++ b/build.sbt @@ -288,7 +288,7 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), osgiSettings ) - .dependsOn(core % "compile->compile;test->test") + .dependsOn(core % "compile->compile;test->test", io % "test") lazy val reactiveStreams = project .in(file("reactive-streams")) diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala index b5fe20eedb..155eafb54e 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/CodecError.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala index 3759d08067..87eaa5e55c 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-protocols, licensed under 3-clause BSD package fs2 diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala index eabe80354d..9acb7118e2 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-stream, licensed under 3-clause BSD package fs2 diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala index 0c27be3164..275766d519 100644 --- a/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-stream, licensed under 3-clause BSD package fs2 diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala new file mode 100644 index 0000000000..b7b952569e --- /dev/null +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala @@ -0,0 +1,269 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +package fs2.interop.scodec.examples + +import scala.concurrent.duration._ + +import cats.effect.{Clock, IO, IOApp} + +import scodec.bits.{BitVector, ByteOrdering, ByteVector} +import scodec.{Attempt, Codec, DecodeResult, Err, SizeBound, codecs} + +import fs2.io.file.{Files, Path} +import fs2.interop.scodec.{CodecError, StreamDecoder} + +object Mpeg extends IOApp.Simple { + import PcapCodec._ + import MpegCodecs._ + + def run: IO[Unit] = { + val streamThroughRecordsOnly: StreamDecoder[MpegPacket] = { + val pcapRecordStreamDecoder: StreamDecoder[PcapRecord] = + StreamDecoder.once(pcapHeader).flatMap { header => + StreamDecoder.many(pcapRecord(header.ordering)) + } + + val mpegPcapDecoder: StreamDecoder[MpegPacket] = pcapRecordStreamDecoder.flatMap { record => + // Drop 22 byte ethernet frame header and 20 byte IPv4/udp header + val datagramPayloadBits = record.data.drop(22 * 8).drop(20 * 8) + val packets = codecs.vector(Codec[MpegPacket]).decode(datagramPayloadBits).map(_.value) + packets.fold(e => StreamDecoder.raiseError(CodecError(e)), StreamDecoder.emits(_)) + } + + mpegPcapDecoder + } + + val streamier: StreamDecoder[MpegPacket] = for { + header <- StreamDecoder.once(pcapHeader) + packet <- StreamDecoder.many(pcapRecordHeader(header.ordering)).flatMap { recordHeader => + StreamDecoder.isolate(recordHeader.includedLength * 8) { + // Drop 22 byte ethernet frame header and 20 byte IPv4/udp header + StreamDecoder.ignore((22 + 20) * 8) ++ + // bail on this record if we fail to parse an `MpegPacket` from it + StreamDecoder.tryMany(mpegPacket) + } + } + } yield packet + + def time[A](a: IO[A]): IO[(FiniteDuration, A)] = + for { + start <- Clock[IO].realTime + result <- a + stop <- Clock[IO].realTime + } yield (stop - start, result) + + val filePath = Path("path/to/file") + + def countElements(decoder: StreamDecoder[_]): IO[Long] = + Files[IO] + .readAll(filePath) + .through(decoder.toPipeByte) + .compile + .count + + def run(label: String, decoder: StreamDecoder[_]): IO[Unit] = + time(countElements(decoder)).flatMap { case (elapsed, cnt) => + IO.println(s"$label stream packet count: $cnt (took $elapsed)") + } + + run("coarse-grained", streamThroughRecordsOnly) >> run("fine-grained", streamier) + } +} + +/** Processes libpcap files. + * + * @see http://wiki.wireshark.org/Development/LibpcapFileFormat + */ +object PcapCodec { + import scodec.codecs._ + + private val MagicNumber = 0xa1b2c3d4L + private val MagicNumberRev = 0xd4c3b2a1L + + private val byteOrdering: Codec[ByteOrdering] = new Codec[ByteOrdering] { + def sizeBound = SizeBound.exact(32) + + def encode(bo: ByteOrdering) = + endiannessDependent(uint32, uint32L)(bo).encode(MagicNumber) + + def decode(buf: BitVector) = + uint32.decode(buf).flatMap { + case DecodeResult(MagicNumber, rest) => + Attempt.successful(DecodeResult(ByteOrdering.BigEndian, rest)) + case DecodeResult(MagicNumberRev, rest) => + Attempt.successful(DecodeResult(ByteOrdering.LittleEndian, rest)) + case DecodeResult(other, _) => + Attempt.failure( + Err(s"unable to detect byte ordering due to unrecognized magic number $other") + ) + } + + override def toString = "byteOrdering" + } + + def gint16(implicit ordering: ByteOrdering): Codec[Int] = + if (ordering == ByteOrdering.BigEndian) int16 else int16L + def guint16(implicit ordering: ByteOrdering): Codec[Int] = + if (ordering == ByteOrdering.BigEndian) uint16 else uint16L + def gint32(implicit ordering: ByteOrdering): Codec[Int] = + if (ordering == ByteOrdering.BigEndian) int32 else int32L + def guint32(implicit ordering: ByteOrdering): Codec[Long] = + if (ordering == ByteOrdering.BigEndian) uint32 else uint32L + + case class PcapHeader( + ordering: ByteOrdering, + versionMajor: Int, + versionMinor: Int, + thiszone: Int, + sigfigs: Long, + snaplen: Long, + network: Long + ) + + val pcapHeader: Codec[PcapHeader] = "global-header" | + ("magic_number" | byteOrdering) + .flatPrepend { implicit ordering => + ("version_major" | guint16) :: + ("version_minor" | guint16) :: + ("thiszone" | gint32) :: + ("sigfigs" | guint32) :: + ("snaplen" | guint32) :: + ("network" | guint32) + } + .as[PcapHeader] + + case class PcapRecordHeader( + timestampSeconds: Long, + timestampMicros: Long, + includedLength: Long, + originalLength: Long + ) { + def timestamp: Double = timestampSeconds + (timestampMicros / (1.second.toMicros.toDouble)) + } + + implicit def pcapRecordHeader(implicit ordering: ByteOrdering): Codec[PcapRecordHeader] = { + ("ts_sec" | guint32) :: + ("ts_usec" | guint32) :: + ("incl_len" | guint32) :: + ("orig_len" | guint32) + }.as[PcapRecordHeader] + + case class PcapRecord(header: PcapRecordHeader, data: BitVector) + + implicit def pcapRecord(implicit ordering: ByteOrdering): Codec[PcapRecord] = + ("record_header" | pcapRecordHeader) + .flatPrepend { hdr => + ("record_data" | bits(hdr.includedLength * 8)).hlist + } + .as[PcapRecord] + + case class PcapFile(header: PcapHeader, records: Vector[PcapRecord]) + + implicit val pcapFile: Codec[PcapFile] = + pcapHeader + .flatPrepend { hdr => + vector(pcapRecord(hdr.ordering)).hlist + } + .as[PcapFile] +} + +object MpegCodecs { + import scodec.codecs._ + + // Define case classes that describe MPEG packets and define an HList iso for each + + case class TransportStreamHeader( + transportStringIndicator: Boolean, + payloadUnitStartIndicator: Boolean, + transportPriority: Boolean, + pid: Int, + scramblingControl: Int, + adaptationFieldControl: Int, + continuityCounter: Int + ) { + def adaptationFieldIncluded: Boolean = adaptationFieldControl >= 2 + def payloadIncluded: Boolean = adaptationFieldControl == 1 || adaptationFieldControl == 3 + } + + case class AdaptationFieldFlags( + discontinuity: Boolean, + randomAccess: Boolean, + priority: Boolean, + pcrFlag: Boolean, + opcrFlag: Boolean, + splicingPointFlag: Boolean, + transportPrivateDataFlag: Boolean, + adaptationFieldExtension: Boolean + ) + + case class AdaptationField( + flags: AdaptationFieldFlags, + pcr: Option[BitVector], + opcr: Option[BitVector], + spliceCountdown: Option[Int] + ) + + case class MpegPacket( + header: TransportStreamHeader, + adaptationField: Option[AdaptationField], + payload: Option[ByteVector] + ) + + implicit val transportStreamHeader: Codec[TransportStreamHeader] = { + ("syncByte" | constant(0x47)) ~> + (("transportStringIndicator" | bool) :: + ("payloadUnitStartIndicator" | bool) :: + ("transportPriority" | bool) :: + ("pid" | uint(13)) :: + ("scramblingControl" | uint2) :: + ("adaptationFieldControl" | uint2) :: + ("continuityCounter" | uint4)) + }.as[TransportStreamHeader] + + implicit val adaptationFieldFlags: Codec[AdaptationFieldFlags] = { + ("discontinuity" | bool) :: + ("randomAccess" | bool) :: + ("priority" | bool) :: + ("pcrFlag" | bool) :: + ("opcrFlag" | bool) :: + ("splicingPointFlag" | bool) :: + ("transportPrivateDataFlag" | bool) :: + ("adaptationFieldExtension" | bool) + }.as[AdaptationFieldFlags] + + implicit val adaptationField: Codec[AdaptationField] = + ("adaptation_flags" | adaptationFieldFlags) + .flatPrepend { flags => + ("pcr" | conditional(flags.pcrFlag, bits(48))) :: + ("opcr" | conditional(flags.opcrFlag, bits(48))) :: + ("spliceCountdown" | conditional(flags.splicingPointFlag, int8)) + } + .as[AdaptationField] + + implicit val mpegPacket: Codec[MpegPacket] = + ("header" | transportStreamHeader) + .flatPrepend { hdr => + ("adaptation_field" | conditional(hdr.adaptationFieldIncluded, adaptationField)) :: + ("payload" | conditional(hdr.payloadIncluded, bytes(184))) + } + .as[MpegPacket] +} diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala index 49ff60bf56..891e0a3810 100644 --- a/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/SpaceLeakTest.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-stream, licensed under 3-clause BSD package fs2 diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala index 2baedfcc57..db6e86d117 100644 --- a/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/StreamCodecSuite.scala @@ -18,6 +18,7 @@ * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. */ + // Adapted from scodec-stream, licensed under 3-clause BSD package fs2 From ce368a97ea17aee3c5f84b6dfa3c05ae61a07c04 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 08:48:10 -0400 Subject: [PATCH 12/33] Update workflow --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index c3531c9a8b..a1a725cba1 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -67,7 +67,7 @@ jobs: run: sbt ++${{ matrix.scala }} microsite/mdoc - name: Compress target directories - run: tar cf targets.tar target node/js/target core/js/target core/jvm/target io/js/target reactive-streams/target io/jvm/target benchmark/target project/target + run: tar cf targets.tar target node/js/target core/js/target core/jvm/target scodec/jvm/target scodec/js/target io/js/target reactive-streams/target io/jvm/target benchmark/target project/target - name: Upload target directories uses: actions/upload-artifact@v2 From 478deaa989a4f62651908d149989029b900537a6 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 09:32:59 -0400 Subject: [PATCH 13/33] Add instances for stream encoder and decoder --- .../fs2/interop/scodec/StreamDecoder.scala | 29 +++++++++++++++++++ .../fs2/interop/scodec/StreamEncoder.scala | 7 +++++ 2 files changed, 36 insertions(+) diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala index 87eaa5e55c..30257e9a73 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala @@ -28,6 +28,8 @@ package scodec import _root_.scodec.{Attempt, DecodeResult, Decoder, Err, codecs} import _root_.scodec.bits.BitVector +import cats.MonadThrow + /** Supports binary decoding of a stream that emits elements as they are decoded. * * The main purpose of using a `StreamDecoder` over a `scodec.Decoder` is mixing @@ -137,6 +139,19 @@ final class StreamDecoder[+A] private (private val step: StreamDecoder.Step[A]) } ) + def handleErrorWith[A2 >: A](f: Throwable => StreamDecoder[A2]): StreamDecoder[A2] = + new StreamDecoder[A2]( + self.step match { + case Empty => Empty + case Result(a) => Result(a) + case Failed(cause) => f(cause).step + case Decode(g, once, failOnErr) => + Decode(in => g(in).map(_.map(_.handleErrorWith(f))), once, failOnErr) + case Isolate(bits, decoder) => Isolate(bits, decoder.handleErrorWith(f)) + case Append(x, y) => Append(x.handleErrorWith(f), () => y().handleErrorWith(f)) + } + ) + /** Maps the supplied function over each output of this decoder. */ def map[B](f: A => B): StreamDecoder[B] = flatMap(a => StreamDecoder.emit(f(a))) @@ -259,4 +274,18 @@ object StreamDecoder { /** Creates a stream decoder that ignores the specified number of bits. */ def ignore(bits: Long): StreamDecoder[Nothing] = once(codecs.ignore(bits)).flatMap(_ => empty) + + implicit val instance: MonadThrow[StreamDecoder] = new MonadThrow[StreamDecoder] { + def pure[A](a: A) = StreamDecoder.emit(a) + def flatMap[A, B](da: StreamDecoder[A])(f: A => StreamDecoder[B]) = da.flatMap(f) + def tailRecM[A, B](a: A)(f: A => StreamDecoder[Either[A, B]]): StreamDecoder[B] = + f(a).flatMap { + case Left(a) => tailRecM(a)(f) + case Right(b) => pure(b) + } + def handleErrorWith[A](da: StreamDecoder[A])(f: Throwable => StreamDecoder[A]): StreamDecoder[A] = + da.handleErrorWith(f) + def raiseError[A](e: Throwable): StreamDecoder[A] = + StreamDecoder.raiseError(e) + } } diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala index 9acb7118e2..4a244c58a3 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamEncoder.scala @@ -28,6 +28,8 @@ package scodec import _root_.scodec.bits.BitVector import _root_.scodec.{Encoder, Err} +import cats.Invariant + /** A streaming encoding process, represented as a `Stream[Pure, A] => Pull[Pure, BitVector, Option[(Stream[Pure, A], StreamEncoder[A])]]`. */ final class StreamEncoder[A] private (private val step: StreamEncoder.Step[A]) { self => @@ -193,4 +195,9 @@ object StreamEncoder { /** The encoder that consumes no input and halts with the given error message. */ def raiseError[A](err: Err): StreamEncoder[A] = raiseError(CodecError(err)) + + implicit val instance: Invariant[StreamEncoder] = new Invariant[StreamEncoder] { + def imap[A, B](fa: StreamEncoder[A])(f: A => B)(g: B => A): StreamEncoder[B] = + fa.xmapc(f)(g) + } } From 98afdb98bc8359aa81797ff2e178c871d641a099 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 09:33:29 -0400 Subject: [PATCH 14/33] Scalafmt --- .../main/scala/fs2/interop/scodec/StreamDecoder.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala index 30257e9a73..e8a8c66485 100644 --- a/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala +++ b/scodec/shared/src/main/scala/fs2/interop/scodec/StreamDecoder.scala @@ -139,10 +139,10 @@ final class StreamDecoder[+A] private (private val step: StreamDecoder.Step[A]) } ) - def handleErrorWith[A2 >: A](f: Throwable => StreamDecoder[A2]): StreamDecoder[A2] = + def handleErrorWith[A2 >: A](f: Throwable => StreamDecoder[A2]): StreamDecoder[A2] = new StreamDecoder[A2]( self.step match { - case Empty => Empty + case Empty => Empty case Result(a) => Result(a) case Failed(cause) => f(cause).step case Decode(g, once, failOnErr) => @@ -280,10 +280,12 @@ object StreamDecoder { def flatMap[A, B](da: StreamDecoder[A])(f: A => StreamDecoder[B]) = da.flatMap(f) def tailRecM[A, B](a: A)(f: A => StreamDecoder[Either[A, B]]): StreamDecoder[B] = f(a).flatMap { - case Left(a) => tailRecM(a)(f) + case Left(a) => tailRecM(a)(f) case Right(b) => pure(b) } - def handleErrorWith[A](da: StreamDecoder[A])(f: Throwable => StreamDecoder[A]): StreamDecoder[A] = + def handleErrorWith[A](da: StreamDecoder[A])( + f: Throwable => StreamDecoder[A] + ): StreamDecoder[A] = da.handleErrorWith(f) def raiseError[A](e: Throwable): StreamDecoder[A] = StreamDecoder.raiseError(e) From 94e02f9d32f3df28aac4888a060c2c10377810a2 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 09:38:38 -0400 Subject: [PATCH 15/33] Scalafmt --- build.sbt | 20 ++++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/build.sbt b/build.sbt index 376dd72054..0581e3d343 100644 --- a/build.sbt +++ b/build.sbt @@ -1,10 +1,10 @@ import com.typesafe.tools.mima.core._ import sbtcrossproject.crossProject -addCommandAlias("fmt", "; compile:scalafmt; test:scalafmt; it:scalafmt; scalafmtSbt") +addCommandAlias("fmt", "; Compile/scalafmt; Test/scalafmt; IntegrationTest/scalafmt; scalafmtSbt") addCommandAlias( "fmtCheck", - "; compile:scalafmtCheck; test:scalafmtCheck; it:scalafmtCheck; scalafmtSbtCheck" + "; Compile/scalafmtCheck; Test/scalafmtCheck; IntegrationTest/scalafmtCheck; scalafmtSbtCheck" ) addCommandAlias("testJVM", ";rootJVM/test") addCommandAlias("testJS", "rootJS/test") @@ -144,7 +144,17 @@ ThisBuild / mimaBinaryIssueFilters ++= Seq( lazy val root = project .in(file(".")) .enablePlugins(NoPublishPlugin, SonatypeCiReleasePlugin) - .aggregate(coreJVM, coreJS, io.jvm, node.js, io.js, scodec.jvm, scodec.js, reactiveStreams, benchmark) + .aggregate( + coreJVM, + coreJS, + io.jvm, + node.js, + io.js, + scodec.jvm, + scodec.js, + reactiveStreams, + benchmark + ) lazy val rootJVM = project .in(file(".")) @@ -274,7 +284,9 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) .enablePlugins(SbtOsgi) .settings( name := "fs2-scodec", - libraryDependencies += "org.scodec" %% "scodec-core" % (if (scalaVersion.value.startsWith("2.")) "1.11.8" else "2.0.0"), + libraryDependencies += "org.scodec" %% "scodec-core" % (if (scalaVersion.value.startsWith("2.")) + "1.11.8" + else "2.0.0"), OsgiKeys.exportPackage := Seq("fs2.interop.scodec.*"), OsgiKeys.privatePackage := Seq(), OsgiKeys.importPackage := { From 6e49175117150f6fa4babe32dbfd30840b3d8253 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 10:05:09 -0400 Subject: [PATCH 16/33] Fix js dependency --- build.sbt | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/build.sbt b/build.sbt index 0581e3d343..3270363c2f 100644 --- a/build.sbt +++ b/build.sbt @@ -282,11 +282,14 @@ lazy val io = crossProject(JVMPlatform, JSPlatform) lazy val scodec = crossProject(JVMPlatform, JSPlatform) .in(file("scodec")) .enablePlugins(SbtOsgi) + .jsConfigure(_.enablePlugins(ScalaJSBundlerPlugin)) .settings( name := "fs2-scodec", - libraryDependencies += "org.scodec" %% "scodec-core" % (if (scalaVersion.value.startsWith("2.")) - "1.11.8" - else "2.0.0"), + libraryDependencies += "org.scodec" %%% "scodec-core" % (if ( + scalaVersion.value.startsWith("2.") + ) + "1.11.8" + else "2.0.0"), OsgiKeys.exportPackage := Seq("fs2.interop.scodec.*"), OsgiKeys.privatePackage := Seq(), OsgiKeys.importPackage := { @@ -300,6 +303,9 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), osgiSettings ) + .jsSettings( + scalaJSLinkerConfig ~= (_.withModuleKind(ModuleKind.CommonJSModule)) + ) .dependsOn(core % "compile->compile;test->test", io % "test") lazy val reactiveStreams = project From 640069dc95e864e50a02b16787822b3b9a95a235 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 11:09:50 -0400 Subject: [PATCH 17/33] Fix mima on scodec module --- build.sbt | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index 3270363c2f..93b9bf5037 100644 --- a/build.sbt +++ b/build.sbt @@ -301,7 +301,10 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) ) }, OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), - osgiSettings + osgiSettings, + mimaPreviousArtifacts := mimaPreviousArtifacts.value.filter { v => + VersionNumber(v.revision).matchesSemVer(SemanticSelector(">3.1.1")) + } ) .jsSettings( scalaJSLinkerConfig ~= (_.withModuleKind(ModuleKind.CommonJSModule)) From bd4b611da6780801f86e7f00b3c417d8bf7acfce Mon Sep 17 00:00:00 2001 From: mpilquist Date: Tue, 7 Sep 2021 11:32:55 -0400 Subject: [PATCH 18/33] Fix compilation on Scala 3 --- .../src/test/scala/fs2/interop/scodec/ListOfNSuite.scala | 2 +- .../src/test/scala/fs2/interop/scodec/MpegExample.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala index 275766d519..2995313d3a 100644 --- a/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/ListOfNSuite.scala @@ -43,7 +43,7 @@ class ListOfNTest extends Fs2Suite { assertEquals(decodedList, ints) } - property("split chunk") { + test("split chunk") { val (splitChunk1, splitChunk2) = encodedBytes.splitAt(6) val splitSource = Stream.chunk(splitChunk1) ++ Stream.chunk(splitChunk2) val decodedList = splitSource.through(pipe).compile.lastOrError.unsafeRunSync() diff --git a/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala b/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala index b7b952569e..bb3e798159 100644 --- a/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala +++ b/scodec/shared/src/test/scala/fs2/interop/scodec/MpegExample.scala @@ -102,8 +102,10 @@ object PcapCodec { private val byteOrdering: Codec[ByteOrdering] = new Codec[ByteOrdering] { def sizeBound = SizeBound.exact(32) - def encode(bo: ByteOrdering) = - endiannessDependent(uint32, uint32L)(bo).encode(MagicNumber) + def encode(bo: ByteOrdering) = { + implicit val implicitByteOrdering: ByteOrdering = bo + endiannessDependent(uint32, uint32L).encode(MagicNumber) + } def decode(buf: BitVector) = uint32.decode(buf).flatMap { From 1b067e10b2c3cd2b1c2a7e154a59a0762ef055d1 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 24 Sep 2021 08:13:47 -0400 Subject: [PATCH 19/33] Add Strong instance for Scan --- core/shared/src/main/scala/fs2/Scan.scala | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index 8f03f6f568..46f41fa6ee 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -24,6 +24,7 @@ package fs2 import cats.{Contravariant, Functor} import cats.data.AndThen +import cats.arrow.Strong /** A stateful transformation of the elements of a stream. * @@ -106,6 +107,15 @@ final class Scan[S, -I, +O]( onComplete_ ) + def dimap[I2, O2](g: I2 => I)(f: O => O2): Scan[S, I2, O2] = + Scan[S, I2, O2](initial)( + { (s, i2) => + val (s2, os) = transform(s, g(i2)) + (s2, os.map(f)) + }, + onComplete_.andThen(_.map(f)) + ) + /** Transforms the state type. */ def imapState[S2](g: S => S2)(f: S2 => S): Scan[S2, I, O] = Scan[S2, I, O](g(initial))( @@ -227,7 +237,7 @@ object Scan { def stateless[I, O](f: I => Chunk[O]): Scan[Unit, I, O] = stateful[Unit, I, O](())((u, i) => (u, f(i))) - def lift[I, O](f: I => O): Scan[Unit, I, O] = + def lift2[I, O](f: I => O): Scan[Unit, I, O] = stateless(i => Chunk.singleton(f(i))) implicit def functor[S, I]: Functor[Scan[S, I, *]] = @@ -239,4 +249,11 @@ object Scan { new Contravariant[Scan[S, *, O]] { def contramap[I, I2](s: Scan[S, I, O])(f: I2 => I) = s.contramap(f) } + + implicit def strong[S]: Strong[Scan[S, *, *]] = new Strong[Scan[S, *, *]] { + def first[A, B, C](fa: Scan[S, A, B]): Scan[S, (A, C), (B, C)] = fa.first + def second[A, B, C](fa: Scan[S, A, B]): Scan[S, (C, A), (C, B)] = fa.second + def dimap[A, B, C, D](fab: Scan[S, A, B])(f: C => A)(g: B => D): Scan[S,C,D] = + fab.dimap(f)(g) + } } From 1e482e3d99b135d902359c47d14704991e798160 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 24 Sep 2021 08:18:08 -0400 Subject: [PATCH 20/33] Scalafmt --- core/shared/src/main/scala/fs2/Scan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index 46f41fa6ee..65f616edb5 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -253,7 +253,7 @@ object Scan { implicit def strong[S]: Strong[Scan[S, *, *]] = new Strong[Scan[S, *, *]] { def first[A, B, C](fa: Scan[S, A, B]): Scan[S, (A, C), (B, C)] = fa.first def second[A, B, C](fa: Scan[S, A, B]): Scan[S, (C, A), (C, B)] = fa.second - def dimap[A, B, C, D](fab: Scan[S, A, B])(f: C => A)(g: B => D): Scan[S,C,D] = + def dimap[A, B, C, D](fab: Scan[S, A, B])(f: C => A)(g: B => D): Scan[S, C, D] = fab.dimap(f)(g) } } From 53ff646136dfa19c5c622ea332d4ca7ae8eabae1 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 24 Sep 2021 08:32:22 -0400 Subject: [PATCH 21/33] Restore Scan.lift --- core/shared/src/main/scala/fs2/Scan.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index 65f616edb5..d9eb7e4170 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -237,7 +237,7 @@ object Scan { def stateless[I, O](f: I => Chunk[O]): Scan[Unit, I, O] = stateful[Unit, I, O](())((u, i) => (u, f(i))) - def lift2[I, O](f: I => O): Scan[Unit, I, O] = + def lift[I, O](f: I => O): Scan[Unit, I, O] = stateless(i => Chunk.singleton(f(i))) implicit def functor[S, I]: Functor[Scan[S, I, *]] = From 3c8d490c0a2d06b6c02539511e946f493e037d90 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 24 Sep 2021 15:13:17 -0400 Subject: [PATCH 22/33] Review comments --- build.sbt | 2 +- core/shared/src/main/scala/fs2/Scan.scala | 6 +++--- core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala | 6 ++---- .../src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala | 2 +- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/build.sbt b/build.sbt index 28e7a9ed09..9feb41dfd7 100644 --- a/build.sbt +++ b/build.sbt @@ -309,7 +309,7 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), osgiSettings, mimaPreviousArtifacts := mimaPreviousArtifacts.value.filter { v => - VersionNumber(v.revision).matchesSemVer(SemanticSelector(">3.1.1")) + VersionNumber(v.revision).matchesSemVer(SemanticSelector(">3.2.0")) } ) .jsSettings( diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index d9eb7e4170..a18d46bb20 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -187,7 +187,7 @@ final class Scan[S, -I, +O]( * are fed through this scan while elements on the right are fed through the * suppplied scan. The outputs are joined together. */ - def or[S2, I2, O2 >: O](that: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], O2] = + def choice[S2, I2, O2 >: O](that: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], O2] = Scan[(S, S2), Either[I, I2], O2]((initial, that.initial))( { case ((s, s2), e) => e match { @@ -202,8 +202,8 @@ final class Scan[S, -I, +O]( { case (s, s2) => onComplete(s) ++ that.onComplete(s2) } ) - /** Like [[or]] but the output elements are kept separate. */ - def either[S2, I2, O2](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], Either[O, O2]] = + /** Like [[choice]] but the output elements are kept separate. */ + def choose[S2, I2, O2](t: Scan[S2, I2, O2]): Scan[(S, S2), Either[I, I2], Either[O, O2]] = Scan[(S, S2), Either[I, I2], Either[O, O2]]((initial, t.initial))( { case ((s, s2), e) => e match { diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index e0115806e1..b385308a78 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -70,12 +70,10 @@ object TimeSeries { case Some((hd, tl)) => hd.indexWhere(_.time >= nextTick) match { case None => - if (hd.isEmpty) Pull.pure(()) - else Pull.output(hd.map(_.toTimeSeriesValue)) >> go(nextTick, tl) + Pull.output(hd.map(_.toTimeSeriesValue)) >> go(nextTick, tl) case Some(idx) => val (prefix, suffix) = hd.splitAt(idx) - val out = - if (prefix.isEmpty) Pull.pure(()) else Pull.output(prefix.map(_.toTimeSeriesValue)) + val out = Pull.output(prefix.map(_.toTimeSeriesValue)) // we know suffix is non-empty and suffix.head has a time >= next tick time val next = suffix(0) val tickCount = diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala index bab85c5a24..ce8ae67fe1 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala @@ -33,7 +33,7 @@ class TimeSeriesTransducerTest extends Fs2Suite { test("support combining two transducers via an either") { val add1: Scan[Unit, Int, Int] = Scan.lift(_ + 1) val add2: Scan[Unit, Int, Int] = Scan.lift(_ + 2) - val x: Scan[Unit, Either[Int, Int], Int] = add1.or(add2).imapState(_._1)(u => (u, u)) + val x: Scan[Unit, Either[Int, Int], Int] = add1.choice(add2).imapState(_._1)(u => (u, u)) val source: TimeSeries[Pure, Either[Int, Int]] = Stream( Right(1).at(0.seconds), From b1475f35ac69681a02b8e4279b8760afc1636e5f Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Wed, 20 Oct 2021 21:01:08 -0400 Subject: [PATCH 23/33] Remove type aliases in TimeSeries package --- core/shared/src/main/scala/fs2/Scan.scala | 2 +- .../scala/fs2/timeseries/TimeSeries.scala | 74 +++++++++++++------ .../scala/fs2/timeseries/TimeSeriesPipe.scala | 39 ---------- .../fs2/timeseries/TimeSeriesValue.scala | 41 ---------- .../scala/fs2/timeseries/TimeStamped.scala | 4 +- .../main/scala/fs2/timeseries/package.scala | 48 ------------ .../fs2/timeseries/TimeSeriesPipeSuite.scala | 53 ------------- .../fs2/timeseries/TimeSeriesSuite.scala | 21 ++++++ 8 files changed, 74 insertions(+), 208 deletions(-) delete mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala delete mode 100644 core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala delete mode 100644 core/shared/src/main/scala/fs2/timeseries/package.scala delete mode 100644 core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala diff --git a/core/shared/src/main/scala/fs2/Scan.scala b/core/shared/src/main/scala/fs2/Scan.scala index a18d46bb20..992a35cd7c 100644 --- a/core/shared/src/main/scala/fs2/Scan.scala +++ b/core/shared/src/main/scala/fs2/Scan.scala @@ -62,7 +62,7 @@ final class Scan[S, -I, +O]( def onComplete(s: S): Chunk[O] = onComplete_(s) /** Converts this scan to a pipe. */ - def toPipe[F[_]]: Pipe[F, I, O] = + def toPipe[F[_]]: Stream[F, I] => Stream[F, O] = _.pull .scanChunks(initial)(transformAccumulate) .flatMap(state => Pull.output(onComplete(state))) diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index b385308a78..6c756a11dc 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -28,7 +28,17 @@ import scala.concurrent.duration._ import cats.effect.Temporal -/** Companion for [[TimeSeries]]. */ +/** A time series is a stream of timestamped values or clock ticks. + * + * Values are represented as `Some` values in a `TimeStamped[Option[A]]`, whereas + * clock ticks are represented as `None`s. This encoding allows for an indication + * of time passage with no observed values. + * + * Generally, time series appear in increasing order, and many combinators that work with + * time series will rely on that. For streams that are globally ordered, but not locally ordered, + * i.e., near adjacent values might be out of order but values at great distance from each other + * are ordered, consider using `TimeStamped.reorderLocally` to adjust. + */ object TimeSeries { /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ @@ -36,20 +46,36 @@ object TimeSeries { source: Stream[F, TimeStamped[A]], tickPeriod: FiniteDuration, reorderOver: FiniteDuration - ): TimeSeries[F, A] = { - val src: TimeSeries[F, A] = source.map(tsa => tsa.map(Some(_): Option[A])) - val ticks: TimeSeries[F, Nothing] = timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) + ): Stream[F, TimeStamped[Option[A]]] = { + val src: Stream[F, TimeStamped[Option[A]]] = source.map(tsa => tsa.map(Some(_): Option[A])) + val ticks: Stream[F, TimeStamped[Option[Nothing]]] = timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) src.merge(ticks).through(TimeStamped.reorderLocally(reorderOver)) } /** Stream of either time ticks (spaced by `tickPeriod`) or values from the source stream. */ - def lift[F[_]: Temporal, A]( + def timePulled[F[_]: Temporal, A]( source: Stream[F, A], tickPeriod: FiniteDuration, reorderOver: FiniteDuration - ): TimeSeries[F, A] = + ): Stream[F, TimeStamped[Option[A]]] = apply(source.map(TimeStamped.unsafeNow), tickPeriod, reorderOver) + /** Lifts a function from `A => B` to a time series pipe. */ + def lift[F[_], A, B](f: A => B): Stream[F, TimeStamped[Option[A]]] => Stream[F, TimeStamped[Option[B]]] = + _.map(_.map(_.map(f))) + + /** Time series pipe which discards right values. */ + def drainRight[F[_], L, R]: Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[L]]] = _.collect { + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[L]]] + case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) + } + + /** Time series pipe which discards left values. */ + def drainLeft[F[_], L, R]: Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[R]]] = _.collect { + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[R]]] + case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) + } + /** Stream of time ticks spaced by `tickPeriod`. */ private def timeTicks[F[_]: Temporal](tickPeriod: FiniteDuration): Stream[F, TimeStamped[Unit]] = Stream.awakeEvery[F](tickPeriod).map(_ => TimeStamped.unsafeNow(())) @@ -58,28 +84,28 @@ object TimeSeries { * to a stream of timestamped ticks or values, where a tick is emitted every `tickPeriod`. * Ticks are emitted between values from the source stream. */ - def interpolateTicks[A]( + def interpolateTicks[F[_], A]( tickPeriod: FiniteDuration - ): Pipe[Pure, TimeStamped[A], TimeSeriesValue[A]] = { + ): Stream[F, TimeStamped[A]] => Stream[F, TimeStamped[Option[A]]] = { def go( nextTick: FiniteDuration, - s: Stream[Pure, TimeStamped[A]] - ): Pull[Pure, TimeSeriesValue[A], Unit] = { + s: Stream[F, TimeStamped[A]] + ): Pull[F, TimeStamped[Option[A]], Unit] = { def tickTime(x: Int) = nextTick + (x * tickPeriod) s.pull.uncons.flatMap { case Some((hd, tl)) => hd.indexWhere(_.time >= nextTick) match { case None => - Pull.output(hd.map(_.toTimeSeriesValue)) >> go(nextTick, tl) + Pull.output(hd.map(_.map(Some(_)))) >> go(nextTick, tl) case Some(idx) => val (prefix, suffix) = hd.splitAt(idx) - val out = Pull.output(prefix.map(_.toTimeSeriesValue)) + val out = Pull.output(prefix.map(_.map(Some(_)))) // we know suffix is non-empty and suffix.head has a time >= next tick time val next = suffix(0) val tickCount = ((next.time.toMillis - nextTick.toMillis) / tickPeriod.toMillis + 1).toInt val tickTimes = (0 until tickCount).map(tickTime) - val ticks = tickTimes.map(TimeSeriesValue.tick) + val ticks = tickTimes.map(TimeStamped.tick) val rest = Pull.output(Chunk.seq(ticks)) >> go(tickTime(tickCount), tl.cons(suffix)) out >> rest } @@ -89,38 +115,38 @@ object TimeSeries { in => in.pull.uncons1.flatMap { case Some((hd, tl)) => - Pull.output1(hd.toTimeSeriesValue) >> go(hd.time + tickPeriod, tl) + Pull.output1(hd.map(Some(_))) >> go(hd.time + tickPeriod, tl) case None => Pull.done }.stream } - /** Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]]` such that + /** Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, TimeStamped[Option[I]], TimeStamped[Option[O]]]` such that * timestamps are preserved on elements that flow through the stream. */ - def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]] = + def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, TimeStamped[Option[I]], TimeStamped[Option[O]]] = preserveTicks(TimeStamped.preserve(t)) - /** Combinator that converts a `Scan[S, TimeStamped[I], TimeStamped[O]]` in to a `Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]]` such that + /** Combinator that converts a `Scan[S, TimeStamped[I], TimeStamped[O]]` in to a `Scan[S, TimeStamped[Option[I]], TimeStamped[Option[O]]]` such that * timestamps are preserved on elements that flow through the stream. */ def preserveTicks[S, I, O]( t: Scan[S, TimeStamped[I], TimeStamped[O]] - ): Scan[S, TimeSeriesValue[I], TimeSeriesValue[O]] = + ): Scan[S, TimeStamped[Option[I]], TimeStamped[Option[O]]] = t.semilens( tsi => tsi.value .map(v => Right(TimeStamped(tsi.time, v))) - .getOrElse(Left(TimeSeriesValue.tick(tsi.time))), + .getOrElse(Left(TimeStamped.tick(tsi.time))), (_, tso) => tso.map(Some(_)) ) - /** Combinator that combines a `Scan[LS, TimeSeriesValue[L], O]` and a `Scan[RS, TimeSeriesValue[R], O]` in to a `Scan[(LS, RS), TimeSeriesVlaue[Either[L, R], O]]`. + /** Combinator that combines a `Scan[LS, TimeStamped[Option[L]], O]` and a `Scan[RS, TimeStamped[Option[R]], O]` in to a `Scan[(LS, RS), TimeSeriesVlaue[Either[L, R], O]]`. */ def choice[LS, L, RS, R, O]( - l: Scan[LS, TimeSeriesValue[L], O], - r: Scan[RS, TimeSeriesValue[R], O] - ): Scan[(LS, RS), TimeSeriesValue[Either[L, R]], O] = - Scan[(LS, RS), TimeSeriesValue[Either[L, R]], O]((l.initial, r.initial))( + l: Scan[LS, TimeStamped[Option[L]], O], + r: Scan[RS, TimeStamped[Option[R]], O] + ): Scan[(LS, RS), TimeStamped[Option[Either[L, R]]], O] = + Scan[(LS, RS), TimeStamped[Option[Either[L, R]]], O]((l.initial, r.initial))( { case ((lState, rState), tsv) => tsv match { case TimeStamped(t, Some(Left(lValue))) => diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala deleted file mode 100644 index 19e04a2201..0000000000 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesPipe.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Copyright (c) 2013 Functional Streams for Scala - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of - * this software and associated documentation files (the "Software"), to deal in - * the Software without restriction, including without limitation the rights to - * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of - * the Software, and to permit persons to whom the Software is furnished to do so, - * subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all - * copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS - * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER - * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - */ - -// Adapted from scodec-protocols, licensed under 3-clause BSD -package fs2.timeseries - -object TimeSeriesPipe { - - def lift[F[_], A, B](f: A => B): TimeSeriesPipe[F, A, B] = - _.map(_.map(_.map(f))) - - def drainRight[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], L] = _.collect { - case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeSeriesValue[L]] - case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) - } - - def drainLeft[F[_], L, R]: TimeSeriesPipe[F, Either[L, R], R] = _.collect { - case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeSeriesValue[R]] - case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) - } -} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala deleted file mode 100644 index b47d3d8738..0000000000 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeriesValue.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Copyright (c) 2013 Functional Streams for Scala - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of - * this software and associated documentation files (the "Software"), to deal in - * the Software without restriction, including without limitation the rights to - * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of - * the Software, and to permit persons to whom the Software is furnished to do so, - * subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all - * copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS - * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER - * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - */ - -// Adapted from scodec-protocols, licensed under 3-clause BSD -package fs2.timeseries - -import scala.concurrent.duration._ - -import cats.Functor -import cats.effect.kernel.Clock - -object TimeSeriesValue { - def apply[A](time: FiniteDuration, value: A): TimeSeriesValue[A] = TimeStamped(time, Some(value)) - def tick(time: FiniteDuration): TimeSeriesValue[Nothing] = TimeStamped(time, None) - - def unsafeNow[A](value: A): TimeSeriesValue[A] = TimeStamped.unsafeNow(Some(value)) - def now[F[_]: Functor: Clock, A](value: A): F[TimeSeriesValue[A]] = TimeStamped.now(Some(value)) - - def unsafeNowTick: TimeSeriesValue[Nothing] = TimeStamped.unsafeNow(None) - def nowTick[F[_]: Functor: Clock]: F[TimeSeriesValue[Nothing]] = TimeStamped.now(None) - - def lift[A](t: TimeStamped[A]): TimeSeriesValue[A] = t.map(Some.apply) -} diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index 1f254ef5bf..d4a0a10827 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -34,8 +34,6 @@ import cats.effect.kernel.{Clock, Temporal} case class TimeStamped[+A](time: FiniteDuration, value: A) { def map[B](f: A => B): TimeStamped[B] = copy(value = f(value)) def mapTime(f: FiniteDuration => FiniteDuration): TimeStamped[A] = copy(time = f(time)) - - def toTimeSeriesValue: TimeSeriesValue[A] = map(Some.apply) } object TimeStamped { @@ -45,6 +43,8 @@ object TimeStamped { def now[F[_]: Functor: Clock, A](a: A): F[TimeStamped[A]] = Clock[F].realTime.map(TimeStamped(_, a)) + def tick(time: FiniteDuration): TimeStamped[Option[Nothing]] = TimeStamped(time, None) + /** Orders values by timestamp -- values with the same timestamp are considered equal. */ def timeBasedOrdering[A]: Ordering[TimeStamped[A]] = new Ordering[TimeStamped[A]] { def compare(x: TimeStamped[A], y: TimeStamped[A]) = x.time.compareTo(y.time) diff --git a/core/shared/src/main/scala/fs2/timeseries/package.scala b/core/shared/src/main/scala/fs2/timeseries/package.scala deleted file mode 100644 index 6e59542db8..0000000000 --- a/core/shared/src/main/scala/fs2/timeseries/package.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright (c) 2013 Functional Streams for Scala - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of - * this software and associated documentation files (the "Software"), to deal in - * the Software without restriction, including without limitation the rights to - * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of - * the Software, and to permit persons to whom the Software is furnished to do so, - * subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all - * copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS - * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER - * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - */ - -// Adapted from scodec-protocols, licensed under 3-clause BSD - -package fs2 - -package object timeseries { - - /** A single value in a `TimeSeries`. Provides a timestamp along with either a value of type `A` or - * a clock tick (represented by a none). - */ - type TimeSeriesValue[+A] = TimeStamped[Option[A]] - - /** A stream of timestamped values or clock ticks. - * - * Values are represented as right values in a `TimeStamped[Option[A]]`, whereas - * clock ticks are represented as nones. This encoding allows for an indication - * of time passage with no observed values. - * - * Generally, time series appear in increasing order, and many combinators that work with - * time series will rely on that. For streams that are globally ordered, but not locally ordered, - * i.e., near adjacent values might be out of order but values at great distance from each other - * are ordered, consider using `TimeStamped.reorderLocally` to adjust. - */ - type TimeSeries[F[_], +A] = Stream[F, TimeSeriesValue[A]] - - /** Alias for a pipe on a time series. */ - type TimeSeriesPipe[F[_], -A, +B] = Stream[F, TimeSeriesValue[A]] => Stream[F, TimeSeriesValue[B]] -} diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala deleted file mode 100644 index ce8ae67fe1..0000000000 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesPipeSuite.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright (c) 2013 Functional Streams for Scala - * - * Permission is hereby granted, free of charge, to any person obtaining a copy of - * this software and associated documentation files (the "Software"), to deal in - * the Software without restriction, including without limitation the rights to - * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of - * the Software, and to permit persons to whom the Software is furnished to do so, - * subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in all - * copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS - * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR - * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER - * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN - * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. - */ - -// Adapted from scodec-protocols, licensed under 3-clause BSD - -package fs2 -package timeseries - -import scala.concurrent.duration._ - -import TimeStamped.syntax._ - -class TimeSeriesTransducerTest extends Fs2Suite { - - test("support combining two transducers via an either") { - val add1: Scan[Unit, Int, Int] = Scan.lift(_ + 1) - val add2: Scan[Unit, Int, Int] = Scan.lift(_ + 2) - val x: Scan[Unit, Either[Int, Int], Int] = add1.choice(add2).imapState(_._1)(u => (u, u)) - val source: TimeSeries[Pure, Either[Int, Int]] = - Stream( - Right(1).at(0.seconds), - Left(2).at(0.5.seconds), - Right(3).at(1.5.seconds) - ).through(TimeSeries.interpolateTicks(1.second)) - assertEquals( - source.through(TimeSeries.preserve(x).toPipe).toList, - List( - TimeSeriesValue(0.millis, 3), - TimeSeriesValue(500.millis, 3), - TimeSeriesValue.tick(1000.millis), - TimeSeriesValue(1500.millis, 5) - ) - ) - } -} diff --git a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala index c6e09ff459..641d71df7d 100644 --- a/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala +++ b/core/shared/src/test/scala/fs2/timeseries/TimeSeriesSuite.scala @@ -60,4 +60,25 @@ class TimeSeriesSuite extends Fs2Suite { ) ) } + + test("support combining two scans via an either") { + val add1: Scan[Unit, Int, Int] = Scan.lift(_ + 1) + val add2: Scan[Unit, Int, Int] = Scan.lift(_ + 2) + val x: Scan[Unit, Either[Int, Int], Int] = add1.choice(add2).imapState(_._1)(u => (u, u)) + val source: Stream[Pure, TimeStamped[Option[Either[Int, Int]]]] = + Stream( + Right(1).at(0.seconds), + Left(2).at(0.5.seconds), + Right(3).at(1.5.seconds) + ).through(TimeSeries.interpolateTicks(1.second)) + assertEquals( + source.through(TimeSeries.preserve(x).toPipe).toList, + List( + Some(3) at 0.millis, + Some(3) at 500.millis, + None at 1000.millis, + Some(5) at 1500.millis + ) + ) + } } From 860b4a5fe590d44af38fba5e98aa7dd818a77169 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Thu, 21 Oct 2021 08:23:28 -0400 Subject: [PATCH 24/33] Imported more of scodec-protocols --- build.sbt | 31 ++++++- .../scala/fs2/timeseries/TimeSeries.scala | 27 +++--- .../main/scala/fs2/protocols/Ip4sCodecs.scala | 34 ++++++++ .../fs2/protocols/ethernet/EtherType.scala | 31 +++++++ .../ethernet/EthernetFrameHeader.scala | 54 ++++++++++++ .../scala/fs2/protocols/ip/Checksum.scala | 41 +++++++++ .../scala/fs2/protocols/ip/tcp/TcpFlags.scala | 54 ++++++++++++ .../fs2/protocols/ip/tcp/TcpHeader.scala | 67 +++++++++++++++ .../scala/fs2/protocols/ip/udp/Datagram.scala | 51 +++++++++++ .../fs2/protocols/ip/udp/DatagramHeader.scala | 49 +++++++++++ .../fs2/protocols/pcap/CaptureFile.scala | 84 +++++++++++++++++++ .../fs2/protocols/pcap/GlobalHeader.scala | 77 +++++++++++++++++ .../scala/fs2/protocols/pcap/LinkType.scala | 66 +++++++++++++++ .../scala/fs2/protocols/pcap/Record.scala | 39 +++++++++ .../fs2/protocols/pcap/RecordHeader.scala | 57 +++++++++++++ .../scala/fs2/protocols/pcap/package.scala | 40 +++++++++ 16 files changed, 791 insertions(+), 11 deletions(-) create mode 100644 protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ethernet/EtherType.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ethernet/EthernetFrameHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/Checksum.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpFlags.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/udp/Datagram.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/udp/DatagramHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/LinkType.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/RecordHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/pcap/package.scala diff --git a/build.sbt b/build.sbt index d82cafd7d1..ac1ae87c84 100644 --- a/build.sbt +++ b/build.sbt @@ -167,6 +167,8 @@ lazy val root = project io.js, scodec.jvm, scodec.js, + protocols.jvm, + protocols.js, reactiveStreams, benchmark ) @@ -266,7 +268,7 @@ lazy val io = crossProject(JVMPlatform, JSPlatform) .jsConfigure(_.enablePlugins(ScalaJSBundlerPlugin)) .settings( name := "fs2-io", - libraryDependencies += "com.comcast" %%% "ip4s-core" % "3.0.4", + libraryDependencies += "com.comcast" %%% "ip4s-core" % "3.0-19-98c5826", OsgiKeys.exportPackage := Seq("fs2.io.*"), OsgiKeys.privatePackage := Seq(), OsgiKeys.importPackage := { @@ -328,6 +330,33 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) ) .dependsOn(core % "compile->compile;test->test", io % "test") +lazy val protocols = crossProject(JVMPlatform, JSPlatform) + .in(file("protocols")) + .enablePlugins(SbtOsgi) + .jsConfigure(_.enablePlugins(ScalaJSBundlerPlugin)) + .settings( + name := "fs2-protocols", + OsgiKeys.exportPackage := Seq("fs2.protocols.*"), + OsgiKeys.privatePackage := Seq(), + OsgiKeys.importPackage := { + val Some((major, minor)) = CrossVersion.partialVersion(scalaVersion.value) + Seq( + s"""scala.*;version="[$major.$minor,$major.${minor + 1})"""", + """fs2.*;version="${Bundle-Version}"""", + "*" + ) + }, + OsgiKeys.additionalHeaders := Map("-removeheaders" -> "Include-Resource,Private-Package"), + osgiSettings, + mimaPreviousArtifacts := mimaPreviousArtifacts.value.filter { v => + VersionNumber(v.revision).matchesSemVer(SemanticSelector(">3.2.0")) + } + ) + .jsSettings( + scalaJSLinkerConfig ~= (_.withModuleKind(ModuleKind.CommonJSModule)) + ) + .dependsOn(core % "compile->compile;test->test", scodec, io) + lazy val reactiveStreams = project .in(file("reactive-streams")) .enablePlugins(SbtOsgi) diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index 6c756a11dc..644b4b3c15 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -48,7 +48,8 @@ object TimeSeries { reorderOver: FiniteDuration ): Stream[F, TimeStamped[Option[A]]] = { val src: Stream[F, TimeStamped[Option[A]]] = source.map(tsa => tsa.map(Some(_): Option[A])) - val ticks: Stream[F, TimeStamped[Option[Nothing]]] = timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) + val ticks: Stream[F, TimeStamped[Option[Nothing]]] = + timeTicks(tickPeriod).map(tsu => tsu.map(_ => None)) src.merge(ticks).through(TimeStamped.reorderLocally(reorderOver)) } @@ -61,20 +62,26 @@ object TimeSeries { apply(source.map(TimeStamped.unsafeNow), tickPeriod, reorderOver) /** Lifts a function from `A => B` to a time series pipe. */ - def lift[F[_], A, B](f: A => B): Stream[F, TimeStamped[Option[A]]] => Stream[F, TimeStamped[Option[B]]] = + def lift[F[_], A, B]( + f: A => B + ): Stream[F, TimeStamped[Option[A]]] => Stream[F, TimeStamped[Option[B]]] = _.map(_.map(_.map(f))) /** Time series pipe which discards right values. */ - def drainRight[F[_], L, R]: Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[L]]] = _.collect { - case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[L]]] - case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) - } + def drainRight[F[_], L, R] + : Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[L]]] = + _.collect { + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[L]]] + case TimeStamped(ts, Some(Left(l))) => TimeStamped(ts, Some(l)) + } /** Time series pipe which discards left values. */ - def drainLeft[F[_], L, R]: Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[R]]] = _.collect { - case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[R]]] - case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) - } + def drainLeft[F[_], L, R] + : Stream[F, TimeStamped[Option[Either[L, R]]]] => Stream[F, TimeStamped[Option[R]]] = + _.collect { + case tick @ TimeStamped(_, None) => tick.asInstanceOf[TimeStamped[Option[R]]] + case TimeStamped(ts, Some(Right(r))) => TimeStamped(ts, Some(r)) + } /** Stream of time ticks spaced by `tickPeriod`. */ private def timeTicks[F[_]: Temporal](tickPeriod: FiniteDuration): Stream[F, TimeStamped[Unit]] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala b/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala new file mode 100644 index 0000000000..83eb6012b7 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala @@ -0,0 +1,34 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +package fs2.protocols + +import scodec.Codec +import scodec.bits._ +import scodec.codecs._ +import com.comcast.ip4s._ + +object Ip4sCodecs { + val macAddress: Codec[MacAddress] = + bytes(6).xmapc(b => MacAddress.fromBytes(b.toArray).get)(m => ByteVector.view(m.toBytes)) + + val port: Codec[Port] = uint16.xmapc(p => Port.fromInt(p).get)(_.value) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ethernet/EtherType.scala b/protocols/shared/src/main/scala/fs2/protocols/ethernet/EtherType.scala new file mode 100644 index 0000000000..c42501bca1 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ethernet/EtherType.scala @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ethernet + +object EtherType { + val IPv4 = 0x0800 + val IPv6 = 0x86dd + val VLAN = 0x8100 +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ethernet/EthernetFrameHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/ethernet/EthernetFrameHeader.scala new file mode 100644 index 0000000000..cac3af866a --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ethernet/EthernetFrameHeader.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ethernet + +import scodec.Codec +import scodec.codecs._ +import fs2.interop.scodec._ + +import com.comcast.ip4s.MacAddress + +/** Header of an ethernet frame as captured in a pcap file. + */ +case class EthernetFrameHeader( + destination: MacAddress, + source: MacAddress, + ethertypeOrLength: Int +) { + def length: Option[Int] = if (ethertypeOrLength <= 1500) Some(ethertypeOrLength) else None + def ethertype: Option[Int] = if (ethertypeOrLength > 1500) Some(ethertypeOrLength) else None +} + +object EthernetFrameHeader { + // format: off + implicit val codec: Codec[EthernetFrameHeader] = { + ("destination" | Ip4sCodecs.macAddress) :: + ("source" | Ip4sCodecs.macAddress) :: + ("ethertype" | uint16) + }.as[EthernetFrameHeader] + // format: on + + val sdecoder: StreamDecoder[EthernetFrameHeader] = StreamDecoder.once(codec) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/Checksum.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/Checksum.scala new file mode 100644 index 0000000000..3c17155fc6 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/Checksum.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.ip + +import scodec.bits.BitVector + +object Checksum { + + /** Computes the 16-bit one's complement checksum of the specified bit vector. + * @see [[https://tools.ietf.org/html/rfc1071]] + */ + def checksum(bits: BitVector): BitVector = { + var sum = bits.bytes.grouped(2).foldLeft(0) { (acc, b) => + acc + b.toInt(signed = false) + } + while ((sum >> 16) != 0) + sum = (0xffff & sum) + (sum >> 16) + ~(BitVector.fromInt(sum).drop(16)) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpFlags.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpFlags.scala new file mode 100644 index 0000000000..781f47be6a --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpFlags.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip +package tcp + +import scodec.Codec +import scodec.codecs._ + +case class TcpFlags( + cwr: Boolean, + ecn: Boolean, + urg: Boolean, + ack: Boolean, + psh: Boolean, + rst: Boolean, + syn: Boolean, + fin: Boolean +) +object TcpFlags { + // format:off + implicit val codec: Codec[TcpFlags] = { + ("cwr" | bool(1)) :: + ("ecn" | bool(1)) :: + ("urg" | bool(1)) :: + ("ack" | bool(1)) :: + ("psh" | bool(1)) :: + ("rst" | bool(1)) :: + ("syn" | bool(1)) :: + ("fin" | bool(1)) + }.as[TcpFlags] + // format:on +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpHeader.scala new file mode 100644 index 0000000000..c9c4500b01 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/tcp/TcpHeader.scala @@ -0,0 +1,67 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip +package tcp + +import scodec.Codec +import scodec.codecs._ +import fs2.interop.scodec._ +import com.comcast.ip4s.Port + +case class TcpHeader( + sourcePort: Port, + destinationPort: Port, + sequenceNumber: Long, + ackNumber: Long, + dataOffset: Int, + flags: TcpFlags, + windowSize: Int, + checksum: Int, + urgentPointer: Int, + options: List[Long] +) + +object TcpHeader { + // format:off + implicit val codec: Codec[TcpHeader] = { + ("source port" | Ip4sCodecs.port) :: + ("destination port" | Ip4sCodecs.port) :: + ("seqNumber" | uint32) :: + ("ackNumber" | uint32) :: + ("dataOffset" | uint4).flatPrepend { headerWords => + ("reserved" | ignore(4)) :: + ("flags" | Codec[TcpFlags]) :: + ("windowSize" | uint16) :: + ("checksum" | uint16) :: + ("urgentPointer" | uint16) :: + ("options" | listOfN(provide(headerWords - 5), uint32)) + } + }.dropUnits.as[TcpHeader] + // format:on + + def sdecoder(protocol: Int): StreamDecoder[TcpHeader] = + if (protocol == 6) StreamDecoder.once(codec) + else StreamDecoder.empty +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/udp/Datagram.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/udp/Datagram.scala new file mode 100644 index 0000000000..92c00a7f5e --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/udp/Datagram.scala @@ -0,0 +1,51 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip +package udp + +import scodec.bits.BitVector +import scodec.Codec +import scodec.codecs._ +import com.comcast.ip4s.Port + +case class Datagram(sourcePort: Port, destinationPort: Port, data: BitVector) + +object Datagram { + implicit val codec: Codec[Datagram] = new Codec[Datagram] { + def sizeBound = Codec[DatagramHeader].sizeBound.atLeast + + def encode(dg: Datagram) = for { + encHeader <- Codec.encode( + DatagramHeader(dg.sourcePort, dg.destinationPort, 8 + dg.data.bytes.size.toInt, 0) + ) + chksum = Checksum.checksum(encHeader ++ dg.data) + } yield encHeader.dropRight(16) ++ chksum ++ dg.data + + def decode(b: BitVector) = (for { + header <- Codec[DatagramHeader] + data <- bits(8L * (header.length - 8)) + } yield Datagram(header.sourcePort, header.destinationPort, data)).decode(b) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/udp/DatagramHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/udp/DatagramHeader.scala new file mode 100644 index 0000000000..2507571873 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/udp/DatagramHeader.scala @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip +package udp + +import scodec.Codec +import scodec.codecs._ +import fs2.interop.scodec._ +import com.comcast.ip4s.Port + +case class DatagramHeader(sourcePort: Port, destinationPort: Port, length: Int, checksum: Int) + +object DatagramHeader { + // format:off + implicit val codec: Codec[DatagramHeader] = { + val port = uint16.xmapc(p => Port.fromInt(p).get)(_.value) + ("source port" | port) :: + ("destination port" | port) :: + ("length" | uint16) :: + ("checksum" | uint16) + }.as[DatagramHeader] + // format:on + + def sdecoder(protocol: Int): StreamDecoder[DatagramHeader] = + if (protocol == 17) StreamDecoder.once(codec) + else StreamDecoder.empty +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala new file mode 100644 index 0000000000..97ff3f8c00 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package pcap + +import scodec.{Codec, Err} +import scodec.codecs._ +import fs2.interop.scodec._ +import fs2.timeseries._ + +import scala.concurrent.duration._ + +case class CaptureFile(header: GlobalHeader, records: Vector[Record]) + +object CaptureFile { + implicit val codec: Codec[CaptureFile] = "capture-file" | + Codec[GlobalHeader] + .flatPrepend { hdr => + vector(Record.codec(hdr.ordering)).hlist + } + .as[CaptureFile] + + def payloadStreamDecoderPF[A]( + linkDecoders: PartialFunction[LinkType, StreamDecoder[A]] + ): StreamDecoder[TimeStamped[A]] = + payloadStreamDecoder(linkDecoders.lift) + + def payloadStreamDecoder[A]( + linkDecoders: LinkType => Option[StreamDecoder[A]] + ): StreamDecoder[TimeStamped[A]] = + streamDecoder { global => + linkDecoders(global.network) match { + case None => Left(Err(s"unsupported link type ${global.network}")) + case Some(decoder) => + Right { hdr => + decoder.map { value => + TimeStamped(hdr.timestamp + global.thiszone.toLong.seconds, value) + } + } + } + } + + def recordStreamDecoder: StreamDecoder[Record] = + streamDecoder[Record] { global => + Right { hdr => + StreamDecoder.once(bits).map { bs => + Record(hdr.copy(timestampSeconds = hdr.timestampSeconds + global.thiszone), bs) + } + } + } + + def streamDecoder[A]( + f: GlobalHeader => Either[Err, (RecordHeader => StreamDecoder[A])] + ): StreamDecoder[A] = for { + global <- StreamDecoder.once(GlobalHeader.codec) + decoderFn <- f(global).fold(StreamDecoder.raiseError, StreamDecoder.emit) + recordDecoder = + RecordHeader.codec(global.ordering).flatMap { header => + StreamDecoder.isolate(header.includedLength * 8)(decoderFn(header)).strict + } + values <- StreamDecoder.many(recordDecoder).flatMap(x => StreamDecoder.emits(x)) + } yield values +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala new file mode 100644 index 0000000000..fd3567baee --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala @@ -0,0 +1,77 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.pcap + +import scodec.Err +import scodec.bits.{BitVector, ByteOrdering} +import scodec.{Attempt, Codec, DecodeResult, SizeBound} +import scodec.codecs._ + +case class GlobalHeader( + ordering: ByteOrdering, + versionMajor: Int, + versionMinor: Int, + thiszone: Int, + sigfigs: Long, + snaplen: Long, + network: LinkType +) + +object GlobalHeader { + private val MagicNumber = 0xa1b2c3d4L + private val MagicNumberRev = 0xd4c3b2a1L + + private val byteOrdering: Codec[ByteOrdering] = new Codec[ByteOrdering] { + def sizeBound = SizeBound.exact(32) + + def encode(bo: ByteOrdering) = + endiannessDependent(uint32, uint32L)(bo).encode(MagicNumber) + + def decode(buf: BitVector) = + uint32.decode(buf).flatMap { + case DecodeResult(MagicNumber, rest) => + Attempt.successful(DecodeResult(ByteOrdering.BigEndian, rest)) + case DecodeResult(MagicNumberRev, rest) => + Attempt.successful(DecodeResult(ByteOrdering.LittleEndian, rest)) + case DecodeResult(other, _) => + Attempt.failure( + Err(s"unable to detect byte ordering due to unrecognized magic number $other") + ) + } + + override def toString = "byteOrdering" + } + + // format: off + implicit val codec: Codec[GlobalHeader] = "global-header" | { + ("magic_number" | byteOrdering ).flatPrepend { implicit ordering => + ("version_major" | guint16 ) :: + ("version_minor" | guint16 ) :: + ("thiszone" | gint32 ) :: + ("sigfigs" | guint32 ) :: + ("snaplen" | guint32 ) :: + ("network" | LinkType.codec ) + }}.as[GlobalHeader] + // format: on +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/LinkType.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/LinkType.scala new file mode 100644 index 0000000000..04ecd741a6 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/LinkType.scala @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.pcap + +import scodec.Codec +import scodec.bits.ByteOrdering + +/** Describes the link layer type in a PCAP capture. + * @see http://www.tcpdump.org/linktypes.html + */ +sealed trait LinkType + +/** Companion for [[LinkType]]. */ +object LinkType { + case object Null extends LinkType + case object Ethernet extends LinkType + case object Raw extends LinkType + case object IPv4 extends LinkType + case object IPv6 extends LinkType + case object MPEG2TS extends LinkType + case class Unknown(value: Long) extends LinkType + + def toLong(lt: LinkType): Long = lt match { + case Null => 0 + case Ethernet => 1 + case Raw => 101 + case IPv4 => 228 + case IPv6 => 229 + case MPEG2TS => 243 + case Unknown(value) => value + } + + def fromLong(l: Long): LinkType = l match { + case 0 => Null + case 1 => Ethernet + case 101 => Raw + case 228 => IPv4 + case 229 => IPv6 + case 243 => MPEG2TS + case other => Unknown(other) + } + + implicit def codec(implicit bo: ByteOrdering): Codec[LinkType] = + guint32.xmap[LinkType](fromLong, toLong) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala new file mode 100644 index 0000000000..9d81d2db45 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.pcap + +import scodec.bits.{BitVector, ByteOrdering} +import scodec.Codec +import scodec.codecs._ + +case class Record(header: RecordHeader, data: BitVector) + +object Record { + // format: off + implicit def codec(implicit ordering: ByteOrdering): Codec[Record] = "record" | { + ("record_header" | RecordHeader.codec ).flatPrepend { hdr => + ("record_data" | bits(hdr.includedLength.toInt * 8L) ).hlist + }}.as[Record] + // format: on +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/RecordHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/RecordHeader.scala new file mode 100644 index 0000000000..327aa6354c --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/RecordHeader.scala @@ -0,0 +1,57 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.pcap + +import scala.concurrent.duration._ +import scodec.bits.ByteOrdering +import scodec.Codec +import scodec.codecs._ + +case class RecordHeader( + timestampSeconds: Long, + timestampMicros: Long, + includedLength: Long, + originalLength: Long +) { + def timestamp: FiniteDuration = RecordHeader.timestamp(timestampSeconds, timestampMicros) + def fullPayload: Boolean = includedLength == originalLength +} + +object RecordHeader { + + private def timestamp(seconds: Long, micros: Long): FiniteDuration = + seconds.seconds + micros.micros + + def apply(time: FiniteDuration, includedLength: Long, originalLength: Long): RecordHeader = + RecordHeader(time.toSeconds, time.toMicros % 1000000L, includedLength, originalLength) + + // format: off + implicit def codec(implicit ordering: ByteOrdering): Codec[RecordHeader] = "record-header" | { + ("ts_sec" | guint32 ) :: + ("ts_usec" | guint32 ) :: + ("incl_len" | guint32 ) :: + ("orig_len" | guint32 ) + }.as[RecordHeader] + // format: on +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/package.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/package.scala new file mode 100644 index 0000000000..cd0cad8eb4 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/package.scala @@ -0,0 +1,40 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols + +import scodec.bits.ByteOrdering +import scodec.Codec +import scodec.codecs._ + +/** Protocol that describes libpcap files. + * + * @see http://wiki.wireshark.org/Development/LibpcapFileFormat + */ +package object pcap { + + def gint16(implicit ordering: ByteOrdering): Codec[Int] = endiannessDependent(int16, int16L) + def guint16(implicit ordering: ByteOrdering): Codec[Int] = endiannessDependent(uint16, uint16L) + def gint32(implicit ordering: ByteOrdering): Codec[Int] = endiannessDependent(int32, int32L) + def guint32(implicit ordering: ByteOrdering): Codec[Long] = endiannessDependent(uint32, uint32L) +} From 01fa30ae062d50648ef70c5d7ef569968c7a572f Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Thu, 21 Oct 2021 08:46:02 -0400 Subject: [PATCH 25/33] Update workflow --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 2b8b964138..ed9dc79b64 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -68,7 +68,7 @@ jobs: run: sbt ++${{ matrix.scala }} microsite/mdoc - name: Compress target directories - run: tar cf targets.tar target node/js/target core/js/target core/jvm/target scodec/jvm/target scodec/js/target io/js/target reactive-streams/target io/jvm/target benchmark/target project/target + run: tar cf targets.tar target node/js/target protocols/js/target core/js/target core/jvm/target scodec/jvm/target scodec/js/target io/js/target reactive-streams/target io/jvm/target protocols/jvm/target benchmark/target project/target - name: Upload target directories uses: actions/upload-artifact@v2 From 05edcdd61d86eb1bc816f7526160b303e29d892d Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 22 Oct 2021 09:18:27 -0400 Subject: [PATCH 26/33] Import the rest of scodec-protocols --- build.sbt | 4 +- .../main/scala/fs2/protocols/Ip4sCodecs.scala | 6 + .../scala/fs2/protocols/ip/Ipv4Header.scala | 102 +++++ .../scala/fs2/protocols/ip/Ipv6Header.scala | 60 +++ .../scala/fs2/protocols/mpeg/Descriptor.scala | 397 ++++++++++++++++++ .../scala/fs2/protocols/mpeg/MpegError.scala | 50 +++ .../scala/fs2/protocols/mpeg/PesPacket.scala | 75 ++++ .../fs2/protocols/mpeg/PesPacketHeader.scala | 186 ++++++++ .../mpeg/PesPacketHeaderPrefix.scala | 39 ++ .../fs2/protocols/mpeg/PesStreamId.scala | 54 +++ .../scala/fs2/protocols/mpeg/package.scala | 36 ++ .../mpeg/transport/AdaptationField.scala | 86 ++++ .../mpeg/transport/AdaptationFieldFlags.scala | 52 +++ .../protocols/mpeg/transport/Clock27MHz.scala | 33 ++ .../mpeg/transport/ContinuityCounter.scala | 41 ++ .../mpeg/transport/Demultiplexer.scala | 250 +++++++++++ .../mpeg/transport/DemultiplexerError.scala | 45 ++ .../fs2/protocols/mpeg/transport/Packet.scala | 148 +++++++ .../fs2/protocols/mpeg/transport/Pid.scala | 55 +++ .../mpeg/transport/ProgramNumber.scala | 39 ++ .../transport/TransportStreamHeader.scala | 54 +++ .../mpeg/transport/TransportStreamId.scala | 39 ++ .../psi/ConditionalAccessTable.scala | 141 +++++++ .../mpeg/transport/psi/GroupedSections.scala | 143 +++++++ .../mpeg/transport/psi/GroupingError.scala | 36 ++ .../psi/ProgramAssociationTable.scala | 115 +++++ .../mpeg/transport/psi/ProgramMapTable.scala | 122 ++++++ .../mpeg/transport/psi/Section.scala | 56 +++ .../transport/psi/SectionAccumulator.scala | 59 +++ .../mpeg/transport/psi/SectionCodec.scala | 156 +++++++ .../transport/psi/SectionFragmentCodec.scala | 101 +++++ .../mpeg/transport/psi/SectionHeader.scala | 46 ++ .../protocols/mpeg/transport/psi/Table.scala | 37 ++ .../mpeg/transport/psi/TableBuilder.scala | 82 ++++ .../transport/psi/TransportStreamEvent.scala | 117 ++++++ .../transport/psi/TransportStreamIndex.scala | 108 +++++ .../fs2/protocols/pcap/CaptureFile.scala | 2 +- .../fs2/protocols/pcap/GlobalHeader.scala | 6 +- .../scala/fs2/protocols/pcap/Record.scala | 2 +- .../scala/fs2/protocols/PcapMpegExample.scala | 69 +++ .../fs2/protocols/mpeg/DescriptorTest.scala | 268 ++++++++++++ .../protocols/mpeg/transport/PacketTest.scala | 54 +++ .../mpeg/transport/psi/GroupingTest.scala | 55 +++ .../mpeg/transport/psi/SectionCodecTest.scala | 116 +++++ 44 files changed, 3736 insertions(+), 6 deletions(-) create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/Ipv4Header.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/package.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ContinuityCounter.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ProgramNumber.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamId.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala create mode 100644 protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala create mode 100644 protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala create mode 100644 protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala create mode 100644 protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala create mode 100644 protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala create mode 100644 protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala diff --git a/build.sbt b/build.sbt index ac1ae87c84..4b242a4abf 100644 --- a/build.sbt +++ b/build.sbt @@ -307,8 +307,8 @@ lazy val scodec = crossProject(JVMPlatform, JSPlatform) libraryDependencies += "org.scodec" %%% "scodec-core" % (if ( scalaVersion.value.startsWith("2.") ) - "1.11.8" - else "2.0.0"), + "1.11.9" + else "2.1.0"), OsgiKeys.exportPackage := Seq("fs2.interop.scodec.*"), OsgiKeys.privatePackage := Seq(), OsgiKeys.importPackage := { diff --git a/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala b/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala index 83eb6012b7..0a77c36597 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/Ip4sCodecs.scala @@ -27,6 +27,12 @@ import scodec.codecs._ import com.comcast.ip4s._ object Ip4sCodecs { + val ipv4: Codec[Ipv4Address] = + bytes(4).xmapc(b => Ipv4Address.fromBytes(b.toArray).get)(a => ByteVector.view(a.toBytes)) + + val ipv6: Codec[Ipv6Address] = + bytes(8).xmapc(b => Ipv6Address.fromBytes(b.toArray).get)(a => ByteVector.view(a.toBytes)) + val macAddress: Codec[MacAddress] = bytes(6).xmapc(b => MacAddress.fromBytes(b.toArray).get)(m => ByteVector.view(m.toBytes)) diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv4Header.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv4Header.scala new file mode 100644 index 0000000000..578a9b1275 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv4Header.scala @@ -0,0 +1,102 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip + +import scodec.bits._ +import scodec.{Codec, SizeBound} +import scodec.codecs._ +import scodec.compat._ +import fs2.interop.scodec._ +import fs2.protocols.ethernet.{EtherType, EthernetFrameHeader} +import com.comcast.ip4s.Ipv4Address + +/** IPv4 header. */ +case class Ipv4Header( + dataLength: Int, + id: Int, + ttl: Int, + protocol: Int, + sourceIp: Ipv4Address, + destinationIp: Ipv4Address, + options: BitVector +) + +object Ipv4Header { + + // format:off + implicit val codec: Codec[Ipv4Header] = { + val componentCodec = { + // Word 1 -------------------------------- + ("version" | constant(bin"0100")) :: + ("ihl" | uint4) :: + ("dscp" | ignore(6)) :: + ("ecn" | ignore(2)) :: + ("total_length" | uint16) :: + // Word 2 -------------------------------- + ("id" | uint16) :: + ("flags" | ignore(3)) :: + ("fragment_offset" | ignore(13)) :: + // Word 3 -------------------------------- + ("ttl" | uint8) :: + ("proto" | uint8) :: + ("checksum" | bits(16)) :: + // Word 4 -------------------------------- + ("src_ip" | Ip4sCodecs.ipv4) :: + // Word 5 -------------------------------- + ("dest_ip" | Ip4sCodecs.ipv4) :: + // Word 6 -------------------------------- + ("options" | bits) + }.dropUnits + // format:on + + new Codec[Ipv4Header] { + def sizeBound = SizeBound.atLeast(160) + + def encode(header: Ipv4Header) = { + val optionWords = (header.options.size.toInt + 31) / 32 + val options = header.options.padRight((optionWords * 32).toLong) + val totalLength = header.dataLength + 20 + (optionWords * 32) + for { + encoded <- componentCodec.encode( + (5 + optionWords) *: totalLength *: header.id *: header.ttl *: header.protocol *: BitVector + .low(16) *: header.sourceIp *: header.destinationIp *: options *: EmptyTuple + ) + chksum = Checksum.checksum(encoded) + } yield encoded.patch(80L, chksum) + } + + def decode(bits: BitVector) = + componentCodec.decode(bits).map { + _.map { t => + Ipv4Header(t(1) - (t(0) * 4), t(2), t(3), t(4), t(6), t(7), t(8)) + } + } + } + } + + def sdecoder(ethernetHeader: EthernetFrameHeader): StreamDecoder[Ipv4Header] = + if (ethernetHeader.ethertype == Some(EtherType.IPv4)) StreamDecoder.once(codec) + else StreamDecoder.empty +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala new file mode 100644 index 0000000000..6d8a43eb83 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala @@ -0,0 +1,60 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package ip + +import scodec._ +import scodec.bits._ +import scodec.codecs._ +import fs2.interop.scodec._ +import fs2.protocols.ethernet.{EthernetFrameHeader, EtherType} +import com.comcast.ip4s.Ipv6Address + +/** Simplified model of an IPv6 header -- extension headers are not directly supported. */ +case class Ipv6Header( + trafficClass: Int, + flowLabel: Int, + payloadLength: Int, + protocol: Int, + hopLimit: Int, + sourceIp: Ipv6Address, + destinationIp: Ipv6Address +) + +object Ipv6Header { + implicit val codec: Codec[Ipv6Header] = { + ("version" | constant(bin"0110")) ~> + ("traffic_class" | uint8) :: + ("flow_label" | uint(20)) :: + ("payload_length" | uint(16)) :: + ("next_header" | uint8) :: + ("hop_limit" | uint8) :: + ("source_address" | Ip4sCodecs.ipv6) :: + ("destination_address" | Ip4sCodecs.ipv6) + }.as[Ipv6Header] + + def sdecoder(ethernetHeader: EthernetFrameHeader): StreamDecoder[Ipv6Header] = + if (ethernetHeader.ethertype == Some(EtherType.IPv6)) StreamDecoder.once(codec) + else StreamDecoder.empty +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala new file mode 100644 index 0000000000..6c9dc9a30c --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala @@ -0,0 +1,397 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec._ +import scodec.bits._ +import scodec.codecs._ + +trait KnownDescriptor + +sealed trait TransportStreamDescriptor extends KnownDescriptor +sealed trait ProgramStreamDescriptor extends KnownDescriptor + +case class Mpeg1Only(profileAndLevelIndication: Int, chromaFormat: Int, frameRateExtensionFlag: Boolean) +object Mpeg1Only { + implicit val codec: Codec[Mpeg1Only] = { + ("profile_and_level_indication" | uint8) :: + ("chroma_format" | uint(2)) :: + ("frame_rate_extension_flag" | bool) :: + ("reserved" | reserved(5)) + }.dropUnits.as[Mpeg1Only] +} +case class VideoStreamDescriptor( + multipleFrameRateFlag: Boolean, + frameRateCode: Int, + mpeg1OnlyFlag: Boolean, + constrainedParameter: Boolean, + stillPictureFlag: Boolean, + mpeg1Only: Option[Mpeg1Only]) extends TransportStreamDescriptor with ProgramStreamDescriptor + +object VideoStreamDescriptor { + val codec: Codec[VideoStreamDescriptor] = { + ("multiple_frame_rate_flag" | bool) :: + ("frame_rate_code" | uint4) :: + (("MPEG_1_only_flag" | bool).flatPrepend { mpeg1Only => + ("constrained_parameter" | bool) :: + ("still_picture_flag" | bool) :: + ("MPEG_1_only_attributes" | conditional(mpeg1Only, Codec[Mpeg1Only])) + }) + }.as[VideoStreamDescriptor] +} + +case class AudioStreamDescriptor(freeFormatFlag: Boolean, id: Boolean, layer: Int, variableRateAudioIndicator: Boolean) extends TransportStreamDescriptor with ProgramStreamDescriptor +object AudioStreamDescriptor { + val codec: Codec[AudioStreamDescriptor] = { + ("free_format_flag" | bool) :: + ("ID" | bool) :: + ("layer" | uint(2)) :: + ("variable_rate_audio_indicator" | bool) :: + ("reserved" | reserved(3)) + }.dropUnits.as[AudioStreamDescriptor] +} + +sealed trait HierarchyType +object HierarchyType { + case object SpatialScalability extends HierarchyType + case object SnrScalability extends HierarchyType + case object TemporalScalability extends HierarchyType + case object DataPartitioning extends HierarchyType + case object ExtensionBitstream extends HierarchyType + case object PrivateStream extends HierarchyType + case object MultiViewProfile extends HierarchyType + case class Reserved(value: Int) extends HierarchyType + case object BaseLayer extends HierarchyType + + implicit val codec: Codec[HierarchyType] = { + val m = discriminated[HierarchyType].by(uint4) + .typecase(0, provide(Reserved(0))) + .typecase(1, provide(SpatialScalability)) + .typecase(2, provide(SnrScalability)) + .typecase(3, provide(TemporalScalability)) + .typecase(4, provide(DataPartitioning)) + .typecase(5, provide(ExtensionBitstream)) + .typecase(6, provide(PrivateStream)) + .typecase(7, provide(MultiViewProfile)) + .typecase(15, provide(BaseLayer)) + (8 to 14).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + } +} +case class HierarchyDescriptor(hierarchyType: HierarchyType, hierarchyLayerIndex: Int, hierarchyEmbeddedLayerIndex: Int, hierarchyChannel: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object HierarchyDescriptor { + val codec: Codec[HierarchyDescriptor] = { + ("reserved" | reserved(4)) :: + ("hierarchy_type" | Codec[HierarchyType]) :: + ("reserved" | reserved(2)) :: + ("hierarchy_layer_index" | uint(6)) :: + ("reserved" | reserved(2)) :: + ("hierarchy_embedded_layer_index" | uint(6)) :: + ("reserved" | reserved(2)) :: + ("hierarchy_channel" | uint(6)) + }.dropUnits.as[HierarchyDescriptor] +} + +case class RegistrationDescriptor(formatIdentifier: ByteVector, additionalIdentificationInfo: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object RegistrationDescriptor { + val codec: Codec[RegistrationDescriptor] = { + (("format_identifier" | bytes(4)) :: bytes) + }.as[RegistrationDescriptor] +} + +sealed trait AlignmentType +object AlignmentType { + case object SliceOrVideoAccessUnit extends AlignmentType + case object VideoAccessUnit extends AlignmentType + case object GopOrSeq extends AlignmentType + case object Seq extends AlignmentType + case class Reserved(value: Int) extends AlignmentType + implicit val codec: Codec[AlignmentType] = { + val m = discriminated[AlignmentType].by(uint8) + .typecase(0, provide(Reserved(0))) + .typecase(1, provide(SliceOrVideoAccessUnit)) + .typecase(2, provide(VideoAccessUnit)) + .typecase(3, provide(GopOrSeq)) + .typecase(4, provide(Seq)) + (5 to 255).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + } +} +case class DataStreamAlignmentDescriptor(alignmentType: AlignmentType) extends TransportStreamDescriptor with ProgramStreamDescriptor +object DataStreamAlignmentDescriptor { + val codec: Codec[DataStreamAlignmentDescriptor] = { + ("alignment_type" | Codec[AlignmentType]) + }.as[DataStreamAlignmentDescriptor] +} + +case class TargetBackgroundGridDescriptor(horizontalSize: Int, verticalSize: Int, aspectRatioInformation: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object TargetBackgroundGridDescriptor { + val codec: Codec[TargetBackgroundGridDescriptor] = { + ("horizontal_size" | uint(14)) :: + ("vertical_size" | uint(14)) :: + ("aspect_ratio_information" | uint4) + }.as[TargetBackgroundGridDescriptor] +} + +case class VideoWindowDescriptor(horizontalOffset: Int, verticalOffset: Int, windowPriority: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object VideoWindowDescriptor { + val codec: Codec[VideoWindowDescriptor] = { + ("horizontal_offset" | uint(14)) :: + ("vertical_offset" | uint(14)) :: + ("window_priority" | uint4) + }.as[VideoWindowDescriptor] +} + +case class CADescriptor(caSystemId: Int, caPid: Pid, privateData: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object CADescriptor { + val codec: Codec[CADescriptor] = { + ("CA_system_id" | uint16) :: reserved(3) :: ("CA_PID" | Codec[Pid]) :: bytes + }.as[CADescriptor] +} + +sealed trait AudioType +object AudioType { + case object Undefined extends AudioType + case object CleanEffects extends AudioType + case object HearingImpaired extends AudioType + case object VisualImpairedCommentary extends AudioType + case class Reserved(value: Int) extends AudioType + + implicit val codec: Codec[AudioType] = { + val m = discriminated[AudioType].by(uint8) + .typecase(0, provide(Undefined)) + .typecase(1, provide(CleanEffects)) + .typecase(2, provide(HearingImpaired)) + .typecase(3, provide(VisualImpairedCommentary)) + (4 to 255).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + } +} + +case class LanguageField(iso639LanguageCode: String, audioType: AudioType) +object LanguageField { + implicit val codec: Codec[LanguageField] = { + ("ISO_639_language_code" | fixedSizeBytes(3, ascii)) :: + ("audio_type" | Codec[AudioType]) + }.as[LanguageField] +} + +case class Iso639LanguageDescriptor(languageFields: Vector[LanguageField]) extends TransportStreamDescriptor with ProgramStreamDescriptor +object Iso639LanguageDescriptor { + val codec: Codec[Iso639LanguageDescriptor] = { + vector(Codec[LanguageField]) + }.as[Iso639LanguageDescriptor] +} + + +case class SystemClockDescriptor(externalClockReferenceIndicator: Boolean, clockAccuracyInteger: Int, clockAccuracyExponent: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object SystemClockDescriptor { + val codec: Codec[SystemClockDescriptor] = { + ("external_clock_reference_indicator" | bool) :: + ("reserved" | reserved(1)) :: + ("clock_accuracy_integer" | uint(6)) :: + ("clock_accuracy_exponent" | uint(3)) :: + ("reserved" | reserved(5)) + }.dropUnits.as[SystemClockDescriptor] +} + +case class MultiplexBufferUtilizationDescriptor(boundValidFlag: Boolean, ltwOffsetLowerBound: Int, ltwOffsetUpperBound: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object MultiplexBufferUtilizationDescriptor { + val codec: Codec[MultiplexBufferUtilizationDescriptor] = { + ("bound_valid_flag" | bool) :: + ("LTW_offset_lower_bound" | uint(15)) :: + ("reserved" | reserved(1)) :: + ("LTW_offset_upper_bound" | uint(15)) + }.dropUnits.as[MultiplexBufferUtilizationDescriptor] +} + +case class CopyrightDescriptor(copyrightIdentifier: ByteVector, additionalCopyrightInfo: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object CopyrightDescriptor { + val codec: Codec[CopyrightDescriptor] = { + bytes(4) :: bytes + }.as[CopyrightDescriptor] +} + +case class MaximumBitrateDescriptor(maximumBitrate: Int) extends TransportStreamDescriptor +object MaximumBitrateDescriptor { + val codec: Codec[MaximumBitrateDescriptor] = { + ("reserved" | reserved(2)) :: + ("maximum_bitrate" | uint(22)) + }.dropUnits.as[MaximumBitrateDescriptor] +} + +case class PrivateDataIndicatorDescriptor(privateDataIndicator: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object PrivateDataIndicatorDescriptor { + val codec: Codec[PrivateDataIndicatorDescriptor] = { + ("private_data_indicator" | bytes(4)) + }.as[PrivateDataIndicatorDescriptor] +} + +case class SmoothingBufferDescriptor(sbLeakRate: Int, sbSize: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object SmoothingBufferDescriptor { + val codec: Codec[SmoothingBufferDescriptor] = { + ("reserved" | reserved(2)) :: + ("sb_leak_rate" | uint(22)) :: + ("reserved" | reserved(2)) :: + ("sb_size" | uint(22)) + }.dropUnits.as[SmoothingBufferDescriptor] +} + +case class StdDescriptor(leakValidFlag: Boolean) extends TransportStreamDescriptor +object StdDescriptor { + val codec: Codec[StdDescriptor] = { + ("reserved" | reserved(7)) :: + ("leak_valid_flag" | bool) + }.dropUnits.as[StdDescriptor] +} + +case class IbpDescriptor(closedGopFlag: Boolean, identicalGopFlag: Boolean, maxGopLength: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object IbpDescriptor { + val codec: Codec[IbpDescriptor] = { + ("closed_gop_flag" | bool) :: + ("identical_gop_flag" | bool) :: + ("max_gop_length" | uint(14)) + }.as[IbpDescriptor] +} + +case class Mpeg4VideoDescriptor(mpeg4VisualProfileAndLevel: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +object Mpeg4VideoDescriptor { + val codec: Codec[Mpeg4VideoDescriptor] = { + ("MPEG-4_visual_profile_and_level" | byte) + }.as[Mpeg4VideoDescriptor] +} + +case class Mpeg4AudioDescriptor(mpeg4AudioProfileAndLevel: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +object Mpeg4AudioDescriptor { + val codec: Codec[Mpeg4AudioDescriptor] = { + ("MPEG-4_audio_profile_and_level" | byte) + }.as[Mpeg4AudioDescriptor] +} + +case class IodDescriptor(scopeOfIodLabel: Byte, iodLabel: Byte, initialObjectDescriptor: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +object IodDescriptor { + val codec: Codec[IodDescriptor] = { + ("Scope_of_IOD_label" | byte) :: + ("IOD_label" | byte) :: + ("initialObjectDescriptor" | byte) + }.as[IodDescriptor] +} + +case class SlDescriptor(esId: Int) extends TransportStreamDescriptor +object SlDescriptor { + val codec: Codec[SlDescriptor] = { + ("ES_ID" | uint16) + }.as[SlDescriptor] +} + +case class EsIdAndChannel(esId: Int, flexMuxChannel: Int) +object EsIdAndChannel { + implicit val codec: Codec[EsIdAndChannel] = { + ("ES_ID" | uint16) :: + ("FlexMuxChannel" | uint8) + }.as[EsIdAndChannel] +} +case class FmcDescriptor(channels: Vector[EsIdAndChannel]) extends TransportStreamDescriptor with ProgramStreamDescriptor +object FmcDescriptor { + val codec: Codec[FmcDescriptor] = { + vector(Codec[EsIdAndChannel]) + }.as[FmcDescriptor] +} + +case class ExternalEsIdDescriptor(esternalEsId: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object ExternalEsIdDescriptor { + val codec: Codec[ExternalEsIdDescriptor] = { + ("External_ES_ID" | uint16) + }.as[ExternalEsIdDescriptor] +} + +case class MuxCodeDescriptor(muxCodeTableEntry: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object MuxCodeDescriptor { + val codec: Codec[MuxCodeDescriptor] = { + bytes + }.as[MuxCodeDescriptor] +} + +case class FmxBufferSizeDescriptor(flexMuxBufferDescriptor: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +object FmxBufferSizeDescriptor { + val codec: Codec[FmxBufferSizeDescriptor] = { + bytes + }.as[FmxBufferSizeDescriptor] +} + +case class MultiplexBufferDescriptor(mbBufferSize: Int, tbLeakRate: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +object MultiplexBufferDescriptor { + val codec: Codec[MultiplexBufferDescriptor] = { + ("MB_buffer_size" | uint24) :: + ("TB_leak_rate" | uint24) + }.as[MultiplexBufferDescriptor] +} + +case class UnknownDescriptor(tag: Int, length: Int, data: ByteVector) +object UnknownDescriptor { + val codec: Codec[UnknownDescriptor] = { + ("descriptor_tag" | uint8) :: + (("descriptor_length" | uint8).flatPrepend { length => + ("descriptor_data" | bytes(length)).tuple + }) + }.as[UnknownDescriptor] +} + +object Descriptor { + type Descriptor = Either[UnknownDescriptor, KnownDescriptor] + + val knownCodec: Codec[KnownDescriptor] = { + def sized[A](c: Codec[A]) = variableSizeBytes(uint8, c) + discriminated[KnownDescriptor].by(uint8) + .typecase(2, sized(VideoStreamDescriptor.codec)) + .typecase(3, sized(AudioStreamDescriptor.codec)) + .typecase(4, sized(HierarchyDescriptor.codec)) + .typecase(5, sized(RegistrationDescriptor.codec)) + .typecase(6, sized(DataStreamAlignmentDescriptor.codec)) + .typecase(7, sized(TargetBackgroundGridDescriptor.codec)) + .typecase(8, sized(VideoWindowDescriptor.codec)) + .typecase(9, sized(CADescriptor.codec)) + .typecase(10, sized(Iso639LanguageDescriptor.codec)) + .typecase(11, sized(SystemClockDescriptor.codec)) + .typecase(12, sized(MultiplexBufferUtilizationDescriptor.codec)) + .typecase(13, sized(CopyrightDescriptor.codec)) + .typecase(14, sized(MaximumBitrateDescriptor.codec)) + .typecase(15, sized(PrivateDataIndicatorDescriptor.codec)) + .typecase(16, sized(SmoothingBufferDescriptor.codec)) + .typecase(17, sized(StdDescriptor.codec)) + .typecase(18, sized(IbpDescriptor.codec)) + .typecase(27, sized(Mpeg4VideoDescriptor.codec)) + .typecase(28, sized(Mpeg4AudioDescriptor.codec)) + .typecase(29, sized(IodDescriptor.codec)) + .typecase(30, sized(SlDescriptor.codec)) + .typecase(31, sized(FmcDescriptor.codec)) + .typecase(32, sized(ExternalEsIdDescriptor.codec)) + .typecase(33, sized(MuxCodeDescriptor.codec)) + .typecase(34, sized(FmxBufferSizeDescriptor.codec)) + .typecase(35, sized(MultiplexBufferDescriptor.codec)) + } + + val codec: Codec[Descriptor] = discriminatorFallback(UnknownDescriptor.codec, knownCodec) + + def lengthCodec: Codec[Int] = ("descriptor_length" | uint8) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala new file mode 100644 index 0000000000..ac8dc0ee28 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala @@ -0,0 +1,50 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg + +import scodec.Err +import scodec.bits.BitVector + +trait MpegError { + def message: String +} + +object MpegError { + + case class General(message: String) extends MpegError { + override def toString = message + } + case class Decoding(data: BitVector, err: Err) extends MpegError { + def message = s"error encountered when decoding: $err ${data.toHex}" + override def toString = message + } + + def joinErrors[S, I, O](t: Scan[S, I, Either[MpegError, O]]): Scan[S, Either[MpegError, I], Either[MpegError, O]] = + t.semipass(_.fold(e => Left(Left(e)), i => Right(i))) + + def passErrors[S, I, O](t: Scan[S, I, O]): Scan[S, Either[MpegError, I], Either[MpegError, O]] = + t.right +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala new file mode 100644 index 0000000000..a0f194bd4b --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala @@ -0,0 +1,75 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg + +import scodec.bits._ +import scodec.{ Attempt, Decoder, DecodeResult, Err } + +trait PesPacket + +object PesPacket { + + case class WithHeader(streamId: Int, header: PesPacketHeader, data: BitVector) extends PesPacket + case class WithoutHeader(streamId: Int, data: BitVector) extends PesPacket + case object Padding extends PesPacket + + def decode(prefix: PesPacketHeaderPrefix, buffer: BitVector): Attempt[DecodeResult[PesPacket]] = + decoder(prefix).decode(buffer) + + def decoder(prefix: PesPacketHeaderPrefix): Decoder[PesPacket] = Decoder { buffer => + val id = prefix.streamId + import PesStreamId._ + if (id != ProgramStreamMap && + id != PaddingStream && + id != PrivateStream2 && + id != ECM && + id != EMM && + id != ProgramStreamDirectory && + id != DSMCC && + id != `ITU-T Rec. H.222.1 type E`) { + PesPacketHeader.codec.decode(buffer) match { + case Attempt.Successful(DecodeResult(header, rest)) => + decodeWithHeader(prefix, header, rest) + case f @ Attempt.Failure(_) => f + } + } else if ( + id == ProgramStreamMap || + id == PrivateStream2 || + id == ECM || + id == EMM | + id == ProgramStreamDirectory || + id == DSMCC || + id == `ITU-T Rec. H.222.1 type E`) { + Attempt.successful(DecodeResult(WithoutHeader(id, buffer), BitVector.empty)) + } else if (id == PaddingStream) { + Attempt.successful(DecodeResult(Padding, BitVector.empty)) + } else { + Attempt.failure(Err(s"Unknown PES stream id: $id")) + } + } + + def decodeWithHeader(prefix: PesPacketHeaderPrefix, header: PesPacketHeader, data: BitVector): Attempt[DecodeResult[PesPacket]] = { + Attempt.successful(DecodeResult(WithHeader(prefix.streamId, header, data), BitVector.empty)) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala new file mode 100644 index 0000000000..0558416af7 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala @@ -0,0 +1,186 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg + +import scodec.bits._ +import scodec.Codec +import scodec.codecs._ + +sealed abstract class PesScramblingControl +object PesScramblingControl { + object NotScrambled extends PesScramblingControl + object UserDefined1 extends PesScramblingControl + object UserDefined2 extends PesScramblingControl + object UserDefined3 extends PesScramblingControl + + implicit val codec: Codec[PesScramblingControl] = mappedEnum(bits(2), + NotScrambled -> bin"00", + UserDefined1 -> bin"01", + UserDefined2 -> bin"10", + UserDefined3 -> bin"11") +} + +case class PesPacketHeader( + pesScramblingControl: PesScramblingControl, + pesPriority: Boolean, + dataAlignmentIndicator: Boolean, + copyright: Boolean, + originalOrCopy: Boolean, + flags: PesPacketHeader.Flags, // TODO + pts: Option[Long], + dts: Option[Long], + escr: Option[Long], + esRate: Option[Int], + dsmTrickMode: Option[BitVector], + additionalCopyInfo: Option[BitVector], + pesCrc: Option[Int], + extension: Option[PesPacketHeader.Extension] +) + +object PesPacketHeader { + + case class Flags( + ptsFlag: Boolean, + dtsFlag: Boolean, + escrFlag: Boolean, + esRateFlag: Boolean, + dsmTrickModeFlag: Boolean, + additionalCopyInfoFlag: Boolean, + pesCrcFlag: Boolean, + pesExtensionFlag: Boolean + ) + + object Flags { + implicit val codec: Codec[Flags] = { + ("pts_dts_flags[0]" | bool ) :: + ("pts_dts_flags[1]" | bool ) :: + ("escr_flag" | bool ) :: + ("es_rate_flag" | bool ) :: + ("dsm_trick_mode_flag" | bool ) :: + ("additional_copy_info_flag" | bool ) :: + ("pes_crc_flag" | bool ) :: + ("pes_extension_flag" | bool ) + }.as[Flags] + } + + case class ExtensionFlags( + pesPrivateDataFlag: Boolean, + packHeaderFieldFlag: Boolean, + programPacketSequenceCounterFlag: Boolean, + pstdBufferFlag: Boolean, + pesExtensionFlag2: Boolean + ) + object ExtensionFlags { + implicit val codec: Codec[ExtensionFlags] = { + ("pes_private_data_flag" | bool ) :: + ("pack_header_field_flag" | bool ) :: + ("program_packet_sequence_counter_flag" | bool ) :: + ("P-STD_buffer_flag" | bool ) :: + reserved(3) :: + ("pes_extension_flag_2" | bool ) + }.dropUnits.as[ExtensionFlags] + } + + case class ProgramPacketSequenceCounter(counter: Int, mpeg1: Boolean, originalStuffLength: Int) + object ProgramPacketSequenceCounter { + implicit val codec: Codec[ProgramPacketSequenceCounter] = { + (marker :: uint(7) :: marker :: bool :: uint(6)).dropUnits.as[ProgramPacketSequenceCounter] + } + } + + case class PStdBuffer(scale: Boolean, size: Int) + object PStdBuffer { + implicit val codec: Codec[PStdBuffer] = { + (constant(bin"01") ~> bool :: uint(13)).as[PStdBuffer] + } + } + + case class Extension( + flags: ExtensionFlags, // TODO + pesPrivateData: Option[BitVector], + packHeaderField: Option[BitVector], + programPacketSequenceCounter: Option[ProgramPacketSequenceCounter], + pstdBuffer: Option[PStdBuffer], + extension: Option[BitVector] + ) + object Extension { + implicit val codec: Codec[Extension] = { + Codec[ExtensionFlags].flatPrepend { flags => + ("pes_private_data" | conditional(flags.pesPrivateDataFlag, bits(128))) :: + ("pack_header_field" | conditional(flags.packHeaderFieldFlag, variableSizeBytes(uint8, bits))) :: + ("program_packet_sequence_counter" | conditional(flags.programPacketSequenceCounterFlag, Codec[ProgramPacketSequenceCounter])) :: + ("P-STD_buffer" | conditional(flags.pstdBufferFlag, Codec[PStdBuffer])) :: + ("pes_extension_2" | conditional(flags.pesExtensionFlag2, marker ~> variableSizeBytes(uint(7), bits))) + } + }.as[Extension] + } + + private val marker: Codec[Unit] = constantLenient(bin"1") + + private def tsCodec(prefix: BitVector) = { + (constant(prefix) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker).dropUnits.xmap[Long]( + { case (a, b, c) => (a ++ b ++ c).toLong() }, + l => { + val b = BitVector.fromLong(l).drop(31) + (b.take(3), b.drop(3).take(15), b.drop(18)) + } + ) + } + + private val escrCodec: Codec[Long] = { + (ignore(2) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker :: uint(9) :: marker).dropUnits.xmap[Long]( + { case (a, b, c, ext) => + val base = (a ++ b ++ c).toLong() + base * 300 + ext + }, + l => { + val base = (l / 300) % (2L << 32) + val b = BitVector.fromLong(base).drop(31) + val ext = (l % 300).toInt + (b.take(3), b.drop(3).take(15), b.drop(18), ext) + } + ) + } + + implicit val codec: Codec[PesPacketHeader] = { + constant(bin"10") ~> + ("pes_scrambling_control" | PesScramblingControl.codec ) :: + ("pes_priority" | bool ) :: + ("data_alignment_indicator" | bool ) :: + ("copyright" | bool ) :: + ("original_or_copy" | bool ) :: + (("flags" | Codec[Flags] ).flatPrepend { flags => + variableSizeBytes(uint8, + ("pts" | conditional(flags.ptsFlag, tsCodec(bin"0011")) ) :: + ("dts" | conditional(flags.dtsFlag, tsCodec(bin"0001")) ) :: + ("escr" | conditional(flags.escrFlag, escrCodec) ) :: + ("es_rate" | conditional(flags.esRateFlag, ignore(1) ~> uint(22) <~ ignore(1))) :: + ("dsm_trick_mode" | conditional(flags.dsmTrickModeFlag, bits(8)) ) :: + ("additional_copy_info" | conditional(flags.additionalCopyInfoFlag, ignore(1) ~> bits(7)) ) :: + ("pes_crc" | conditional(flags.pesCrcFlag, uint16) ) :: + ("extension" | conditional(flags.pesExtensionFlag, Codec[Extension]) ) + ) + }) // .removeElem[Flags](Generic[Flags].from(optionalFields.map(_.isDefined))) + }.withContext("pes_packet_header").as[PesPacketHeader] +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala new file mode 100644 index 0000000000..f4dcb36552 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg + +import scodec.Codec +import scodec.codecs._ + +case class PesPacketHeaderPrefix(streamId: Int, length: Int) + +object PesPacketHeaderPrefix { + + implicit val codec: Codec[PesPacketHeaderPrefix] = { + fixedSizeBytes(3, + ("stream_id" | uint8 ) :: + ("pes_packet_length" | uint16) + ).as[PesPacketHeaderPrefix] + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala new file mode 100644 index 0000000000..dc869977de --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg + +import scodec.bits._ + +object PesStreamId { + // format:off + val ProgramStreamMap = bin"1011 1100".toInt(signed = false) + val PrivateStream1 = bin"1011 1101".toInt(signed = false) + val PaddingStream = bin"1011 1110".toInt(signed = false) + val PrivateStream2 = bin"1011 1111".toInt(signed = false) + val AudioStreamMin = bin"1100 0000".toInt(signed = false) + val AudioStreamMax = bin"1101 1111".toInt(signed = false) + val VideoStreamMin = bin"1110 0000".toInt(signed = false) + val VideoStreamMax = bin"1110 1111".toInt(signed = false) + val ECM = bin"1111 0000".toInt(signed = false) + val EMM = bin"1111 0001".toInt(signed = false) + val DSMCC = bin"1111 0010".toInt(signed = false) + val `ISO/IEC 13522` = bin"1111 0011".toInt(signed = false) + val `ITU-T Rec. H.222.1 type A` = bin"1111 0100".toInt(signed = false) + val `ITU-T Rec. H.222.1 type B` = bin"1111 0101".toInt(signed = false) + val `ITU-T Rec. H.222.1 type C` = bin"1111 0110".toInt(signed = false) + val `ITU-T Rec. H.222.1 type D` = bin"1111 0111".toInt(signed = false) + val `ITU-T Rec. H.222.1 type E` = bin"1111 1000".toInt(signed = false) + val Ancillary = bin"1111 1001".toInt(signed = false) + val `ISO/IEC14496-1 SL Packetized` = bin"1111 1010".toInt(signed = false) + val `ISO/IEC14496-1 FlexMux` = bin"1111 1011".toInt(signed = false) + val ReservedMin = bin"1111 1100".toInt(signed = false) + val ReservedMax = bin"1111 1110".toInt(signed = false) + val ProgramStreamDirectory = bin"1111 1111".toInt(signed = false) + // format:on +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/package.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/package.scala new file mode 100644 index 0000000000..3f78ca6582 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/package.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols + +import scodec.Codec +import scodec.bits._ +import scodec.codecs._ + +package object mpeg { + + def reserved(bits: Int): Codec[Unit] = constantLenient(BitVector.high(bits.toLong)) + + val crc32mpeg: BitVector => BitVector = + crc(hex"04c11db7".bits, BitVector.high(32), false, false, BitVector.low(32)) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala new file mode 100644 index 0000000000..abb93f06f4 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala @@ -0,0 +1,86 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.{ Attempt, Codec, DecodeResult, SizeBound } +import scodec.bits.BitVector +import scodec.codecs._ + +/** + * Partial modelling of the adaptation field. + * The field extension, if present, is ignored upon decoding. + */ +case class AdaptationField( + flags: Option[AdaptationFieldFlags], + pcr: Option[Clock27MHz], + opcr: Option[Clock27MHz], + spliceCountdown: Option[Int], + transportPrivateData: Option[BitVector] +) + +object AdaptationField { + + final val Empty: AdaptationField = AdaptationField(None, None, None, None, None) + + implicit val codec: Codec[AdaptationField] = new Codec[AdaptationField] { + private case class NonEmptyAF( + flags: AdaptationFieldFlags, + pcr: Option[Clock27MHz], + opcr: Option[Clock27MHz], + spliceCountdown: Option[Int], + transportPrivateData: Option[BitVector] + ) { def asAF: AdaptationField = AdaptationField(Some(flags), pcr, opcr, spliceCountdown, transportPrivateData) } + + private val pcrCodec: Codec[Clock27MHz] = + ((ulong(33) <~ ignore(6)) :: uint(9)).xmap[Clock27MHz]({ case (base, ext) => + Clock27MHz(base * 300 + ext) + }, { clock => + val value = clock.value + val base = value / 300 + val ext = (value % 300).toInt + (base, ext) + }) + private val transportPrivateData: Codec[BitVector] = variableSizeBits(uint8, bits) + private val nonEmptyAFCodec: Codec[NonEmptyAF] = "adaptation_field" | { + variableSizeBytes(uint8, + ("adaptation_flags" | Codec[AdaptationFieldFlags] ).flatPrepend { flags => + ("pcr" | conditional(flags.pcrFlag, pcrCodec) ) :: + ("opcr" | conditional(flags.opcrFlag, pcrCodec) ) :: + ("splice_countdown" | conditional(flags.splicingPointFlag, int8) ) :: + ("transport_private_data" | conditional(flags.transportPrivateDataFlag, transportPrivateData)) + }) + }.as[NonEmptyAF] + + def sizeBound: SizeBound = SizeBound.unknown + + def encode(af: AdaptationField): Attempt[BitVector] = af.flags.fold(uint8.encode(0)) { flags => + nonEmptyAFCodec.encode(NonEmptyAF(flags, af.pcr, af.opcr, af.spliceCountdown, af.transportPrivateData)) + } + + def decode(bv: BitVector): Attempt[DecodeResult[AdaptationField]] = uint8.decode(bv) flatMap { size => + if (size.value > 0) nonEmptyAFCodec.decode(bv).map(_.map(_.asAF)) else Attempt.successful(DecodeResult(Empty, size.remainder)) + } + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala new file mode 100644 index 0000000000..6313cc3c4c --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Codec +import scodec.codecs._ + +/** Flags in the adaptation field. */ +case class AdaptationFieldFlags( + discontinuity: Boolean, + randomAccess: Boolean, + priority: Boolean, + pcrFlag: Boolean, + opcrFlag: Boolean, + splicingPointFlag: Boolean, + transportPrivateDataFlag: Boolean, + adaptationFieldExtension: Boolean) + +object AdaptationFieldFlags { + implicit val codec: Codec[AdaptationFieldFlags] = "adaptation_field_flags" | fixedSizeBytes(1, + ("discontinuity" | bool ) :: + ("randomAccess" | bool ) :: + ("priority" | bool ) :: + ("pcrFlag" | bool ) :: + ("opcrFlag" | bool ) :: + ("splicingPointFlag" | bool ) :: + ("transportPrivateDataFlag" | bool ) :: + ("adaptationFieldExtension" | bool ) + ).as[AdaptationFieldFlags] +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala new file mode 100644 index 0000000000..99028b3aa0 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg.transport + +import scala.concurrent.duration.{FiniteDuration, MICROSECONDS} + +case class Clock27MHz(value: Long) { + def toDuration: FiniteDuration = FiniteDuration(((1000000d / 27000000) * value).toLong, MICROSECONDS) + + def +(that: Clock27MHz): Clock27MHz = Clock27MHz(value + that.value) + def -(that: Clock27MHz): Clock27MHz = Clock27MHz(value - that.value) +} \ No newline at end of file diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ContinuityCounter.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ContinuityCounter.scala new file mode 100644 index 0000000000..165334dbee --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ContinuityCounter.scala @@ -0,0 +1,41 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Codec +import scodec.codecs.uint + +case class ContinuityCounter(value: Int) { + require(value >= ContinuityCounter.MinValue && value <= ContinuityCounter.MaxValue) + + def next: ContinuityCounter = ContinuityCounter((value + 1) % 16) +} + +object ContinuityCounter { + val MinValue = 0 + val MaxValue = 15 + + implicit val codec: Codec[ContinuityCounter] = uint(4).xmap(ContinuityCounter.apply, _.value) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala new file mode 100644 index 0000000000..702c7c0aec --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala @@ -0,0 +1,250 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport + +import scodec.{ Attempt, Codec, DecodeResult, Err } +import scodec.Decoder +import scodec.bits._ +import scodec.codecs.fixedSizeBits + +import fs2.protocols.mpeg.transport.psi.{ Section, SectionHeader, SectionCodec } + +/** Supports depacketization of an MPEG transport stream, represented as a stream of `Packet`s. */ +object Demultiplexer { + + sealed trait Result + case class SectionResult(section: Section) extends Result + case class PesPacketResult(body: PesPacket) extends Result + + /** + * Indication that a header was decoded successfully and there was enough information on how to decode the body of the message. + * + * Upon receiving a result of this type, the demultiplexer will accumulate the number of bits specified by `neededBits` if that value + * is defined. If `neededBits` is undefined, the demultiplexer will accumulate all payload bits until the start of the next message (as + * indicated by the payload unit start indicator). When accumulation has completed, the specified decoder will be invoked to decode + * a message. + */ + case class DecodeBody[A](neededBits: Option[Long], decoder: Decoder[A]) + + /** Error that indicates any data accumulated by the demultiplexer should be dropped and no further decoding should occur until the next + * payload start. */ + case class ResetDecodeState(context: List[String]) extends Err { + def message = "reset decode state" + def pushContext(ctx: String) = ResetDecodeState(ctx :: context) + } + + final case class State(byPid: Map[Pid, DecodeState]) + + sealed trait DecodeState + object DecodeState { + + final case class AwaitingHeader(acc: BitVector, startedAtOffsetZero: Boolean) extends DecodeState + + final case class AwaitingBody[A](headerBits: BitVector, neededBits: Option[Long], bitsPostHeader: BitVector, decoder: Decoder[A]) extends DecodeState { + def decode: Attempt[DecodeResult[A]] = decoder.decode(bitsPostHeader) + def accumulate(data: BitVector): AwaitingBody[A] = copy(bitsPostHeader = bitsPostHeader ++ data) + } + } + + private case class StepResult[+A](state: Option[DecodeState], output: Chunk[Either[DemultiplexerError, A]]) { + def ++[AA >: A](that: StepResult[AA]): StepResult[AA] = StepResult(that.state, Chunk.concat(List(output, that.output))) + } + private object StepResult { + def noOutput(state: Option[DecodeState]): StepResult[Nothing] = apply(state, Chunk.empty) + def state(state: DecodeState): StepResult[Nothing] = StepResult(Some(state), Chunk.empty) + def oneResult[A](state: Option[DecodeState], output: A): StepResult[A] = apply(state, Chunk.singleton(Right(output))) + def oneError(state: Option[DecodeState], err: DemultiplexerError): StepResult[Nothing] = apply(state, Chunk.singleton(Left(err))) + } + + /** + * Stream transducer that converts packets in to sections and PES packets. + * + * The packets may span PID values. De-packetization is performed on each PID and as whole messages are received, + * reassembled messages are emitted. + * + * PES packets emitted by this method never include parsed headers -- that is, every emitted PES packet is of + * type `PesPacket.WithoutHeader`. To get PES packets with parsed headers, use `demultiplexWithPesHeaders`. + * + * Errors encountered while depacketizing are emitted. + * + * Upon noticing a PID discontinuity, an error is emitted and PID decoding state is discarded, resulting in any in-progress + * section decoding to be lost for that PID. + */ + def demultiplex[F[_]](sectionCodec: SectionCodec): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = + demultiplexSectionsAndPesPackets(sectionCodec.decoder, pph => Decoder(b => Attempt.successful(DecodeResult(PesPacket.WithoutHeader(pph.streamId, b), BitVector.empty)))) + + /** Variant of `demultiplex` that parses PES packet headers. */ + def demultiplexWithPesHeaders[F[_]](sectionCodec: SectionCodec): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = + demultiplexSectionsAndPesPackets(sectionCodec.decoder, PesPacket.decoder) + + /** Variant of `demultiplex` that allows section and PES decoding to be explicitly specified. */ + def demultiplexSectionsAndPesPackets[F[_]]( + decodeSectionBody: SectionHeader => Decoder[Section], + decodePesBody: PesPacketHeaderPrefix => Decoder[PesPacket]): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = { + + val stuffingByte = bin"11111111" + + def decodeHeader(data: BitVector, startedAtOffsetZero: Boolean): Attempt[DecodeResult[DecodeBody[Result]]] = { + if (data.sizeLessThan(16)) { + Attempt.failure(Err.InsufficientBits(16, data.size, Nil)) + } else if (data startsWith stuffingByte) { + Attempt.failure(ResetDecodeState(Nil)) + } else { + if (startedAtOffsetZero && data.take(16) == hex"0001".bits) { + if (data.sizeLessThan(40)) { + Attempt.failure(Err.InsufficientBits(40, data.size, Nil)) + } else { + Codec[PesPacketHeaderPrefix].decode(data.drop(16)) map { _ map { header => + val neededBits = if (header.length == 0) None else Some(header.length * 8L) + DecodeBody(neededBits, decodePesBody(header).map(PesPacketResult.apply)) + }} + } + } else { + if (data.sizeLessThan(24)) { + Attempt.failure(Err.InsufficientBits(24, data.size, Nil)) + } else { + Codec[SectionHeader].decode(data) map { _ map { header => + DecodeBody(Some(header.length * 8L), decodeSectionBody(header).map(SectionResult.apply)) + }} + } + } + } + } + demultiplexGeneral(decodeHeader) + } + + /** + * Most general way to perform demultiplexing, allowing parsing of arbitrary headers and decoding of a specified output type. + * + * When processing the payload in a packet, the start of the payload is passed along to `decodeHeader`, which determines how to + * process the body of the message. + * + * In addition to the payload data, a flag is passed to `decodeHeader` -- true is passed when the payload data started at byte 0 of + * the packet and false is passed when the payload data started later in the packet. + * + * See the documentation on `DecodeBody` for more information. + */ + def demultiplexGeneral[F[_], Out]( + decodeHeader: (BitVector, Boolean) => Attempt[DecodeResult[DecodeBody[Out]]] + ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Out]]] = { + + def processBody[A](awaitingBody: DecodeState.AwaitingBody[A], payloadUnitStartAfterData: Boolean): StepResult[Out] = { + val haveFullBody = awaitingBody.neededBits match { + case None => payloadUnitStartAfterData + case Some(needed) => awaitingBody.bitsPostHeader.size >= needed + } + if (haveFullBody) { + awaitingBody.decode match { + case Attempt.Successful(DecodeResult(body, remainder)) => + val decoded = StepResult.oneResult(None, body.asInstanceOf[Out]) // Safe cast b/c DecodeBody must provide a Decoder[Out] + decoded ++ processHeader(remainder, false, payloadUnitStartAfterData) + case Attempt.Failure(err) => + val out = { + if (err.isInstanceOf[ResetDecodeState]) Chunk.empty + else Chunk.singleton(Left(DemultiplexerError.Decoding( + awaitingBody.headerBits ++ + awaitingBody.neededBits. + map { n => awaitingBody.bitsPostHeader.take(n) }. + getOrElse(awaitingBody.bitsPostHeader), err))) + } + val failure = StepResult(None, out) + awaitingBody.neededBits match { + case Some(n) => + val remainder = awaitingBody.bitsPostHeader.drop(n.toLong) + failure ++ processHeader(remainder, false, payloadUnitStartAfterData) + case None => failure + } + } + } else { + StepResult.state(awaitingBody) + } + } + + def processHeader(acc: BitVector, startedAtOffsetZero: Boolean, payloadUnitStartAfterData: Boolean): StepResult[Out] = { + decodeHeader(acc, startedAtOffsetZero) match { + case Attempt.Failure(e: Err.InsufficientBits) => + StepResult.state(DecodeState.AwaitingHeader(acc, startedAtOffsetZero)) + case Attempt.Failure(_: ResetDecodeState) => + StepResult.noOutput(None) + case Attempt.Failure(e) => + StepResult.oneError(None, DemultiplexerError.Decoding(acc, e)) + case Attempt.Successful(DecodeResult(DecodeBody(neededBits, decoder), bitsPostHeader)) => + val guardedDecoder = neededBits match { + case None => decoder + case Some(n) => fixedSizeBits(n, decoder.decodeOnly) + } + processBody(DecodeState.AwaitingBody(acc.take(24L), neededBits, bitsPostHeader, guardedDecoder), payloadUnitStartAfterData) + } + } + + def resume(state: DecodeState, newData: BitVector, payloadUnitStartAfterData: Boolean): StepResult[Out] = state match { + case ah: DecodeState.AwaitingHeader => + processHeader(ah.acc ++ newData, ah.startedAtOffsetZero, payloadUnitStartAfterData) + + case ab: DecodeState.AwaitingBody[_] => + processBody(ab.accumulate(newData), payloadUnitStartAfterData) + } + + def handlePacket(state: Option[DecodeState], packet: Packet): StepResult[Out] = { + packet.payload match { + case None => StepResult.noOutput(state) + case Some(payload) => + val currentResult = state match { + case None => StepResult.noOutput(state) + case Some(state) => + val currentData = packet.payloadUnitStart.map { start => payload.take(start.toLong * 8L) }.getOrElse(payload) + resume(state, currentData, payloadUnitStartAfterData = packet.payloadUnitStart.isDefined) + } + packet.payloadUnitStart match { + case None => + currentResult + case Some(start) => + val nextResult = processHeader(payload.drop(start * 8L), start == 0, false) + currentResult ++ nextResult + } + } + } + + val demux = Scan.stateful[State, Either[PidStamped[DemultiplexerError.Discontinuity], Packet], PidStamped[Either[DemultiplexerError, Out]]](State(Map.empty)) { (state, event) => + event match { + case Right(packet) => + val pid = packet.header.pid + val oldStateForPid = state.byPid.get(pid) + val result = handlePacket(oldStateForPid, packet) + val newState = State(result.state.map { s => state.byPid.updated(pid, s) }.getOrElse(state.byPid - pid)) + val out = result.output.map { e => PidStamped(pid, e) } + (newState, out) + case Left(discontinuity) => + val newState = State(state.byPid - discontinuity.pid) + val out = Chunk.singleton(PidStamped(discontinuity.pid, Left(discontinuity.value))) + (newState, out) + } + } + + Packet.validateContinuity andThen demux + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala new file mode 100644 index 0000000000..1053894599 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Err +import scodec.bits.BitVector + +sealed abstract class DemultiplexerError { + def toMpegError: MpegError +} + +object DemultiplexerError { + + case class Discontinuity(last: ContinuityCounter, current: ContinuityCounter, adaptationFieldControl: Int) extends DemultiplexerError with MpegError { + def message = s"pid discontinuity: $last to $current with adaptation field control $adaptationFieldControl" + def toMpegError = this + } + + case class Decoding(data: BitVector, decodingError: Err) extends DemultiplexerError { + def message = s"decoding error ($decodingError) while decoding ${data.toHex}" + def toMpegError = MpegError.Decoding(data, decodingError) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala new file mode 100644 index 0000000000..30b6b5103e --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala @@ -0,0 +1,148 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport + +import scodec.Codec +import scodec.bits.BitVector +import scodec.codecs._ + +/** Transport stream packet. */ +case class Packet( + header: TransportStreamHeader, + adaptationField: Option[AdaptationField], + payloadUnitStart: Option[Int], + payload: Option[BitVector] +) + +object Packet { + + def packetize(pid: Pid, startingCountinuityCounter: ContinuityCounter, section: BitVector): Vector[Packet] = { + @annotation.tailrec + def go(first: Boolean, cc: ContinuityCounter, remaining: BitVector, acc: Vector[Packet]): Vector[Packet] = { + if (remaining.isEmpty) acc + else { + val (packetData, remData) = remaining.splitAt(8L * (if (first) 183 else 184)) + go(false, cc.next, remData, acc :+ payload(pid, cc, if (first) Some(0) else None, packetData)) + } + } + go(true, startingCountinuityCounter, section, Vector.empty) + } + + def packetizeMany(pid: Pid, startingCountinuityCounter: ContinuityCounter, sections: Vector[BitVector]): Vector[Packet] = { + + /* + * Accumulates up to `n` bits from the specified bit vectors. + * Returns a triple consisting of: + * - the accumulated bits (up to size `n`) + * - the left over bits of the last consumed input section + * - the remaining unconsumed sections + */ + def accumulateN(n: Long, sections: Vector[BitVector]): (BitVector, BitVector, Vector[BitVector]) = { + @annotation.tailrec + def go(needed: Long, remainingSections: Vector[BitVector], acc: BitVector): (BitVector, BitVector, Vector[BitVector]) = { + if (remainingSections.isEmpty) (acc, BitVector.empty, Vector.empty) + else { + val (x, rem) = remainingSections.head.splitAt(needed) + val newAcc = acc ++ x + val left = needed - x.size + if (left == 0) (newAcc, rem, remainingSections.tail) + else go(left, remainingSections.tail, newAcc) + } + } + go(n, sections, BitVector.empty) + } + + @annotation.tailrec + def go(cc: ContinuityCounter, remaining: BitVector, remainingSections: Vector[BitVector], acc: Vector[Packet]): Vector[Packet] = { + if (remaining.isEmpty && remainingSections.isEmpty) acc + else { + val (packetData, overflow, remSections) = accumulateN(184 * 8, remaining +: remainingSections) + val payloadUnitStart = { + if (remSections.size < remainingSections.size) Some((remaining.size / 8).toInt) + else None + } + val (adjPacketData, adjOverflow) = { + if (payloadUnitStart.isDefined) (packetData.take(183 * 8), packetData.drop(183 * 8) ++ overflow) + else (packetData, overflow) + } + val packet = payload(pid, cc, payloadUnitStart, adjPacketData) + go(cc.next, adjOverflow, remSections, acc :+ packet) + } + } + go(startingCountinuityCounter, BitVector.empty, sections, Vector.empty) + } + + def payload(pid: Pid, continuityCounter: ContinuityCounter, payloadUnitStart: Option[Int], payload: BitVector): Packet = { + val thisPid = pid + val thisContinuityCounter = continuityCounter + val thisPayloadUnitStart = payloadUnitStart + val payloadLength = 8 * (if (payloadUnitStart.isDefined) 183 else 184) + require(payload.length <= payloadLength, s"payload too long; must be <= $payloadLength") + val thisPayload = payload ++ BitVector.high(payloadLength - payload.length) + Packet( + header = TransportStreamHeader( + transportErrorIndicator = false, + payloadUnitStartIndicator = payloadUnitStart.isDefined, + transportPriority = false, + pid = thisPid, + scramblingControl = 0, + adaptationFieldControl = 1, + continuityCounter = thisContinuityCounter + ), + adaptationField = None, + payloadUnitStart = thisPayloadUnitStart, + payload = Some(thisPayload)) + } + + implicit def codec(implicit adaptationField: Codec[AdaptationField]): Codec[Packet] = + "packet" | fixedSizeBytes(188, + ("header" | Codec[TransportStreamHeader] ).flatPrepend { hdr => + ("adaptation_field" | conditional(hdr.adaptationFieldIncluded, adaptationField) ) :: + ("payload_start_ind" | conditional(hdr.payloadUnitStartIndicator, uint8) ) :: + ("payload" | conditional(hdr.payloadIncluded, bits) ) + }).as[Packet] + + def validateContinuity: Scan[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[DemultiplexerError.Discontinuity], Packet]] = + Scan.stateful[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[DemultiplexerError.Discontinuity], Packet]](Map.empty) { (state, packet) => + val pid = packet.header.pid + val currentContinuityCounter = packet.header.continuityCounter + val err = state.get(pid).map { lastContinuityCounter => + val expectedContinuityCounter = + if (packet.header.adaptationFieldControl == 0 || packet.header.adaptationFieldControl == 2) lastContinuityCounter else lastContinuityCounter.next + if (expectedContinuityCounter == currentContinuityCounter) { + None + } else { + val err: Either[PidStamped[DemultiplexerError.Discontinuity], Packet] = + Left(PidStamped(pid, DemultiplexerError.Discontinuity(lastContinuityCounter, currentContinuityCounter, packet.header.adaptationFieldControl))) + Some(err) + } + }.getOrElse(None) + val newState = state + (pid -> currentContinuityCounter) + val out = err.map { e => Chunk(e, Right(packet)) }.getOrElse(Chunk.singleton(Right(packet))) + (newState, out) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala new file mode 100644 index 0000000000..218c3b01e9 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport + +import scodec.Codec +import scodec.codecs.uint + +case class Pid(value: Int) { + require(value >= Pid.MinValue && value <= Pid.MaxValue) +} + +object Pid { + val MinValue = 0 + val MaxValue = 8191 + + implicit val codec: Codec[Pid] = uint(13).as[Pid] +} + +case class PidStamped[+A](pid: Pid, value: A) { + def map[B](f: A => B): PidStamped[B] = copy(value = f(value)) +} + +object PidStamped { + + /** + * Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, PidStamped[I], PidStamped[O]]` such that + * pidstamps are preserved on elements that flow through the stream. + */ + def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, PidStamped[I], PidStamped[O]] = + t.lens(_.value, (psi, o) => psi.copy(value = o)) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ProgramNumber.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ProgramNumber.scala new file mode 100644 index 0000000000..17ab700c13 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/ProgramNumber.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Codec +import scodec.codecs.uint16 + +case class ProgramNumber(value: Int) { + require(value >= ProgramNumber.MinValue && value <= ProgramNumber.MaxValue) +} + +object ProgramNumber { + val MinValue = 0 + val MaxValue = 65535 + + implicit val codec: Codec[ProgramNumber] = uint16.xmap(ProgramNumber.apply, _.value) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala new file mode 100644 index 0000000000..4ad40b4a46 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Codec +import scodec.codecs._ + +case class TransportStreamHeader( + transportErrorIndicator: Boolean, + payloadUnitStartIndicator: Boolean, + transportPriority: Boolean, + pid: Pid, + scramblingControl: Int, + adaptationFieldControl: Int, + continuityCounter: ContinuityCounter +) { + def adaptationFieldIncluded: Boolean = adaptationFieldControl >= 2 + def payloadIncluded: Boolean = adaptationFieldControl == 1 || adaptationFieldControl == 3 +} + +object TransportStreamHeader { + implicit val codec: Codec[TransportStreamHeader] = "transport_stream_header" | fixedSizeBytes(4, + ("syncByte" | constant(0x47) ) ~> + ("transportErrorIndicator" | bool ) :: + ("payloadUnitStartIndicator" | bool ) :: + ("transportPriority" | bool ) :: + ("pid" | Codec[Pid] ) :: + ("scramblingControl" | uint2 ) :: + ("adaptationFieldControl" | uint2 ) :: + ("continuityCounter" | Codec[ContinuityCounter]) + ).as[TransportStreamHeader] +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamId.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamId.scala new file mode 100644 index 0000000000..f18367944f --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamId.scala @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport + +import scodec.Codec +import scodec.codecs.uint16 + +case class TransportStreamId(value: Int) { + require(value >= TransportStreamId.MinValue && value <= TransportStreamId.MaxValue) +} + +object TransportStreamId { + val MinValue = 0 + val MaxValue = 65535 + + implicit val codec: Codec[TransportStreamId] = uint16.xmap(TransportStreamId.apply, _.value) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala new file mode 100644 index 0000000000..0181337898 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala @@ -0,0 +1,141 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.Codec +import scodec.bits._ +import scodec.codecs._ + +case class ConditionalAccessTable( + version: Int, + current: Boolean, + descriptors: List[ConditionalAccessDescriptor] +) extends Table { + def tableId = ConditionalAccessSection.TableId + def toSections: GroupedSections[ConditionalAccessSection] = ConditionalAccessTable.toSections(this) +} + +object ConditionalAccessTable { + + def toSections(cat: ConditionalAccessTable): GroupedSections[ConditionalAccessSection] = { + val grouped = groupBasedOnSize(cat.descriptors.toVector) + val lastSection = grouped.size - 1 + val sections = grouped.zipWithIndex.map { case (ds, idx) => + ConditionalAccessSection(SectionExtension(65535, cat.version, cat.current, idx, lastSection), ds.toList) + } + if (sections.isEmpty) + GroupedSections(ConditionalAccessSection(SectionExtension(65535, cat.version, cat.current, 0, 0), Nil)) + else + GroupedSections(sections.head, sections.tail.toList) + } + + private def groupBasedOnSize(sections: Vector[ConditionalAccessDescriptor]): Vector[Vector[ConditionalAccessDescriptor]] = { + val MaxBitsLeft = (1024 - 12) * 8L + def sizeOf(c: ConditionalAccessDescriptor): Long = (6 * 8) + c.privateData.size + @annotation.tailrec + def go(remaining: Vector[ConditionalAccessDescriptor], cur: Vector[ConditionalAccessDescriptor], bitsLeft: Long, acc: Vector[Vector[ConditionalAccessDescriptor]]): Vector[Vector[ConditionalAccessDescriptor]] = { + if (remaining.isEmpty) acc :+ cur + else { + val next = remaining.head + val bitsNeeded = (6 * 8) + sizeOf(next) + val newBitsLeft = bitsLeft - bitsNeeded + if (newBitsLeft >= 0) go(remaining.tail, cur :+ next, newBitsLeft, acc) + else { + go(remaining, Vector.empty, MaxBitsLeft, acc :+ cur) + } + } + } + go(sections, Vector.empty, MaxBitsLeft, Vector.empty) + } + + def fromSections(sections: GroupedSections[ConditionalAccessSection]): Either[String, ConditionalAccessTable] = { + def extract[A](name: String, f: ConditionalAccessSection => A): Either[String, A] = { + val extracted = sections.list.map(f).distinct + if (extracted.size == 1) Right(extracted.head) + else Left(s"sections have diferring $name: " + extracted.mkString(", ")) + } + for { + version <- extract("versions", _.extension.version) + } yield { + val current = sections.list.foldLeft(false) { (acc, s) => acc || s.extension.current } + ConditionalAccessTable( + version, + current, + (for { + section <- sections.list + descriptor <- section.descriptors + } yield descriptor) + ) + } + } + + implicit val tableSupport: TableSupport[ConditionalAccessTable] = new TableSupport[ConditionalAccessTable] { + def tableId = ConditionalAccessSection.TableId + def toTable(gs: GroupedSections[Section]) = + gs.narrow[ConditionalAccessSection].toRight(s"Not CAT sections").flatMap { sections => fromSections(sections) } + def toSections(cat: ConditionalAccessTable) = ConditionalAccessTable.toSections(cat) + } +} + +case class ConditionalAccessSection( + extension: SectionExtension, + descriptors: List[ConditionalAccessDescriptor] +) extends ExtendedSection { + def tableId = ConditionalAccessSection.TableId +} + +object ConditionalAccessSection { + val TableId = 1 + + type Fragment = List[ConditionalAccessDescriptor] + + private val fragmentCodec: Codec[Fragment] = + list(Codec[ConditionalAccessDescriptor]) + + implicit val sectionFragmentCodec: SectionFragmentCodec[ConditionalAccessSection] = + SectionFragmentCodec.psi[ConditionalAccessSection, Fragment]( + TableId, + (ext, descriptors) => ConditionalAccessSection(ext, descriptors), + cat => (cat.extension, cat.descriptors) + )(fragmentCodec) +} + +case class ConditionalAccessDescriptor(systemId: Int, pid: Pid, privateData: BitVector) + +object ConditionalAccessDescriptor { + val Tag = 9 + + implicit val codec: Codec[ConditionalAccessDescriptor] = { + constant(Tag) ~> + variableSizeBytes(uint8, + ("ca_system_id" | uint16) :: + (reserved(3) ~> + ("ca_pid" | Codec[Pid])) :: + bits + ) + }.as[ConditionalAccessDescriptor] + +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala new file mode 100644 index 0000000000..db30aacaf4 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala @@ -0,0 +1,143 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package mpeg +package transport +package psi + +import scala.reflect.ClassTag + +import fs2._ + +/** + * Group of sections that make up a logical message. + * + * Intermediate representation between sections and tables. All sections must share the same table id. + */ +sealed abstract class GroupedSections[+A <: Section] { + def tableId: Int + + def head: A + def tail: List[A] + + def list: List[A] +} + +object GroupedSections { + implicit class InvariantOps[A <: Section](val self: GroupedSections[A]) extends AnyVal { + def narrow[B <: A : ClassTag]: Option[GroupedSections[B]] = { + val matched = self.list.foldLeft(true) { (acc, s) => s match { case _: B => true; case _ => false } } + if (matched) Some(self.asInstanceOf[GroupedSections[B]]) + else None + } + } + + private case class DefaultGroupedSections[A <: Section](head: A, tail: List[A]) extends GroupedSections[A] { + val tableId = head.tableId + val list = head :: tail + } + + def apply[A <: Section](head: A, tail: List[A] = Nil): GroupedSections[A] = + DefaultGroupedSections[A](head, tail) + + final case class ExtendedTableId(tableId: Int, tableIdExtension: Int) + final case class ExtendedSectionGrouperState[A <: ExtendedSection](accumulatorByIds: Map[ExtendedTableId, SectionAccumulator[A]]) + + def groupExtendedSections[A <: ExtendedSection]: Scan[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]] = { + def toKey(section: A): ExtendedTableId = ExtendedTableId(section.tableId, section.extension.tableIdExtension) + Scan.stateful[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]](ExtendedSectionGrouperState(Map.empty)) { (state, section) => + val key = toKey(section) + val (err, acc) = state.accumulatorByIds.get(key) match { + case None => (None, SectionAccumulator(section)) + case Some(acc) => + acc.add(section) match { + case Right(acc) => (None, acc) + case Left(err) => (Some(GroupingError(section.tableId, section.extension.tableIdExtension, err)), SectionAccumulator(section)) + } + } + + acc.complete match { + case None => + val newState = ExtendedSectionGrouperState(state.accumulatorByIds + (key -> acc)) + val out = err.map(e => Chunk.singleton(Left(e))).getOrElse(Chunk.empty) + (newState, out) + case Some(sections) => + val newState = ExtendedSectionGrouperState(state.accumulatorByIds - key) + val out = Chunk.seq((Right(sections) :: err.map(e => Left(e)).toList).reverse) + (newState, out) + } + } + } + + def noGrouping: Scan[Unit, Section, Either[GroupingError, GroupedSections[Section]]] = + Scan.lift(s => Right(GroupedSections(s))) + + /** + * Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. + * Non-extended sections are emitted as singleton groups. + */ + def group: Scan[ExtendedSectionGrouperState[ExtendedSection], Section, Either[GroupingError, GroupedSections[Section]]] = { + groupGeneral((), noGrouping).imapState(_._2)(s => ((), s)) + } + + /** + * Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. + * The specified `nonExtended` process is used to handle non-extended sections. + */ + def groupGeneral[NonExtendedState]( + initialNonExtendedState: NonExtendedState, + nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]] + ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]] = { + groupGeneralConditionally(initialNonExtendedState, nonExtended, _ => true) + } + + /** + * Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled if `true` is returned from the + * `groupExtended` function when applied with the section in question. + * + * The specified `nonExtended` transducer is used to handle non-extended sections. + */ + def groupGeneralConditionally[NonExtendedState]( + initialNonExtendedState: NonExtendedState, + nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]], + groupExtended: ExtendedSection => Boolean = _ => true + ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]] = { + Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]]((initialNonExtendedState, ExtendedSectionGrouperState(Map.empty)))({ case ((nonExtendedState, extendedState), section) => + section match { + case s: ExtendedSection if groupExtended(s) => + val (s2, out) = groupExtendedSections.transform(extendedState, s) + (nonExtendedState -> s2, out) + case s: Section => + val (s2, out) = nonExtended.transform(nonExtendedState, s) + (s2 -> extendedState, out) + } + }, { case (nonExtendedState, extendedState) => Chunk.concat(List(nonExtended.onComplete(nonExtendedState), groupExtendedSections.onComplete(extendedState))) }) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala new file mode 100644 index 0000000000..7356b65201 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +case class GroupingError(tableId: Int, tableIdExtension: Option[Int], message: String) extends MpegError + +object GroupingError { + def apply(tableId: Int, tableIdExtension: Int, message: String): GroupingError = + new GroupingError(tableId, Some(tableIdExtension), message) + + def apply(tableId: Int, message: String): GroupingError = + new GroupingError(tableId, None, message) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala new file mode 100644 index 0000000000..50b1775d32 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala @@ -0,0 +1,115 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.Codec +import scodec.codecs._ + +case class ProgramAssociationTable( + tsid: TransportStreamId, + version: Int, + current: Boolean, + programByPid: Map[ProgramNumber, Pid] +) extends Table { + def tableId = ProgramAssociationSection.TableId + def toSections: GroupedSections[ProgramAssociationSection] = ProgramAssociationTable.toSections(this) +} + +object ProgramAssociationTable { + + val MaxProgramsPerSection = 253 + + def toSections(pat: ProgramAssociationTable): GroupedSections[ProgramAssociationSection] = { + val entries = pat.programByPid.toVector.sortBy { case (ProgramNumber(n), _) => n } + val groupedEntries = entries.grouped(MaxProgramsPerSection).toVector + val lastSection = groupedEntries.size - 1 + val sections = groupedEntries.zipWithIndex.map { case (es, idx) => + ProgramAssociationSection(SectionExtension(pat.tsid.value, pat.version, pat.current, idx, lastSection), es) + } + if (sections.isEmpty) + GroupedSections(ProgramAssociationSection(SectionExtension(pat.tsid.value, pat.version, pat.current, 0, 0), Vector.empty)) + else + GroupedSections(sections.head, sections.tail.toList) + } + + def fromSections(sections: GroupedSections[ProgramAssociationSection]): Either[String, ProgramAssociationTable] = { + def extract[A](name: String, f: ProgramAssociationSection => A): Either[String, A] = { + val extracted = sections.list.map(f).distinct + if (extracted.size == 1) Right(extracted.head) + else Left(s"sections have diferring $name: " + extracted.mkString(", ")) + } + for { + tsid <- extract("TSIDs", _.tsid) + version <- extract("versions", _.extension.version) + } yield { + val current = sections.list.foldLeft(false) { (acc, s) => acc || s.extension.current } + ProgramAssociationTable( + tsid, + version, + current, + (for { + section <- sections.list + pidMapping <- section.pidMappings + } yield pidMapping).toMap) + } + } + + implicit val tableSupport: TableSupport[ProgramAssociationTable] = new TableSupport[ProgramAssociationTable] { + def tableId = ProgramAssociationSection.TableId + def toTable(gs: GroupedSections[Section]) = + gs.narrow[ProgramAssociationSection].toRight("Not PAT sections").flatMap { sections => fromSections(sections) } + def toSections(pat: ProgramAssociationTable) = ProgramAssociationTable.toSections(pat) + } +} + +case class ProgramAssociationSection( + extension: SectionExtension, + pidMappings: Vector[(ProgramNumber, Pid)] +) extends ExtendedSection { + def tableId = ProgramAssociationSection.TableId + def tsid: TransportStreamId = TransportStreamId(extension.tableIdExtension) +} + +object ProgramAssociationSection { + val TableId = 0 + + private type Fragment = Vector[(ProgramNumber, Pid)] + + private val fragmentCodec: Codec[Fragment] = { + vector { + ("program_number" | Codec[ProgramNumber]) :: + (reserved(3) ~> + ("pid" | Codec[Pid])) + } + } + + implicit val sectionFragmentCodec: SectionFragmentCodec[ProgramAssociationSection] = + SectionFragmentCodec.psi[ProgramAssociationSection, Vector[(ProgramNumber, Pid)]]( + TableId, + (ext, mappings) => ProgramAssociationSection(ext, mappings), + pat => (pat.extension, pat.pidMappings) + )(fragmentCodec) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala new file mode 100644 index 0000000000..aff15c5c56 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala @@ -0,0 +1,122 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.Codec +import scodec.codecs._ + +import Descriptor._ + +case class ProgramMapTable( + programNumber: ProgramNumber, + version: Int, + current: Boolean, + pcrPid: Pid, + programInfoDescriptors: List[Descriptor], + componentStreamMapping: Map[StreamType, List[ProgramMapRecord]] +) extends Table { + def tableId = ProgramMapSection.TableId +} + +object ProgramMapTable { + + def toSection(pmt: ProgramMapTable): ProgramMapSection = { + ProgramMapSection( + SectionExtension(pmt.programNumber.value, pmt.version, pmt.current, 0, 0), + pmt.pcrPid, + pmt.programInfoDescriptors, + (for ((st, pmrs) <- pmt.componentStreamMapping.toVector; pmr <- pmrs) yield (st, pmr)).sortBy { case (k, v) => (k.value, v.pid.value) } + ) + } + + def fromSection(section: ProgramMapSection): ProgramMapTable = { + val componentStreamMapping = section.componentStreamMapping.foldLeft(Map.empty[StreamType, List[ProgramMapRecord]]) { case (acc, (st, pmr)) => + acc.updated(st, acc.get(st).fold(List(pmr))(existing => pmr :: existing)) + }.map { case (k, v) => (k, v.reverse) } + ProgramMapTable( + section.programNumber, + section.extension.version, + section.extension.current, + section.pcrPid, + section.programInfoDescriptors, + componentStreamMapping + ) + } + + implicit val tableSupport: TableSupport[ProgramMapTable] = new TableSupport[ProgramMapTable] { + def tableId = ProgramMapSection.TableId + def toTable(gs: GroupedSections[Section]) = + gs.narrow[ProgramMapSection].toRight("Not PMT sections").flatMap { sections => + if (sections.tail.isEmpty) Right(fromSection(sections.head)) + else Left(s"PMT supports only 1 section but got ${sections.list.size}") + } + def toSections(pmt: ProgramMapTable) = GroupedSections(ProgramMapTable.toSection(pmt)) + } +} + +case class StreamType(value: Int) +case class ProgramMapRecord(pid: Pid, descriptors: List[Descriptor]) +object ProgramMapRecord { + def apply(pid: Pid) = new ProgramMapRecord(pid, Nil) +} + +case class ProgramMapSection( + extension: SectionExtension, + pcrPid: Pid, + programInfoDescriptors: List[Descriptor], + componentStreamMapping: Vector[(StreamType, ProgramMapRecord)] +) extends ExtendedSection { + def tableId = ProgramMapSection.TableId + def programNumber: ProgramNumber = ProgramNumber(extension.tableIdExtension) +} + +object ProgramMapSection { + val TableId = 2 + + private type Fragment = (Pid, List[Descriptor], Vector[(StreamType, ProgramMapRecord)]) + private val fragmentCodec: Codec[Fragment] = { + def pid: Codec[Pid] = reserved(3) ~> Codec[Pid] + def descriptors: Codec[List[Descriptor]] = + reserved(4) ~> variableSizeBytes(uint(12), list(Descriptor.codec)) + def programMapRecord: Codec[ProgramMapRecord] = + (("pid" | pid) :: ("es_descriptors" | descriptors)).as[ProgramMapRecord] + + ("pcr_pid" | pid) :: + ("program_info_descriptors" | descriptors) :: + vector { + ("stream_type" | uint8.as[StreamType]) :: programMapRecord + } + } + + implicit val sectionSubCodec: SectionFragmentCodec[ProgramMapSection] = + SectionFragmentCodec.psi[ProgramMapSection, Fragment]( + TableId, + (ext, fragment) => fragment match { + case (pcrPid, descriptors, mapping) => ProgramMapSection(ext, pcrPid, descriptors, mapping) + }, + pmt => (pmt.extension, (pmt.pcrPid, pmt.programInfoDescriptors, pmt.componentStreamMapping)) + )(fragmentCodec) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala new file mode 100644 index 0000000000..676329bf9f --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala @@ -0,0 +1,56 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.Codec +import scodec.codecs._ + +trait Section { + def tableId: Int +} + +trait ExtendedSection extends Section { + def extension: SectionExtension +} + +case class SectionExtension( + tableIdExtension: Int, + version: Int, + current: Boolean, + sectionNumber: Int, + lastSectionNumber: Int +) + +object SectionExtension { + implicit val codec: Codec[SectionExtension] = { + ("table_id_extension" | uint16) :: + (reserved(2) ~> + ("version_number" | uint(5))) :: + ("current_next_indicator" | bool) :: + ("section_number" | uint8) :: + ("last_section_number" | uint8) + }.as[SectionExtension] +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala new file mode 100644 index 0000000000..05a83d955e --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala @@ -0,0 +1,59 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +/** Accumulates sections of the same table id and table id extension. */ +private[psi] class SectionAccumulator[A <: ExtendedSection] private (val sections: GroupedSections[A], sectionByNumber: Map[Int, A]) { + + def add(section: A): Either[String, SectionAccumulator[A]] = { + def validate(err: => String)(f: Boolean): Either[String, Unit] = + if (f) Right(()) else Left(err) + + def checkEquality[B](name: String)(f: A => B): Either[String, Unit] = + validate(name + " do not match")(f(section) == f(sections.head)) + + val sectionNumber = section.extension.sectionNumber + for { + _ <- checkEquality("table ids")(_.tableId) + _ <- checkEquality("table id extensions")(_.extension.tableIdExtension) + _ <- checkEquality("versions")(_.extension.version) + _ <- checkEquality("last section numbers")(_.extension.lastSectionNumber) + _ <- validate("invalid section number")(sectionNumber <= sections.head.extension.lastSectionNumber) + _ <- validate("duplicate section number")(!sectionByNumber.contains(sectionNumber)) + } yield new SectionAccumulator(GroupedSections(section, sections.list), sectionByNumber + (section.extension.sectionNumber -> section)) + } + + def complete: Option[GroupedSections[A]] = + if (sectionByNumber.size == (sections.head.extension.lastSectionNumber + 1)) Some(sections) else None +} + +private[psi] object SectionAccumulator { + + def apply[A <: ExtendedSection](section: A): SectionAccumulator[A] = + new SectionAccumulator(GroupedSections(section), Map(section.extension.sectionNumber -> section)) +} + + diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala new file mode 100644 index 0000000000..cc7daa0ee5 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala @@ -0,0 +1,156 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.{ Attempt, Codec, Decoder, DecodeResult, Err, SizeBound } +import scodec.bits._ +import scodec.codecs._ + +class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section]]], verifyCrc: Boolean = true) extends Codec[Section] { + import SectionCodec._ + + def supporting[A <: Section](implicit c: SectionFragmentCodec[A]): SectionCodec = { + val newCases = Case.fromSectionFragmentCodec(c) :: cases.getOrElse(c.tableId, Nil) + new SectionCodec(cases + (c.tableId -> newCases), verifyCrc) + } + + def disableCrcVerification: SectionCodec = new SectionCodec(cases, false) + + def sizeBound = SizeBound.unknown + + def encode(section: Section) = { + def tryEncode(c: SectionCodec.Case[Any, Section]) = { + val (privateBits, extension, data) = c.fromSection(section) + val preHeader = SectionHeader(section.tableId, extension.isDefined, privateBits, 0) + for { + encData <- extension match { + case None => c.codec(preHeader, verifyCrc).encode(data) + case Some(ext) => + for { + encExt <- Codec[SectionExtension].encode(ext) + encData <- c.codec(preHeader, verifyCrc).encode(data) + } yield encExt ++ encData + } + includeCrc = extension.isDefined + size = (encData.size / 8).toInt + (if (includeCrc) 4 else 0) + postHeader = preHeader.copy(length = size) + encHeader <- Codec[SectionHeader].encode(postHeader) + withoutCrc = encHeader ++ encData + } yield if (includeCrc) withoutCrc ++ crc32mpeg(withoutCrc) else withoutCrc + } + + for { + cs <- Attempt.fromOption(cases.get(section.tableId), Err(s"unsupported table id ${section.tableId}")) + enc <- cs.dropRight(1).foldRight(tryEncode(cs.last)) { (next, res) => res orElse tryEncode(next) } + } yield enc + } + + def decode(bits: BitVector) = (for { + header <- Codec[SectionHeader] + section <- Decoder(decodeSection(header)) + } yield section).decode(bits) + + def decodeSection(header: SectionHeader)(bits: BitVector): Attempt[DecodeResult[Section]] = + decoder(header).decode(bits) + + def decoder(header: SectionHeader): Decoder[Section] = { + decoder(header, Codec.encode(header).require) + } + + def decoder(header: SectionHeader, headerBits: BitVector): Decoder[Section] = Decoder { bits => + + def ensureCrcMatches(actual: Int, expected: Int) = + if (actual == expected) { Attempt.successful(()) } + else Attempt.failure(Err(s"CRC mismatch: calculated $expected does not equal $actual")) + + def generateCrc: Int = + crc32mpeg(headerBits ++ bits.take((header.length.toLong - 4) * 8)).toInt() + + def decodeExtended(c: SectionCodec.Case[Any, Section]): Decoder[(Option[SectionExtension], Any)] = for { + ext <- Codec[SectionExtension] + data <- fixedSizeBytes(header.length.toLong - 9, c.codec(header, verifyCrc)) + actualCrc <- int32 + expectedCrc = if (verifyCrc) generateCrc else actualCrc + _ <- Decoder.liftAttempt(ensureCrcMatches(actualCrc, expectedCrc)) + } yield Some(ext) -> data + + def decodeStandard(c: SectionCodec.Case[Any, Section]): Decoder[(Option[SectionExtension], Any)] = for { + data <- fixedSizeBytes(header.length.toLong, c.codec(header, verifyCrc)) + } yield None -> data + + def attemptDecode(c: SectionCodec.Case[Any, Section]): Attempt[DecodeResult[Section]] = for { + result <- ( if (header.extendedSyntax) decodeExtended(c) else decodeStandard(c) ).decode(bits) + DecodeResult((ext, data), remainder) = result + section <- c.toSection(header.privateBits, ext, data) + } yield DecodeResult(section, remainder) + + val cs = cases.getOrElse(header.tableId, List(unknownSectionCase(header.tableId).asInstanceOf[Case[Any, Section]])) + cs.foldRight(attemptDecode(cs.head)) { (next, res) => res orElse attemptDecode(next) } + } +} + +object SectionCodec { + + def empty: SectionCodec = new SectionCodec(Map.empty) + + def supporting[S <: Section : SectionFragmentCodec]: SectionCodec = + empty.supporting[S] + + def psi: SectionCodec = + supporting[ProgramAssociationSection]. + supporting[ProgramMapSection]. + supporting[ConditionalAccessSection] + + sealed trait UnknownSection extends Section + case class UnknownNonExtendedSection(tableId: Int, privateBits: BitVector, data: ByteVector) extends UnknownSection + case class UnknownExtendedSection(tableId: Int, privateBits: BitVector, extension: SectionExtension, data: ByteVector) extends UnknownSection with ExtendedSection + + private case class Case[A, B <: Section]( + codec: (SectionHeader, Boolean) => Codec[A], + toSection: (BitVector, Option[SectionExtension], A) => Attempt[B], + fromSection: B => (BitVector, Option[SectionExtension], A)) + + private object Case { + def fromSectionFragmentCodec[A <: Section](c: SectionFragmentCodec[A]): Case[Any, Section] = { + Case[Any, Section]( + (hdr, verifyCrc) => c.subCodec(hdr, verifyCrc).asInstanceOf[Codec[Any]], + (privateBits, extension, data) => c.toSection(privateBits, extension, data.asInstanceOf[c.Repr]), + section => c.fromSection(section.asInstanceOf[A]) + ) + } + } + + private def unknownSectionCase(tableId: Int): Case[BitVector, UnknownSection] = Case( + (hdr, verifyCrc) => bits, + (privateBits, ext, bits) => Attempt.successful(ext match { + case Some(e) => UnknownExtendedSection(tableId, privateBits, e, bits.bytes) + case None => UnknownNonExtendedSection(tableId, privateBits, bits.bytes) + }), + section => section match { + case u: UnknownExtendedSection => (u.privateBits, Some(u.extension), u.data.bits) + case u: UnknownNonExtendedSection => (u.privateBits, None, u.data.bits) + }) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala new file mode 100644 index 0000000000..5cd1b1aa54 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala @@ -0,0 +1,101 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.bits._ +import scodec.{ Attempt, Codec, Err } + +trait SectionFragmentCodec[A] { + type Repr + def tableId: Int + def subCodec(header: SectionHeader, verifyCrc: Boolean): Codec[Repr] + def toSection(privateBits: BitVector, extension: Option[SectionExtension], data: Repr): Attempt[A] + def fromSection(section: A): (BitVector, Option[SectionExtension], Repr) +} + +object SectionFragmentCodec { + private val PsiPrivateBits = bin"011" + + def psi[A, R: Codec](tableId: Int, toSection: (SectionExtension, R) => A, fromSection: A => (SectionExtension, R)): SectionFragmentCodec[A] = + extended[A, R](tableId, (_, ext, r) => toSection(ext, r), s => { val (ext, r) = fromSection(s); (PsiPrivateBits, ext, r) }) + + def extended[A, R: Codec](tableId: Int, toSection: (BitVector, SectionExtension, R) => A, fromSection: A => (BitVector, SectionExtension, R)): SectionFragmentCodec[A] = { + val tid = tableId + val build = toSection + val extract = fromSection + new SectionFragmentCodec[A] { + type Repr = R + def tableId = tid + def subCodec(header: SectionHeader, verifyCrc: Boolean) = Codec[Repr] + def toSection(privateBits: BitVector, extension: Option[SectionExtension], data: Repr) = + Attempt.fromOption(extension.map { ext => build(privateBits, ext, data) }, Err("extended section missing expected section extension")) + def fromSection(section: A) = + extract(section) match { case (privateBits, ext, data) => (privateBits, Some(ext), data) } + } + } + + def nonExtended[A, R](tableId: Int, toCodec: SectionHeader => Codec[R], toSection: (BitVector, R) => A, fromSection: A => (BitVector, R)): SectionFragmentCodec[A] = { + val tid = tableId + val codec = toCodec + val build = toSection + val extract = fromSection + new SectionFragmentCodec[A] { + type Repr = R + def tableId = tid + def subCodec(header: SectionHeader, verifyCrc: Boolean) = codec(header) + def toSection(privateBits: BitVector, extension: Option[SectionExtension], data: Repr) = + Attempt.successful(build(privateBits, data)) + def fromSection(section: A) = { + val (privateBits, r) = extract(section) + (privateBits, None, r) + } + } + } + + def nonExtendedWithCrc[A, R](tableId: Int, toCodec: (SectionHeader, Boolean) => Codec[R], toSection: (BitVector, R) => A, fromSection: A => (BitVector, R)): SectionFragmentCodec[A] = { + val tid = tableId + val codec = toCodec + val build = toSection + val extract = fromSection + new SectionFragmentCodec[A] { + type Repr = R + def tableId = tid + def subCodec(header: SectionHeader, verifyCrc: Boolean) = codec(header, verifyCrc) + def toSection(privateBits: BitVector, extension: Option[SectionExtension], data: Repr) = + Attempt.successful(build(privateBits, data)) + def fromSection(section: A) = { + val (privateBits, r) = extract(section) + (privateBits, None, r) + } + } + } + + def nonExtendedIdentity[A](tableId: Int, toCodec: SectionHeader => Codec[A]): SectionFragmentCodec[A] = + SectionFragmentCodec.nonExtended[A, A](tableId, sHdr => toCodec(sHdr), (bits, a) => a, a => (BitVector.empty, a)) + + def nonExtendedIdentityWithCrc[A](tableId: Int, toCodec: (SectionHeader, Boolean) => Codec[A]): SectionFragmentCodec[A] = + SectionFragmentCodec.nonExtendedWithCrc[A, A](tableId, (sHdr, verifyCrc) => toCodec(sHdr, verifyCrc), (bits, a) => a, a => (BitVector.empty, a)) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala new file mode 100644 index 0000000000..57c9c40d1f --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +import scodec.Codec +import scodec.bits.BitVector +import scodec.codecs._ + +case class SectionHeader( + tableId: Int, + extendedSyntax: Boolean, + privateBits: BitVector, + length: Int) + +object SectionHeader { + + implicit val codec: Codec[SectionHeader] = { + ("table_id" | uint8 ) :: + ("section_syntax_indicator" | bool ) :: + ("private_bits" | bits(3) ) :: + ("length" | uint(12) ) + }.as[SectionHeader] +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala new file mode 100644 index 0000000000..8f4adafa70 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala @@ -0,0 +1,37 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols.mpeg +package transport +package psi + +/** + * Indicates the implementor can be treated as a message delivered in an MPEG transport stream. + * + * This library differentiates tables from sections. Sections are the actual messages delivered + * in the transport stream whereas tables are the result of grouping multiple related sections + * together in to a single logical message. + */ +trait Table { + def tableId: Int +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala new file mode 100644 index 0000000000..6745d20b83 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala @@ -0,0 +1,82 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package mpeg +package transport +package psi + +case class TableBuildingError(tableId: Int, message: String) extends MpegError + +class TableBuilder private (cases: Map[Int, List[TableSupport[_]]]) { + + def supporting[T <: Table](implicit ts: TableSupport[T]): TableBuilder = { + val newCases = ts :: cases.getOrElse(ts.tableId, Nil) + new TableBuilder(cases + (ts.tableId -> newCases)) + } + + def build(gs: GroupedSections[Section]): Either[TableBuildingError, Table] = { + cases.get(gs.tableId) match { + case None | Some(Nil) => Left(TableBuildingError(gs.tableId, "Unknown table id")) + case Some(list) => + list.dropRight(1).foldRight[Either[String, _]](list.last.toTable(gs)) { (next, res) => res.fold(_ => next.toTable(gs), Right(_)) } match { + case Right(table) => Right(table.asInstanceOf[Table]) + case Left(err) => Left(TableBuildingError(gs.tableId, err)) + } + } + } +} + +object TableBuilder { + + def empty: TableBuilder = new TableBuilder(Map.empty) + + def supporting[T <: Table : TableSupport] = empty.supporting[T] + + def psi: TableBuilder = + supporting[ProgramAssociationTable]. + supporting[ProgramMapTable]. + supporting[ConditionalAccessTable] +} + +trait TableSupport[T <: Table] { + def tableId: Int + def toTable(gs: GroupedSections[Section]): Either[String, T] + def toSections(t: T): GroupedSections[Section] +} + +object TableSupport { + + def singleton[A <: Section with Table : reflect.ClassTag](tableId: Int): TableSupport[A] = { + val tid = tableId + new TableSupport[A] { + def tableId = tid + def toTable(gs: GroupedSections[Section]) = + gs.narrow[A].toRight(s"Not a ${reflect.ClassTag[A]}").flatMap { sections => + if (sections.tail.isEmpty) Right(sections.head) + else Left(s"${reflect.ClassTag[A]} supports only 1 section but got ${sections.list.size}") + } + def toSections(table: A) = GroupedSections(table) + } + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala new file mode 100644 index 0000000000..8f1dee002f --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala @@ -0,0 +1,117 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport +package psi + +import cats.data.Chain +import scodec.bits.BitVector + +import psi.{ Table => TableMessage } + +abstract class TransportStreamEvent + +object TransportStreamEvent { + case class Pes(pid: Pid, pes: PesPacket) extends TransportStreamEvent + case class Table(pid: Pid, table: TableMessage) extends TransportStreamEvent + case class ScrambledPayload(pid: Pid, payload: BitVector) extends TransportStreamEvent + case class Metadata[A](pid: Option[Pid], metadata: A) extends TransportStreamEvent + case class Error(pid: Option[Pid], err: MpegError) extends TransportStreamEvent + + def pes(pid: Pid, pes: PesPacket): TransportStreamEvent = Pes(pid, pes) + def table(pid: Pid, table: TableMessage): TransportStreamEvent = Table(pid, table) + def scrambledPayload(pid: Pid, content: BitVector): TransportStreamEvent = ScrambledPayload(pid, content) + def metadata[A](md: A): TransportStreamEvent = Metadata(None, md) + def metadata[A](pid: Pid, md: A): TransportStreamEvent = Metadata(Some(pid), md) + def error(pid: Pid, e: MpegError): TransportStreamEvent = Error(Some(pid), e) + def error(pid: Option[Pid], e: MpegError): TransportStreamEvent = Error(pid, e) + + private def sectionsToTables[S]( + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], tableBuilder: TableBuilder + ): Scan[(Map[Pid, S], TransportStreamIndex), PidStamped[Either[MpegError, Section]], TransportStreamEvent] = { + + import MpegError._ + + val sectionsToTablesForPid: Scan[S, Section, Either[MpegError, TableMessage]] = + group.map { + case Left(e) => Left(e) + case Right(gs) => tableBuilder.build(gs) + } + + val sectionsToTables: Scan[Map[Pid, S], PidStamped[Either[MpegError, Section]], PidStamped[Either[MpegError, TableMessage]]] = + Scan(Map.empty[Pid, S])({ + case (state, PidStamped(pid, e)) => + e match { + case Right(section) => + val groupingState = state.getOrElse(pid, group.initial) + val (s, out) = sectionsToTablesForPid.transform(groupingState, section) + (state.updated(pid, s), out.map(PidStamped(pid, _))) + case Left(err) => + (state, Chunk.singleton(PidStamped(pid, Left(err)))) + } + }, { state => + Chunk.concat(state.foldLeft(Chain.empty[Chunk[PidStamped[Either[MpegError, TableMessage]]]]) { case (acc, (pid, gs)) => + acc :+ sectionsToTablesForPid.onComplete(gs).map(PidStamped(pid, _)) + }.toList) + }) + + sectionsToTables.andThen(PidStamped.preserve(passErrors(TransportStreamIndex.build))).map { case PidStamped(pid, value) => + value match { + case Left(e) => error(pid, e) + case Right(Left(tsi)) => metadata(tsi) + case Right(Right(tbl)) => table(pid, tbl) + } + } + } + + def fromPacketStream[S]( + sectionCodec: SectionCodec, + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], + tableBuilder: TableBuilder + ): Scan[((Map[Pid, ContinuityCounter], Demultiplexer.State), (Map[Pid, S], TransportStreamIndex)), Packet, TransportStreamEvent] = { + val demuxed = { + Demultiplexer.demultiplex(sectionCodec).andThen( + sectionsToTables(group, tableBuilder).semipass[PidStamped[Either[DemultiplexerError, Demultiplexer.Result]], TransportStreamEvent]( + { + case PidStamped(pid, Right(Demultiplexer.SectionResult(section))) => Right(PidStamped(pid, Right(section))) + case PidStamped(pid, Right(Demultiplexer.PesPacketResult(p))) => Left(pes(pid, p)) + case PidStamped(pid, Left(e)) => Right(PidStamped(pid, Left(e.toMpegError))) + })) + } + demuxed.semipass[Packet, TransportStreamEvent]({ + case Packet(header, _, _, Some(payload)) if header.scramblingControl != 0 => + Left(scrambledPayload(header.pid, payload)) + case p @ Packet(_, _, _, _) => + Right(p) + }) + } + + def fromSectionStream[S]( + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], + tableBuilder: TableBuilder + ): Scan[(Map[Pid, S], TransportStreamIndex), PidStamped[Section], TransportStreamEvent] = + sectionsToTables(group, tableBuilder).contramap[PidStamped[Section]](_.map(Right(_))) +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala new file mode 100644 index 0000000000..e1897bffa1 --- /dev/null +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala @@ -0,0 +1,108 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2.protocols +package mpeg +package transport +package psi + +import fs2._ + +sealed abstract class TransportStreamIndex { + import TransportStreamIndex._ + + def pat: Option[ProgramAssociationTable] + def cat: Option[ConditionalAccessTable] + + def pmt(prg: ProgramNumber): Either[LookupError, ProgramMapTable] + + def programMapRecords(program: ProgramNumber, streamType: StreamType): Either[LookupError, List[ProgramMapRecord]] = + for { + p <- pat.toRight(LookupError.MissingProgramAssociation) + _ <- p.programByPid.get(program).toRight(LookupError.UnknownProgram) + q <- pmt(program) + pmrs <- q.componentStreamMapping.get(streamType).toRight(LookupError.UnknownStreamType) + } yield pmrs + + def programManRecord(program: ProgramNumber, streamType: StreamType): Either[LookupError, ProgramMapRecord] = + programMapRecords(program, streamType).map { _.head } + + def withPat(pat: ProgramAssociationTable): TransportStreamIndex + def withPmt(pmt: ProgramMapTable): TransportStreamIndex + def withCat(cat: ConditionalAccessTable): TransportStreamIndex +} + + +object TransportStreamIndex { + + sealed abstract class LookupError + object LookupError { + case object UnknownProgram extends LookupError + case object UnknownStreamType extends LookupError + case object MissingProgramAssociation extends LookupError + case object MissingProgramMap extends LookupError + } + + private case class DefaultTransportStreamIndex( + pat: Option[ProgramAssociationTable], + cat: Option[ConditionalAccessTable], + pmts: Map[ProgramNumber, ProgramMapTable] + ) extends TransportStreamIndex { + + def pmt(prg: ProgramNumber): Either[LookupError, ProgramMapTable] = + pmts.get(prg).toRight(LookupError.UnknownProgram) + + def withPat(pat: ProgramAssociationTable): TransportStreamIndex = { + val programs = pat.programByPid.keys.toSet + copy(pat = Some(pat), pmts = pmts.view.filterKeys(programs).toMap) + } + + def withPmt(pmt: ProgramMapTable): TransportStreamIndex = { + copy(pmts = pmts + (pmt.programNumber -> pmt)) + } + + def withCat(cat: ConditionalAccessTable): TransportStreamIndex = + copy(cat = Some(cat)) + } + + def empty: TransportStreamIndex = DefaultTransportStreamIndex(None, None, Map.empty) + + def build: Scan[TransportStreamIndex, Table, Either[TransportStreamIndex, Table]] = Scan.stateful(empty) { (tsi, section) => + val updatedTsi = section match { + case pat: ProgramAssociationTable => + Some(tsi.withPat(pat)) + case pmt: ProgramMapTable => + Some(tsi.withPmt(pmt)) + case cat: ConditionalAccessTable => + Some(tsi.withCat(cat)) + case other => None + } + val out = updatedTsi match { + case Some(newTsi) if newTsi != tsi => + Chunk(Right(section), Left(newTsi)) + case _ => + Chunk(Right(section)) + } + (updatedTsi.getOrElse(tsi), out) + } +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala index 97ff3f8c00..632771037b 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/CaptureFile.scala @@ -37,7 +37,7 @@ object CaptureFile { implicit val codec: Codec[CaptureFile] = "capture-file" | Codec[GlobalHeader] .flatPrepend { hdr => - vector(Record.codec(hdr.ordering)).hlist + vector(Record.codec(hdr.ordering)).tuple } .as[CaptureFile] diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala index fd3567baee..ad93b0d438 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/GlobalHeader.scala @@ -45,8 +45,10 @@ object GlobalHeader { private val byteOrdering: Codec[ByteOrdering] = new Codec[ByteOrdering] { def sizeBound = SizeBound.exact(32) - def encode(bo: ByteOrdering) = - endiannessDependent(uint32, uint32L)(bo).encode(MagicNumber) + def encode(bo: ByteOrdering) = { + implicit val boImplicit: ByteOrdering = bo + endiannessDependent(uint32, uint32L).encode(MagicNumber) + } def decode(buf: BitVector) = uint32.decode(buf).flatMap { diff --git a/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala b/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala index 9d81d2db45..f283874c16 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/pcap/Record.scala @@ -33,7 +33,7 @@ object Record { // format: off implicit def codec(implicit ordering: ByteOrdering): Codec[Record] = "record" | { ("record_header" | RecordHeader.codec ).flatPrepend { hdr => - ("record_data" | bits(hdr.includedLength.toInt * 8L) ).hlist + ("record_data" | bits(hdr.includedLength.toInt * 8L) ).tuple }}.as[Record] // format: on } diff --git a/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala b/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala new file mode 100644 index 0000000000..eedc7d1526 --- /dev/null +++ b/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala @@ -0,0 +1,69 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols + +import cats.effect.{IO, IOApp} +import com.comcast.ip4s.{Ipv4Address, Port, SocketAddress} +import fs2.interop.scodec.StreamDecoder +import fs2.io.file.{Files, Path} +import fs2.timeseries.TimeStamped + +import pcap.{ CaptureFile, LinkType } + +/** + * Example of decoding a PCAP file that contains: + * - captured ethernet frames + * - of IPv4 packets + * - of UDP datagrams + * - containing MPEG transport stream packets + */ +object PcapMpegExample extends IOApp.Simple { + + case class CapturedPacket(source: SocketAddress[Ipv4Address], destination: SocketAddress[Ipv4Address], packet: mpeg.transport.Packet) + + val run: IO[Unit] = { + val decoder: StreamDecoder[TimeStamped[CapturedPacket]] = CaptureFile.payloadStreamDecoderPF { + case LinkType.Ethernet => + for { + ethernetHeader <- ethernet.EthernetFrameHeader.sdecoder + ipHeader <- ip.Ipv4Header.sdecoder(ethernetHeader) + udpDatagram <- ip.udp.DatagramHeader.sdecoder(ipHeader.protocol) + packets <- StreamDecoder.tryMany(mpeg.transport.Packet.codec).map { p => + CapturedPacket( + SocketAddress(ipHeader.sourceIp, udpDatagram.sourcePort), + SocketAddress(ipHeader.destinationIp, udpDatagram.destinationPort), + p) + } + } yield packets + } + + Files[IO].readAll(Path("path/to/pcap")) + .through(decoder.toPipeByte) + .map(_.toString) + .foreach(IO.println) + .compile + .drain + } +} diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala new file mode 100644 index 0000000000..a5049f945a --- /dev/null +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala @@ -0,0 +1,268 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport +package psi + +import Descriptor._ + +import scodec._ +import scodec.bits._ + +import org.scalacheck.{ Arbitrary, Gen, Prop } + +class DescriptorTest extends Fs2Suite { + import DescriptorTestData._ + + override def scalaCheckTestParameters = + super.scalaCheckTestParameters + .withMinSuccessfulTests(1000) + + test("support relevant descriptors which handles decoding and encoding for valid values") { + Prop.forAll { (d: Descriptor) => roundtrip(Descriptor.codec, d) } + } + + private def roundtrip[A](codec: Codec[A], value: A) = { + val encoded = codec.encode(value) + val Attempt.Successful(DecodeResult(decoded, remainder)) = codec.decode(encoded.require) + assertEquals(remainder, BitVector.empty) + assertEquals(decoded, value) + } +} + +object DescriptorTestData { + + def genMpeg1Only(flag: Boolean): Gen[Option[Mpeg1Only]] = + if (!flag) None + else + for { + profileAndLevelIndication <- Gen.chooseNum(0, 255) + chromaFormat <- Gen.chooseNum(0, 3) + frameRateExtensionFlag <- Gen.oneOf(true, false) + } yield Some(Mpeg1Only(profileAndLevelIndication, chromaFormat, frameRateExtensionFlag)) + + lazy val genVideoStreamDescriptor: Gen[VideoStreamDescriptor] = for { + multipleFrameRateFlag <- Gen.oneOf(true, false) + frameRateCode <- Gen.chooseNum(0, 15) + mpeg1OnlyFlag <- Gen.oneOf(true, false) + constrainedParameter <- Gen.oneOf(true, false) + stillPictureFlag <- Gen.oneOf(true, false) + mpeg1Only <- genMpeg1Only(mpeg1OnlyFlag) + } yield VideoStreamDescriptor(multipleFrameRateFlag, frameRateCode, mpeg1OnlyFlag, constrainedParameter, stillPictureFlag, mpeg1Only) + + lazy val genAudioStreamDescriptor: Gen[AudioStreamDescriptor] = for { + freeFormatFlag <- Gen.oneOf(true, false) + id <- Gen.oneOf(true, false) + layer <- Gen.chooseNum(0, 3) + variableRateAudioIndicator <- Gen.oneOf(true, false) + } yield AudioStreamDescriptor(freeFormatFlag, id, layer, variableRateAudioIndicator) + + lazy val genHierarchyType: Gen[HierarchyType] = Gen.oneOf( + HierarchyType.SpatialScalability, + HierarchyType.SnrScalability, + HierarchyType.TemporalScalability, + HierarchyType.DataPartitioning, + HierarchyType.ExtensionBitstream, + HierarchyType.PrivateStream, + HierarchyType.MultiViewProfile, + HierarchyType.Reserved(0), + HierarchyType.BaseLayer) + + lazy val genHierarchyDescriptor: Gen[HierarchyDescriptor] = for { + hierarchyType <- genHierarchyType + hierarchyLayerIndex <- Gen.chooseNum(0, 63) + hierarchyEmbeddedLayerIndex <- Gen.chooseNum(0, 63) + hierarchyChannel <- Gen.chooseNum(0, 63) + } yield HierarchyDescriptor(hierarchyType, hierarchyLayerIndex, hierarchyEmbeddedLayerIndex, hierarchyChannel) + + lazy val genRegistrationDescriptor: Gen[RegistrationDescriptor] = for { + length <- Gen.chooseNum(4, 255) + formatIdentifier <- Gen.listOfN(4, Gen.chooseNum(0, 255)) + additionalIdentificationInfo <- Gen.listOfN(length - 4, Gen.chooseNum(0, 255)) + } yield RegistrationDescriptor(ByteVector(formatIdentifier: _*), ByteVector(additionalIdentificationInfo: _*)) + + lazy val genDataStreamAlignmentDescriptor: Gen[DataStreamAlignmentDescriptor] = for { + alignmentType <- Gen.oneOf(AlignmentType.Reserved(0), + AlignmentType.SliceOrVideoAccessUnit, + AlignmentType.VideoAccessUnit, + AlignmentType.GopOrSeq, + AlignmentType.Seq) + } yield DataStreamAlignmentDescriptor(alignmentType) + + lazy val genTargetBackgroundGridDescriptor: Gen[TargetBackgroundGridDescriptor] = for { + horizontalSize <- Gen.chooseNum(0, 16383) + verticalSize <- Gen.chooseNum(0, 16383) + aspectRatioInformation <- Gen.choose(0, 15) + } yield TargetBackgroundGridDescriptor(horizontalSize, verticalSize, aspectRatioInformation) + + lazy val genVideoWindowDescriptor: Gen[VideoWindowDescriptor] = for { + horizontalOffset <- Gen.chooseNum(0, 16383) + verticalOffset <- Gen.chooseNum(0, 16383) + windowPriority <- Gen.choose(0, 15) + } yield VideoWindowDescriptor(horizontalOffset, verticalOffset, windowPriority) + + lazy val genCADescriptor: Gen[CADescriptor] = for { + length <- Gen.chooseNum(4, 255) + caSystemId <- Gen.chooseNum(0, 65535) + caPid <- Gen.choose(0, 8191) + privateData <- Gen.listOfN(length - 4, Gen.chooseNum(0, 255)) + } yield CADescriptor(caSystemId, Pid(caPid), ByteVector(privateData: _*)) + + lazy val genLanguageField: Gen[LanguageField] = for { + iso639LanguageCode <- Gen.listOfN(3, Gen.alphaChar) + audioType <- Gen.oneOf(AudioType.Undefined, AudioType.CleanEffects, AudioType.HearingImpaired, AudioType.VisualImpairedCommentary, AudioType.Reserved(4)) + } yield LanguageField(iso639LanguageCode.mkString, audioType) + + lazy val genIso639LanguageDescriptor: Gen[Iso639LanguageDescriptor] = for { + numberOfLanguagueField <- Gen.chooseNum(0, 63) + languageFields <- Gen.listOfN(numberOfLanguagueField, genLanguageField) + length = languageFields.size * 4 + } yield Iso639LanguageDescriptor(languageFields.toVector) + + lazy val genSystemClockDescriptor: Gen[SystemClockDescriptor] = for { + externalClockReferenceIndicator <- Gen.oneOf(true, false) + clockAccuracyInteger <- Gen.oneOf(0, 63) + clockAccuracyExponent <- Gen.oneOf(0, 7) + } yield SystemClockDescriptor(externalClockReferenceIndicator, clockAccuracyInteger, clockAccuracyExponent) + + lazy val genMultiplexBufferUtilizationDescriptor: Gen[MultiplexBufferUtilizationDescriptor] = for { + boundValidFlag <- Gen.oneOf(true, false) + ltwOffsetLowerBound <- Gen.oneOf(0, 32767) + ltwOffsetUpperBound <- Gen.oneOf(0, 16383) + } yield MultiplexBufferUtilizationDescriptor(boundValidFlag, ltwOffsetLowerBound, ltwOffsetUpperBound) + + lazy val genCopyrightDescriptor: Gen[CopyrightDescriptor] = for { + length <- Gen.chooseNum(4, 255) + copyrightIdentifier <- Gen.listOfN(4, Gen.chooseNum(0, 255)) + additionalCopyrightInfo <- Gen.listOfN(length - 4, Gen.chooseNum(0, 255)) + } yield CopyrightDescriptor(ByteVector(copyrightIdentifier: _*), ByteVector(additionalCopyrightInfo: _*)) + + lazy val genMaximumBitrateDescriptor: Gen[MaximumBitrateDescriptor] = for { + maximumBitrate <- Gen.chooseNum(0, 4194303) + } yield MaximumBitrateDescriptor(maximumBitrate) + + lazy val genPrivateDataIndicatorDescriptor: Gen[PrivateDataIndicatorDescriptor] = for { + privateDataIndicator <- Gen.listOfN(4, Gen.chooseNum(0, 255)) + } yield PrivateDataIndicatorDescriptor(ByteVector(privateDataIndicator: _*)) + + lazy val genSmoothingBufferDescriptor: Gen[SmoothingBufferDescriptor] = for { + sbLeakRate <- Gen.chooseNum(0, 4194303) + sbSize <- Gen.chooseNum(0, 4194303) + } yield SmoothingBufferDescriptor(sbLeakRate, sbSize) + + lazy val genStdDescriptor: Gen[StdDescriptor] = + for { leakValidFlag <- Gen.oneOf(true, false) } yield StdDescriptor(leakValidFlag) + + lazy val genIbpDescriptor: Gen[IbpDescriptor] = for { + closedGopFlag <- Gen.oneOf(true, false) + identicalGopFlag <- Gen.oneOf(true, false) + maxGopLength <- Gen.chooseNum(0, 16383) + } yield IbpDescriptor(closedGopFlag, identicalGopFlag, maxGopLength) + + lazy val genMpeg4VideoDescriptor: Gen[Mpeg4VideoDescriptor] = + for { mpeg4VisualProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4VideoDescriptor(mpeg4VisualProfileAndLevel.toByte) + + lazy val genMpeg4AudioDescriptor: Gen[Mpeg4AudioDescriptor] = + for { mpeg4AudioProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4AudioDescriptor(mpeg4AudioProfileAndLevel.toByte) + + lazy val genIodDescriptor: Gen[IodDescriptor] = for { + scopeOfIodLabel <- Gen.chooseNum(0, 255) + iodLabel <- Gen.chooseNum(0, 255) + initialObjectDescriptor <- Gen.chooseNum(0, 255) + } yield IodDescriptor(scopeOfIodLabel.toByte, iodLabel.toByte, initialObjectDescriptor.toByte) + + lazy val genSlDescriptor: Gen[SlDescriptor] = + for { esId <- Gen.chooseNum(0, 65535) } yield SlDescriptor(esId: Int) + + lazy val genEsIdAndChannel: Gen[EsIdAndChannel] = for { + esId <- Gen.chooseNum(0, 65535) + flexMuxChannel <- Gen.chooseNum(0, 255) + } yield EsIdAndChannel(esId, flexMuxChannel) + + lazy val genFmcDescriptor: Gen[FmcDescriptor] = for { + numberOf <- Gen.chooseNum(0, 85) + channels <- Gen.listOfN(numberOf, genEsIdAndChannel) + } yield FmcDescriptor(channels.toVector) + + lazy val genExternalEsIdDescriptor: Gen[ExternalEsIdDescriptor] = + for { externalEsId <- Gen.chooseNum(0, 65535) } yield ExternalEsIdDescriptor(externalEsId) + + lazy val genMuxCodeDescriptor: Gen[MuxCodeDescriptor] = for { + length <- Gen.chooseNum(0, 255) + muxCodeTableEntry <- Gen.listOfN(length, Gen.chooseNum(0, 255)) + } yield MuxCodeDescriptor(ByteVector(muxCodeTableEntry: _*)) + + lazy val genFmxBufferSizeDescriptor: Gen[FmxBufferSizeDescriptor] = for { + length <- Gen.chooseNum(0, 255) + flexMuxBufferDescriptor <- Gen.listOfN(length, Gen.chooseNum(0, 255)) + } yield FmxBufferSizeDescriptor(ByteVector(flexMuxBufferDescriptor: _*)) + + lazy val genMultiplexBufferDescriptor: Gen[MultiplexBufferDescriptor] = for { + mbBufferSize <- Gen.chooseNum(0, 16777215) + tbLeakRate <- Gen.chooseNum(0, 16777215) + } yield MultiplexBufferDescriptor(mbBufferSize, tbLeakRate) + + lazy val genKnownDescriptor: Gen[KnownDescriptor] = Gen.oneOf( + genVideoStreamDescriptor, + genAudioStreamDescriptor, + genHierarchyDescriptor, + genRegistrationDescriptor, + genDataStreamAlignmentDescriptor, + genTargetBackgroundGridDescriptor, + genVideoWindowDescriptor, + genCADescriptor, + genIso639LanguageDescriptor, + genSystemClockDescriptor, + genMultiplexBufferUtilizationDescriptor, + genCopyrightDescriptor, + genMaximumBitrateDescriptor, + genPrivateDataIndicatorDescriptor, + genSmoothingBufferDescriptor, + genStdDescriptor, + genIbpDescriptor, + genMpeg4VideoDescriptor, + genMpeg4AudioDescriptor, + genIodDescriptor, + genSlDescriptor, + genFmcDescriptor, + genExternalEsIdDescriptor, + genMuxCodeDescriptor, + genFmxBufferSizeDescriptor, + genMultiplexBufferDescriptor) + + lazy val genUnknownDescriptor: Gen[UnknownDescriptor] = for { + tag <- Gen.chooseNum(36, 255) + length <- Gen.chooseNum(0, 255) + data <- Gen.listOfN(length, Gen.chooseNum(0, 255)) + } yield UnknownDescriptor(tag, length, ByteVector(data: _*)) + + lazy val genDescriptor: Gen[Descriptor] = Gen.oneOf(genKnownDescriptor, genUnknownDescriptor).map { + case known: KnownDescriptor => Right(known) + case unknown: UnknownDescriptor => Left(unknown) + } + + implicit lazy val arbitraryDescriptor: Arbitrary[Descriptor] = Arbitrary(genDescriptor) +} diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala new file mode 100644 index 0000000000..09b513f714 --- /dev/null +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport +package psi + +import scodec.bits._ + +class PacketTest extends Fs2Suite { + + test("support packetizing multiple sections in to a single packet") { + val a = ByteVector.fill(10)(0).bits + val b = ByteVector.fill(10)(1).bits + val c = ByteVector.fill(10)(2).bits + val sections = Vector(a, b, c) + val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), sections) + assertEquals(packets, Vector(Packet.payload(Pid(0), ContinuityCounter(0), Some(0), a ++ b ++ c ++ BitVector.fill((183 * 8) - a.size - b.size - c.size)(true)))) + } + + test("support packetizing multiple sections across multiple packets") { + val sections = (0 until 256).map { x => ByteVector.fill(10)(x).bits }.toVector + val data = sections.foldLeft(BitVector.empty)(_ ++ _) + val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), sections) + + packets.zipWithIndex.foreach { case (packet, idx) => + val payloadOffset = if (idx == 0) 0 else 10 * ((idx * 183) / 10 + 1) - (idx * 183) + val offset = 183L * 8 * idx + assertEquals(packets(idx), Packet.payload(Pid(0), ContinuityCounter(idx), Some(payloadOffset), data.drop(offset).take(183 * 8))) + } + } +} diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala new file mode 100644 index 0000000000..e376157c8c --- /dev/null +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala @@ -0,0 +1,55 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport +package psi + +class GroupingTest extends Fs2Suite { + + val des = GroupedSections.groupExtendedSections[ProgramAssociationSection].toPipe[Pure] + + val pat3: ProgramAssociationTable = { + val pidMap = (0 until ProgramAssociationTable.MaxProgramsPerSection * 3).map { n => ProgramNumber(n) -> Pid(n) }.toMap + ProgramAssociationTable(TransportStreamId(5), 1, true, pidMap) + } + + test("handles stream of a specific table id and extension") { + val p = Stream.emits(pat3.toSections.list).through(des).map { + case Right(sections) => ProgramAssociationTable.fromSections(sections) + case l @ Left(_) => l + } + assertEquals(p.toList, List(Right(pat3))) + } + + test("handles stream containing sections for the same table id but differing extension ids") { + val patA = pat3 + val patB = pat3.copy(tsid = TransportStreamId(pat3.tsid.value + 1), programByPid = pat3.programByPid.map { case (prg, Pid(n)) => prg -> Pid(n + 1)} ) + + val sections = Stream.emits(patA.toSections.list) interleave Stream.emits(patB.toSections.list) + val p = sections.through(des).map { _.flatMap(ProgramAssociationTable.fromSections) } + assertEquals(p.toList, List(Right(patA), Right(patB))) + } +} diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala new file mode 100644 index 0000000000..d85140b94b --- /dev/null +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala @@ -0,0 +1,116 @@ +/* + * Copyright (c) 2013 Functional Streams for Scala + * + * Permission is hereby granted, free of charge, to any person obtaining a copy of + * this software and associated documentation files (the "Software"), to deal in + * the Software without restriction, including without limitation the rights to + * use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of + * the Software, and to permit persons to whom the Software is furnished to do so, + * subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS + * FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR + * COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER + * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + */ + +// Adapted from scodec-protocols, licensed under 3-clause BSD + +package fs2 +package protocols +package mpeg +package transport +package psi + +import scodec.Err +import scodec.bits._ +import scodec.codecs._ + +class SectionCodecTest extends Fs2Suite { + + group("support decoding a stream of packets in to a stream of sections") { + + val sectionCodec = SectionCodec.supporting[ProgramAssociationSection] + + test("handles case where section starts at beginning of packet and is fully contained within packet") { + val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + val pasEnc = sectionCodec.encode(pas).require + val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), pasEnc) + + val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe + assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + } + + test("handles case where section starts at beginning of packet and spans multiple packets") { + val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, + (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) + yield ProgramNumber(i) -> Pid(i)).toMap + )).head + val pasEnc = sectionCodec.encode(pas).require + val packets = Packet.packetize(Pid(0), ContinuityCounter(0), pasEnc) + + val p = Stream.emits(packets) through Demultiplexer.demultiplex(sectionCodec).toPipe + assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + } + + test("checks packet continuity") { + val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, + (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) + yield ProgramNumber(i) -> Pid(i)).toMap + )).head + val pasEnc = sectionCodec.encode(pas).require + val packets = Packet.packetize(Pid(0), ContinuityCounter(1), pasEnc) + val withDiscontinuity = packets.updated(0, packets.head.copy(header = packets.head.header.copy(continuityCounter = ContinuityCounter(15)))) + + val p = Stream.emits(withDiscontinuity) through Demultiplexer.demultiplex(sectionCodec).toPipe + assertEquals(p.toList, List(PidStamped(Pid(0), Left(DemultiplexerError.Discontinuity(ContinuityCounter(15), ContinuityCounter(2), 1))))) + } + + test("upon decoding failure of a section, remaining sections in packet are decoded") { + case class SmallSection(x: Int) extends Section { def tableId = 0 } + val sections = List(SmallSection(0), SmallSection(1)) + + implicit val sfc: SectionFragmentCodec[SmallSection] = + SectionFragmentCodec.nonExtended[SmallSection, Int](0, h => (constant(bin"0") ~> uint(7)), (p, i) => SmallSection(i), ss => (bin"010", ss.x)) + val sc = SectionCodec.supporting[SmallSection] + + val encodedSections = sections.toVector map { s => sc.encode(s).require } + val ss0 = encodedSections(0).bytes + val ss1 = encodedSections(1).bytes + val indexOfInt = ss0.toIndexedSeq.zipWithIndex.find { case (x, idx) => ss1(idx.toLong) != x }.map { case (x, idx) => idx }.get + val ss255 = ss0.update(indexOfInt.toLong, 255.toByte) + + val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), ss255.bits +: encodedSections) + val p = Stream.emits(packets) through Demultiplexer.demultiplex(sc).toPipe + + assertEquals(p.toList, + PidStamped(Pid(0), Left(DemultiplexerError.Decoding(hex"002001ff".bits, Err("expected constant BitVector(1 bits, 0x0) but got BitVector(1 bits, 0x8)")))) +: + sections.map { x => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(x))) } + ) + } + + test("reports invalid CRC") { + val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + val pasEnc = sectionCodec.encode(pas).require + val corruptedSection = pasEnc.dropRight(32) ++ (~pasEnc.takeRight(32)) + val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), corruptedSection) + val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe + assertEquals(p.toList, List(PidStamped(Pid(0), Left(DemultiplexerError.Decoding(corruptedSection, Err("CRC mismatch: calculated 18564404 does not equal -18564405")))))) + } + + test("does not report invalid CRC when verifyCrc is disabled") { + val sectionCodec = SectionCodec.psi.disableCrcVerification.supporting[ProgramAssociationSection] + val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + val pasEnc = sectionCodec.encode(pas).require + val corruptedSection = pasEnc.dropRight(32) ++ (~pasEnc.dropRight(32)) + val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), corruptedSection) + val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe + assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + } + } +} From 6ff52ff9968f28e9015dcc0cae41c072a98e821c Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 22 Oct 2021 09:28:33 -0400 Subject: [PATCH 27/33] Scalafmt --- .../scala/fs2/protocols/ip/Ipv6Header.scala | 32 +-- .../scala/fs2/protocols/mpeg/Descriptor.scala | 264 ++++++++++++------ .../scala/fs2/protocols/mpeg/MpegError.scala | 4 +- .../scala/fs2/protocols/mpeg/PesPacket.scala | 34 ++- .../fs2/protocols/mpeg/PesPacketHeader.scala | 199 +++++++------ .../mpeg/PesPacketHeaderPrefix.scala | 10 +- .../fs2/protocols/mpeg/PesStreamId.scala | 44 +-- .../mpeg/transport/AdaptationField.scala | 88 +++--- .../mpeg/transport/AdaptationFieldFlags.scala | 36 +-- .../protocols/mpeg/transport/Clock27MHz.scala | 5 +- .../mpeg/transport/Demultiplexer.scala | 254 +++++++++++------ .../mpeg/transport/DemultiplexerError.scala | 10 +- .../fs2/protocols/mpeg/transport/Packet.scala | 130 ++++++--- .../fs2/protocols/mpeg/transport/Pid.scala | 7 +- .../transport/TransportStreamHeader.scala | 33 +-- .../psi/ConditionalAccessTable.scala | 68 +++-- .../mpeg/transport/psi/GroupedSections.scala | 136 +++++---- .../mpeg/transport/psi/GroupingError.scala | 3 +- .../psi/ProgramAssociationTable.scala | 58 ++-- .../mpeg/transport/psi/ProgramMapTable.scala | 54 ++-- .../mpeg/transport/psi/Section.scala | 22 +- .../transport/psi/SectionAccumulator.scala | 24 +- .../mpeg/transport/psi/SectionCodec.scala | 86 ++++-- .../transport/psi/SectionFragmentCodec.scala | 63 ++++- .../mpeg/transport/psi/SectionHeader.scala | 14 +- .../protocols/mpeg/transport/psi/Table.scala | 13 +- .../mpeg/transport/psi/TableBuilder.scala | 19 +- .../transport/psi/TransportStreamEvent.scala | 87 +++--- .../transport/psi/TransportStreamIndex.scala | 57 ++-- .../scala/fs2/protocols/PcapMpegExample.scala | 27 +- .../fs2/protocols/mpeg/DescriptorTest.scala | 104 +++++-- .../protocols/mpeg/transport/PacketTest.scala | 24 +- .../mpeg/transport/psi/GroupingTest.scala | 15 +- .../mpeg/transport/psi/SectionCodecTest.scala | 149 +++++++--- 34 files changed, 1372 insertions(+), 801 deletions(-) diff --git a/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala index 6d8a43eb83..20322b20de 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/ip/Ipv6Header.scala @@ -28,30 +28,30 @@ import scodec._ import scodec.bits._ import scodec.codecs._ import fs2.interop.scodec._ -import fs2.protocols.ethernet.{EthernetFrameHeader, EtherType} +import fs2.protocols.ethernet.{EtherType, EthernetFrameHeader} import com.comcast.ip4s.Ipv6Address /** Simplified model of an IPv6 header -- extension headers are not directly supported. */ case class Ipv6Header( - trafficClass: Int, - flowLabel: Int, - payloadLength: Int, - protocol: Int, - hopLimit: Int, - sourceIp: Ipv6Address, - destinationIp: Ipv6Address + trafficClass: Int, + flowLabel: Int, + payloadLength: Int, + protocol: Int, + hopLimit: Int, + sourceIp: Ipv6Address, + destinationIp: Ipv6Address ) object Ipv6Header { implicit val codec: Codec[Ipv6Header] = { - ("version" | constant(bin"0110")) ~> - ("traffic_class" | uint8) :: - ("flow_label" | uint(20)) :: - ("payload_length" | uint(16)) :: - ("next_header" | uint8) :: - ("hop_limit" | uint8) :: - ("source_address" | Ip4sCodecs.ipv6) :: - ("destination_address" | Ip4sCodecs.ipv6) + ("version" | constant(bin"0110")) ~> + ("traffic_class" | uint8) :: + ("flow_label" | uint(20)) :: + ("payload_length" | uint(16)) :: + ("next_header" | uint8) :: + ("hop_limit" | uint8) :: + ("source_address" | Ip4sCodecs.ipv6) :: + ("destination_address" | Ip4sCodecs.ipv6) }.as[Ipv6Header] def sdecoder(ethernetHeader: EthernetFrameHeader): StreamDecoder[Ipv6Header] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala index 6c9dc9a30c..60af7fc473 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/Descriptor.scala @@ -34,43 +34,55 @@ trait KnownDescriptor sealed trait TransportStreamDescriptor extends KnownDescriptor sealed trait ProgramStreamDescriptor extends KnownDescriptor -case class Mpeg1Only(profileAndLevelIndication: Int, chromaFormat: Int, frameRateExtensionFlag: Boolean) +case class Mpeg1Only( + profileAndLevelIndication: Int, + chromaFormat: Int, + frameRateExtensionFlag: Boolean +) object Mpeg1Only { implicit val codec: Codec[Mpeg1Only] = { ("profile_and_level_indication" | uint8) :: - ("chroma_format" | uint(2)) :: - ("frame_rate_extension_flag" | bool) :: - ("reserved" | reserved(5)) + ("chroma_format" | uint(2)) :: + ("frame_rate_extension_flag" | bool) :: + ("reserved" | reserved(5)) }.dropUnits.as[Mpeg1Only] } case class VideoStreamDescriptor( - multipleFrameRateFlag: Boolean, - frameRateCode: Int, - mpeg1OnlyFlag: Boolean, - constrainedParameter: Boolean, - stillPictureFlag: Boolean, - mpeg1Only: Option[Mpeg1Only]) extends TransportStreamDescriptor with ProgramStreamDescriptor + multipleFrameRateFlag: Boolean, + frameRateCode: Int, + mpeg1OnlyFlag: Boolean, + constrainedParameter: Boolean, + stillPictureFlag: Boolean, + mpeg1Only: Option[Mpeg1Only] +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object VideoStreamDescriptor { val codec: Codec[VideoStreamDescriptor] = { ("multiple_frame_rate_flag" | bool) :: - ("frame_rate_code" | uint4) :: - (("MPEG_1_only_flag" | bool).flatPrepend { mpeg1Only => - ("constrained_parameter" | bool) :: - ("still_picture_flag" | bool) :: - ("MPEG_1_only_attributes" | conditional(mpeg1Only, Codec[Mpeg1Only])) - }) + ("frame_rate_code" | uint4) :: + ("MPEG_1_only_flag" | bool).flatPrepend { mpeg1Only => + ("constrained_parameter" | bool) :: + ("still_picture_flag" | bool) :: + ("MPEG_1_only_attributes" | conditional(mpeg1Only, Codec[Mpeg1Only])) + } }.as[VideoStreamDescriptor] } -case class AudioStreamDescriptor(freeFormatFlag: Boolean, id: Boolean, layer: Int, variableRateAudioIndicator: Boolean) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class AudioStreamDescriptor( + freeFormatFlag: Boolean, + id: Boolean, + layer: Int, + variableRateAudioIndicator: Boolean +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object AudioStreamDescriptor { val codec: Codec[AudioStreamDescriptor] = { ("free_format_flag" | bool) :: - ("ID" | bool) :: - ("layer" | uint(2)) :: - ("variable_rate_audio_indicator" | bool) :: - ("reserved" | reserved(3)) + ("ID" | bool) :: + ("layer" | uint(2)) :: + ("variable_rate_audio_indicator" | bool) :: + ("reserved" | reserved(3)) }.dropUnits.as[AudioStreamDescriptor] } @@ -87,7 +99,8 @@ object HierarchyType { case object BaseLayer extends HierarchyType implicit val codec: Codec[HierarchyType] = { - val m = discriminated[HierarchyType].by(uint4) + val m = discriminated[HierarchyType] + .by(uint4) .typecase(0, provide(Reserved(0))) .typecase(1, provide(SpatialScalability)) .typecase(2, provide(SnrScalability)) @@ -97,24 +110,36 @@ object HierarchyType { .typecase(6, provide(PrivateStream)) .typecase(7, provide(MultiViewProfile)) .typecase(15, provide(BaseLayer)) - (8 to 14).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + (8 to 14).foldLeft(m) { (acc, x) => + acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) + } } } -case class HierarchyDescriptor(hierarchyType: HierarchyType, hierarchyLayerIndex: Int, hierarchyEmbeddedLayerIndex: Int, hierarchyChannel: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class HierarchyDescriptor( + hierarchyType: HierarchyType, + hierarchyLayerIndex: Int, + hierarchyEmbeddedLayerIndex: Int, + hierarchyChannel: Int +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object HierarchyDescriptor { val codec: Codec[HierarchyDescriptor] = { ("reserved" | reserved(4)) :: - ("hierarchy_type" | Codec[HierarchyType]) :: - ("reserved" | reserved(2)) :: - ("hierarchy_layer_index" | uint(6)) :: - ("reserved" | reserved(2)) :: - ("hierarchy_embedded_layer_index" | uint(6)) :: - ("reserved" | reserved(2)) :: - ("hierarchy_channel" | uint(6)) + ("hierarchy_type" | Codec[HierarchyType]) :: + ("reserved" | reserved(2)) :: + ("hierarchy_layer_index" | uint(6)) :: + ("reserved" | reserved(2)) :: + ("hierarchy_embedded_layer_index" | uint(6)) :: + ("reserved" | reserved(2)) :: + ("hierarchy_channel" | uint(6)) }.dropUnits.as[HierarchyDescriptor] } -case class RegistrationDescriptor(formatIdentifier: ByteVector, additionalIdentificationInfo: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class RegistrationDescriptor( + formatIdentifier: ByteVector, + additionalIdentificationInfo: ByteVector +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object RegistrationDescriptor { val codec: Codec[RegistrationDescriptor] = { (("format_identifier" | bytes(4)) :: bytes) @@ -129,41 +154,55 @@ object AlignmentType { case object Seq extends AlignmentType case class Reserved(value: Int) extends AlignmentType implicit val codec: Codec[AlignmentType] = { - val m = discriminated[AlignmentType].by(uint8) + val m = discriminated[AlignmentType] + .by(uint8) .typecase(0, provide(Reserved(0))) .typecase(1, provide(SliceOrVideoAccessUnit)) .typecase(2, provide(VideoAccessUnit)) .typecase(3, provide(GopOrSeq)) .typecase(4, provide(Seq)) - (5 to 255).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + (5 to 255).foldLeft(m) { (acc, x) => + acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) + } } } -case class DataStreamAlignmentDescriptor(alignmentType: AlignmentType) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class DataStreamAlignmentDescriptor(alignmentType: AlignmentType) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object DataStreamAlignmentDescriptor { val codec: Codec[DataStreamAlignmentDescriptor] = { ("alignment_type" | Codec[AlignmentType]) }.as[DataStreamAlignmentDescriptor] } -case class TargetBackgroundGridDescriptor(horizontalSize: Int, verticalSize: Int, aspectRatioInformation: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class TargetBackgroundGridDescriptor( + horizontalSize: Int, + verticalSize: Int, + aspectRatioInformation: Int +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object TargetBackgroundGridDescriptor { val codec: Codec[TargetBackgroundGridDescriptor] = { ("horizontal_size" | uint(14)) :: - ("vertical_size" | uint(14)) :: - ("aspect_ratio_information" | uint4) + ("vertical_size" | uint(14)) :: + ("aspect_ratio_information" | uint4) }.as[TargetBackgroundGridDescriptor] } -case class VideoWindowDescriptor(horizontalOffset: Int, verticalOffset: Int, windowPriority: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class VideoWindowDescriptor(horizontalOffset: Int, verticalOffset: Int, windowPriority: Int) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object VideoWindowDescriptor { val codec: Codec[VideoWindowDescriptor] = { ("horizontal_offset" | uint(14)) :: - ("vertical_offset" | uint(14)) :: - ("window_priority" | uint4) + ("vertical_offset" | uint(14)) :: + ("window_priority" | uint4) }.as[VideoWindowDescriptor] } -case class CADescriptor(caSystemId: Int, caPid: Pid, privateData: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class CADescriptor(caSystemId: Int, caPid: Pid, privateData: ByteVector) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object CADescriptor { val codec: Codec[CADescriptor] = { ("CA_system_id" | uint16) :: reserved(3) :: ("CA_PID" | Codec[Pid]) :: bytes @@ -179,12 +218,15 @@ object AudioType { case class Reserved(value: Int) extends AudioType implicit val codec: Codec[AudioType] = { - val m = discriminated[AudioType].by(uint8) + val m = discriminated[AudioType] + .by(uint8) .typecase(0, provide(Undefined)) .typecase(1, provide(CleanEffects)) .typecase(2, provide(HearingImpaired)) .typecase(3, provide(VisualImpairedCommentary)) - (4 to 255).foldLeft(m) { (acc, x) => acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) } + (4 to 255).foldLeft(m) { (acc, x) => + acc.subcaseP(x)({ case Reserved(y) if x == y => Reserved(y) })(provide(Reserved(x))) + } } } @@ -192,40 +234,53 @@ case class LanguageField(iso639LanguageCode: String, audioType: AudioType) object LanguageField { implicit val codec: Codec[LanguageField] = { ("ISO_639_language_code" | fixedSizeBytes(3, ascii)) :: - ("audio_type" | Codec[AudioType]) + ("audio_type" | Codec[AudioType]) }.as[LanguageField] } -case class Iso639LanguageDescriptor(languageFields: Vector[LanguageField]) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class Iso639LanguageDescriptor(languageFields: Vector[LanguageField]) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object Iso639LanguageDescriptor { - val codec: Codec[Iso639LanguageDescriptor] = { + val codec: Codec[Iso639LanguageDescriptor] = vector(Codec[LanguageField]) - }.as[Iso639LanguageDescriptor] + .as[Iso639LanguageDescriptor] } - -case class SystemClockDescriptor(externalClockReferenceIndicator: Boolean, clockAccuracyInteger: Int, clockAccuracyExponent: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class SystemClockDescriptor( + externalClockReferenceIndicator: Boolean, + clockAccuracyInteger: Int, + clockAccuracyExponent: Int +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object SystemClockDescriptor { val codec: Codec[SystemClockDescriptor] = { ("external_clock_reference_indicator" | bool) :: - ("reserved" | reserved(1)) :: - ("clock_accuracy_integer" | uint(6)) :: - ("clock_accuracy_exponent" | uint(3)) :: - ("reserved" | reserved(5)) + ("reserved" | reserved(1)) :: + ("clock_accuracy_integer" | uint(6)) :: + ("clock_accuracy_exponent" | uint(3)) :: + ("reserved" | reserved(5)) }.dropUnits.as[SystemClockDescriptor] } -case class MultiplexBufferUtilizationDescriptor(boundValidFlag: Boolean, ltwOffsetLowerBound: Int, ltwOffsetUpperBound: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class MultiplexBufferUtilizationDescriptor( + boundValidFlag: Boolean, + ltwOffsetLowerBound: Int, + ltwOffsetUpperBound: Int +) extends TransportStreamDescriptor + with ProgramStreamDescriptor object MultiplexBufferUtilizationDescriptor { val codec: Codec[MultiplexBufferUtilizationDescriptor] = { ("bound_valid_flag" | bool) :: - ("LTW_offset_lower_bound" | uint(15)) :: - ("reserved" | reserved(1)) :: - ("LTW_offset_upper_bound" | uint(15)) + ("LTW_offset_lower_bound" | uint(15)) :: + ("reserved" | reserved(1)) :: + ("LTW_offset_upper_bound" | uint(15)) }.dropUnits.as[MultiplexBufferUtilizationDescriptor] } -case class CopyrightDescriptor(copyrightIdentifier: ByteVector, additionalCopyrightInfo: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class CopyrightDescriptor(copyrightIdentifier: ByteVector, additionalCopyrightInfo: ByteVector) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object CopyrightDescriptor { val codec: Codec[CopyrightDescriptor] = { bytes(4) :: bytes @@ -236,24 +291,28 @@ case class MaximumBitrateDescriptor(maximumBitrate: Int) extends TransportStream object MaximumBitrateDescriptor { val codec: Codec[MaximumBitrateDescriptor] = { ("reserved" | reserved(2)) :: - ("maximum_bitrate" | uint(22)) + ("maximum_bitrate" | uint(22)) }.dropUnits.as[MaximumBitrateDescriptor] } -case class PrivateDataIndicatorDescriptor(privateDataIndicator: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class PrivateDataIndicatorDescriptor(privateDataIndicator: ByteVector) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object PrivateDataIndicatorDescriptor { val codec: Codec[PrivateDataIndicatorDescriptor] = { ("private_data_indicator" | bytes(4)) }.as[PrivateDataIndicatorDescriptor] } -case class SmoothingBufferDescriptor(sbLeakRate: Int, sbSize: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class SmoothingBufferDescriptor(sbLeakRate: Int, sbSize: Int) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object SmoothingBufferDescriptor { val codec: Codec[SmoothingBufferDescriptor] = { ("reserved" | reserved(2)) :: - ("sb_leak_rate" | uint(22)) :: - ("reserved" | reserved(2)) :: - ("sb_size" | uint(22)) + ("sb_leak_rate" | uint(22)) :: + ("reserved" | reserved(2)) :: + ("sb_size" | uint(22)) }.dropUnits.as[SmoothingBufferDescriptor] } @@ -261,39 +320,47 @@ case class StdDescriptor(leakValidFlag: Boolean) extends TransportStreamDescript object StdDescriptor { val codec: Codec[StdDescriptor] = { ("reserved" | reserved(7)) :: - ("leak_valid_flag" | bool) + ("leak_valid_flag" | bool) }.dropUnits.as[StdDescriptor] } -case class IbpDescriptor(closedGopFlag: Boolean, identicalGopFlag: Boolean, maxGopLength: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class IbpDescriptor(closedGopFlag: Boolean, identicalGopFlag: Boolean, maxGopLength: Int) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object IbpDescriptor { val codec: Codec[IbpDescriptor] = { ("closed_gop_flag" | bool) :: - ("identical_gop_flag" | bool) :: - ("max_gop_length" | uint(14)) + ("identical_gop_flag" | bool) :: + ("max_gop_length" | uint(14)) }.as[IbpDescriptor] } -case class Mpeg4VideoDescriptor(mpeg4VisualProfileAndLevel: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class Mpeg4VideoDescriptor(mpeg4VisualProfileAndLevel: Byte) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object Mpeg4VideoDescriptor { val codec: Codec[Mpeg4VideoDescriptor] = { ("MPEG-4_visual_profile_and_level" | byte) }.as[Mpeg4VideoDescriptor] } -case class Mpeg4AudioDescriptor(mpeg4AudioProfileAndLevel: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class Mpeg4AudioDescriptor(mpeg4AudioProfileAndLevel: Byte) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object Mpeg4AudioDescriptor { val codec: Codec[Mpeg4AudioDescriptor] = { ("MPEG-4_audio_profile_and_level" | byte) }.as[Mpeg4AudioDescriptor] } -case class IodDescriptor(scopeOfIodLabel: Byte, iodLabel: Byte, initialObjectDescriptor: Byte) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class IodDescriptor(scopeOfIodLabel: Byte, iodLabel: Byte, initialObjectDescriptor: Byte) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object IodDescriptor { val codec: Codec[IodDescriptor] = { ("Scope_of_IOD_label" | byte) :: - ("IOD_label" | byte) :: - ("initialObjectDescriptor" | byte) + ("IOD_label" | byte) :: + ("initialObjectDescriptor" | byte) }.as[IodDescriptor] } @@ -308,42 +375,52 @@ case class EsIdAndChannel(esId: Int, flexMuxChannel: Int) object EsIdAndChannel { implicit val codec: Codec[EsIdAndChannel] = { ("ES_ID" | uint16) :: - ("FlexMuxChannel" | uint8) + ("FlexMuxChannel" | uint8) }.as[EsIdAndChannel] } -case class FmcDescriptor(channels: Vector[EsIdAndChannel]) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class FmcDescriptor(channels: Vector[EsIdAndChannel]) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object FmcDescriptor { - val codec: Codec[FmcDescriptor] = { + val codec: Codec[FmcDescriptor] = vector(Codec[EsIdAndChannel]) - }.as[FmcDescriptor] + .as[FmcDescriptor] } -case class ExternalEsIdDescriptor(esternalEsId: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class ExternalEsIdDescriptor(esternalEsId: Int) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object ExternalEsIdDescriptor { val codec: Codec[ExternalEsIdDescriptor] = { ("External_ES_ID" | uint16) }.as[ExternalEsIdDescriptor] } -case class MuxCodeDescriptor(muxCodeTableEntry: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class MuxCodeDescriptor(muxCodeTableEntry: ByteVector) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object MuxCodeDescriptor { - val codec: Codec[MuxCodeDescriptor] = { - bytes - }.as[MuxCodeDescriptor] + val codec: Codec[MuxCodeDescriptor] = + bytes + .as[MuxCodeDescriptor] } -case class FmxBufferSizeDescriptor(flexMuxBufferDescriptor: ByteVector) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class FmxBufferSizeDescriptor(flexMuxBufferDescriptor: ByteVector) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object FmxBufferSizeDescriptor { - val codec: Codec[FmxBufferSizeDescriptor] = { + val codec: Codec[FmxBufferSizeDescriptor] = bytes - }.as[FmxBufferSizeDescriptor] + .as[FmxBufferSizeDescriptor] } -case class MultiplexBufferDescriptor(mbBufferSize: Int, tbLeakRate: Int) extends TransportStreamDescriptor with ProgramStreamDescriptor +case class MultiplexBufferDescriptor(mbBufferSize: Int, tbLeakRate: Int) + extends TransportStreamDescriptor + with ProgramStreamDescriptor object MultiplexBufferDescriptor { val codec: Codec[MultiplexBufferDescriptor] = { ("MB_buffer_size" | uint24) :: - ("TB_leak_rate" | uint24) + ("TB_leak_rate" | uint24) }.as[MultiplexBufferDescriptor] } @@ -351,9 +428,9 @@ case class UnknownDescriptor(tag: Int, length: Int, data: ByteVector) object UnknownDescriptor { val codec: Codec[UnknownDescriptor] = { ("descriptor_tag" | uint8) :: - (("descriptor_length" | uint8).flatPrepend { length => - ("descriptor_data" | bytes(length)).tuple - }) + ("descriptor_length" | uint8).flatPrepend { length => + ("descriptor_data" | bytes(length)).tuple + } }.as[UnknownDescriptor] } @@ -362,7 +439,8 @@ object Descriptor { val knownCodec: Codec[KnownDescriptor] = { def sized[A](c: Codec[A]) = variableSizeBytes(uint8, c) - discriminated[KnownDescriptor].by(uint8) + discriminated[KnownDescriptor] + .by(uint8) .typecase(2, sized(VideoStreamDescriptor.codec)) .typecase(3, sized(AudioStreamDescriptor.codec)) .typecase(4, sized(HierarchyDescriptor.codec)) @@ -393,5 +471,5 @@ object Descriptor { val codec: Codec[Descriptor] = discriminatorFallback(UnknownDescriptor.codec, knownCodec) - def lengthCodec: Codec[Int] = ("descriptor_length" | uint8) + def lengthCodec: Codec[Int] = "descriptor_length" | uint8 } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala index ac8dc0ee28..662943224e 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/MpegError.scala @@ -42,7 +42,9 @@ object MpegError { override def toString = message } - def joinErrors[S, I, O](t: Scan[S, I, Either[MpegError, O]]): Scan[S, Either[MpegError, I], Either[MpegError, O]] = + def joinErrors[S, I, O]( + t: Scan[S, I, Either[MpegError, O]] + ): Scan[S, Either[MpegError, I], Either[MpegError, O]] = t.semipass(_.fold(e => Left(Left(e)), i => Right(i))) def passErrors[S, I, O](t: Scan[S, I, O]): Scan[S, Either[MpegError, I], Either[MpegError, O]] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala index a0f194bd4b..624df751b7 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacket.scala @@ -24,7 +24,7 @@ package fs2.protocols.mpeg import scodec.bits._ -import scodec.{ Attempt, Decoder, DecodeResult, Err } +import scodec.{Attempt, DecodeResult, Decoder, Err} trait PesPacket @@ -40,14 +40,16 @@ object PesPacket { def decoder(prefix: PesPacketHeaderPrefix): Decoder[PesPacket] = Decoder { buffer => val id = prefix.streamId import PesStreamId._ - if (id != ProgramStreamMap && - id != PaddingStream && - id != PrivateStream2 && - id != ECM && - id != EMM && - id != ProgramStreamDirectory && - id != DSMCC && - id != `ITU-T Rec. H.222.1 type E`) { + if ( + id != ProgramStreamMap && + id != PaddingStream && + id != PrivateStream2 && + id != ECM && + id != EMM && + id != ProgramStreamDirectory && + id != DSMCC && + id != `ITU-T Rec. H.222.1 type E` + ) { PesPacketHeader.codec.decode(buffer) match { case Attempt.Successful(DecodeResult(header, rest)) => decodeWithHeader(prefix, header, rest) @@ -58,9 +60,10 @@ object PesPacket { id == PrivateStream2 || id == ECM || id == EMM | - id == ProgramStreamDirectory || - id == DSMCC || - id == `ITU-T Rec. H.222.1 type E`) { + id == ProgramStreamDirectory || + id == DSMCC || + id == `ITU-T Rec. H.222.1 type E` + ) { Attempt.successful(DecodeResult(WithoutHeader(id, buffer), BitVector.empty)) } else if (id == PaddingStream) { Attempt.successful(DecodeResult(Padding, BitVector.empty)) @@ -69,7 +72,10 @@ object PesPacket { } } - def decodeWithHeader(prefix: PesPacketHeaderPrefix, header: PesPacketHeader, data: BitVector): Attempt[DecodeResult[PesPacket]] = { + def decodeWithHeader( + prefix: PesPacketHeaderPrefix, + header: PesPacketHeader, + data: BitVector + ): Attempt[DecodeResult[PesPacket]] = Attempt.successful(DecodeResult(WithHeader(prefix.streamId, header, data), BitVector.empty)) - } } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala index 0558416af7..91e432b2a2 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala @@ -34,122 +34,134 @@ object PesScramblingControl { object UserDefined2 extends PesScramblingControl object UserDefined3 extends PesScramblingControl - implicit val codec: Codec[PesScramblingControl] = mappedEnum(bits(2), + implicit val codec: Codec[PesScramblingControl] = mappedEnum( + bits(2), NotScrambled -> bin"00", UserDefined1 -> bin"01", UserDefined2 -> bin"10", - UserDefined3 -> bin"11") + UserDefined3 -> bin"11" + ) } case class PesPacketHeader( - pesScramblingControl: PesScramblingControl, - pesPriority: Boolean, - dataAlignmentIndicator: Boolean, - copyright: Boolean, - originalOrCopy: Boolean, - flags: PesPacketHeader.Flags, // TODO - pts: Option[Long], - dts: Option[Long], - escr: Option[Long], - esRate: Option[Int], - dsmTrickMode: Option[BitVector], - additionalCopyInfo: Option[BitVector], - pesCrc: Option[Int], - extension: Option[PesPacketHeader.Extension] + pesScramblingControl: PesScramblingControl, + pesPriority: Boolean, + dataAlignmentIndicator: Boolean, + copyright: Boolean, + originalOrCopy: Boolean, + flags: PesPacketHeader.Flags, // TODO + pts: Option[Long], + dts: Option[Long], + escr: Option[Long], + esRate: Option[Int], + dsmTrickMode: Option[BitVector], + additionalCopyInfo: Option[BitVector], + pesCrc: Option[Int], + extension: Option[PesPacketHeader.Extension] ) object PesPacketHeader { case class Flags( - ptsFlag: Boolean, - dtsFlag: Boolean, - escrFlag: Boolean, - esRateFlag: Boolean, - dsmTrickModeFlag: Boolean, - additionalCopyInfoFlag: Boolean, - pesCrcFlag: Boolean, - pesExtensionFlag: Boolean + ptsFlag: Boolean, + dtsFlag: Boolean, + escrFlag: Boolean, + esRateFlag: Boolean, + dsmTrickModeFlag: Boolean, + additionalCopyInfoFlag: Boolean, + pesCrcFlag: Boolean, + pesExtensionFlag: Boolean ) object Flags { implicit val codec: Codec[Flags] = { - ("pts_dts_flags[0]" | bool ) :: - ("pts_dts_flags[1]" | bool ) :: - ("escr_flag" | bool ) :: - ("es_rate_flag" | bool ) :: - ("dsm_trick_mode_flag" | bool ) :: - ("additional_copy_info_flag" | bool ) :: - ("pes_crc_flag" | bool ) :: - ("pes_extension_flag" | bool ) + ("pts_dts_flags[0]" | bool) :: + ("pts_dts_flags[1]" | bool) :: + ("escr_flag" | bool) :: + ("es_rate_flag" | bool) :: + ("dsm_trick_mode_flag" | bool) :: + ("additional_copy_info_flag" | bool) :: + ("pes_crc_flag" | bool) :: + ("pes_extension_flag" | bool) }.as[Flags] } case class ExtensionFlags( - pesPrivateDataFlag: Boolean, - packHeaderFieldFlag: Boolean, - programPacketSequenceCounterFlag: Boolean, - pstdBufferFlag: Boolean, - pesExtensionFlag2: Boolean + pesPrivateDataFlag: Boolean, + packHeaderFieldFlag: Boolean, + programPacketSequenceCounterFlag: Boolean, + pstdBufferFlag: Boolean, + pesExtensionFlag2: Boolean ) object ExtensionFlags { implicit val codec: Codec[ExtensionFlags] = { - ("pes_private_data_flag" | bool ) :: - ("pack_header_field_flag" | bool ) :: - ("program_packet_sequence_counter_flag" | bool ) :: - ("P-STD_buffer_flag" | bool ) :: - reserved(3) :: - ("pes_extension_flag_2" | bool ) + ("pes_private_data_flag" | bool) :: + ("pack_header_field_flag" | bool) :: + ("program_packet_sequence_counter_flag" | bool) :: + ("P-STD_buffer_flag" | bool) :: + reserved(3) :: + ("pes_extension_flag_2" | bool) }.dropUnits.as[ExtensionFlags] } case class ProgramPacketSequenceCounter(counter: Int, mpeg1: Boolean, originalStuffLength: Int) object ProgramPacketSequenceCounter { - implicit val codec: Codec[ProgramPacketSequenceCounter] = { + implicit val codec: Codec[ProgramPacketSequenceCounter] = (marker :: uint(7) :: marker :: bool :: uint(6)).dropUnits.as[ProgramPacketSequenceCounter] - } } case class PStdBuffer(scale: Boolean, size: Int) object PStdBuffer { - implicit val codec: Codec[PStdBuffer] = { + implicit val codec: Codec[PStdBuffer] = (constant(bin"01") ~> bool :: uint(13)).as[PStdBuffer] - } } case class Extension( - flags: ExtensionFlags, // TODO - pesPrivateData: Option[BitVector], - packHeaderField: Option[BitVector], - programPacketSequenceCounter: Option[ProgramPacketSequenceCounter], - pstdBuffer: Option[PStdBuffer], - extension: Option[BitVector] + flags: ExtensionFlags, // TODO + pesPrivateData: Option[BitVector], + packHeaderField: Option[BitVector], + programPacketSequenceCounter: Option[ProgramPacketSequenceCounter], + pstdBuffer: Option[PStdBuffer], + extension: Option[BitVector] ) object Extension { - implicit val codec: Codec[Extension] = { - Codec[ExtensionFlags].flatPrepend { flags => - ("pes_private_data" | conditional(flags.pesPrivateDataFlag, bits(128))) :: - ("pack_header_field" | conditional(flags.packHeaderFieldFlag, variableSizeBytes(uint8, bits))) :: - ("program_packet_sequence_counter" | conditional(flags.programPacketSequenceCounterFlag, Codec[ProgramPacketSequenceCounter])) :: - ("P-STD_buffer" | conditional(flags.pstdBufferFlag, Codec[PStdBuffer])) :: - ("pes_extension_2" | conditional(flags.pesExtensionFlag2, marker ~> variableSizeBytes(uint(7), bits))) - } - }.as[Extension] + implicit val codec: Codec[Extension] = + Codec[ExtensionFlags] + .flatPrepend { flags => + ("pes_private_data" | conditional(flags.pesPrivateDataFlag, bits(128))) :: + ("pack_header_field" | conditional( + flags.packHeaderFieldFlag, + variableSizeBytes(uint8, bits) + )) :: + ("program_packet_sequence_counter" | conditional( + flags.programPacketSequenceCounterFlag, + Codec[ProgramPacketSequenceCounter] + )) :: + ("P-STD_buffer" | conditional(flags.pstdBufferFlag, Codec[PStdBuffer])) :: + ("pes_extension_2" | conditional( + flags.pesExtensionFlag2, + marker ~> variableSizeBytes(uint(7), bits) + )) + } + .as[Extension] } private val marker: Codec[Unit] = constantLenient(bin"1") - private def tsCodec(prefix: BitVector) = { - (constant(prefix) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker).dropUnits.xmap[Long]( - { case (a, b, c) => (a ++ b ++ c).toLong() }, - l => { - val b = BitVector.fromLong(l).drop(31) - (b.take(3), b.drop(3).take(15), b.drop(18)) - } - ) - } + private def tsCodec(prefix: BitVector) = + (constant(prefix) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker).dropUnits + .xmap[Long]( + { case (a, b, c) => (a ++ b ++ c).toLong() }, + l => { + val b = BitVector.fromLong(l).drop(31) + (b.take(3), b.drop(3).take(15), b.drop(18)) + } + ) - private val escrCodec: Codec[Long] = { - (ignore(2) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker :: uint(9) :: marker).dropUnits.xmap[Long]( + private val escrCodec: Codec[Long] = + (ignore(2) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker :: uint( + 9 + ) :: marker).dropUnits.xmap[Long]( { case (a, b, c, ext) => val base = (a ++ b ++ c).toLong() base * 300 + ext @@ -161,26 +173,29 @@ object PesPacketHeader { (b.take(3), b.drop(3).take(15), b.drop(18), ext) } ) - } implicit val codec: Codec[PesPacketHeader] = { - constant(bin"10") ~> - ("pes_scrambling_control" | PesScramblingControl.codec ) :: - ("pes_priority" | bool ) :: - ("data_alignment_indicator" | bool ) :: - ("copyright" | bool ) :: - ("original_or_copy" | bool ) :: - (("flags" | Codec[Flags] ).flatPrepend { flags => - variableSizeBytes(uint8, - ("pts" | conditional(flags.ptsFlag, tsCodec(bin"0011")) ) :: - ("dts" | conditional(flags.dtsFlag, tsCodec(bin"0001")) ) :: - ("escr" | conditional(flags.escrFlag, escrCodec) ) :: - ("es_rate" | conditional(flags.esRateFlag, ignore(1) ~> uint(22) <~ ignore(1))) :: - ("dsm_trick_mode" | conditional(flags.dsmTrickModeFlag, bits(8)) ) :: - ("additional_copy_info" | conditional(flags.additionalCopyInfoFlag, ignore(1) ~> bits(7)) ) :: - ("pes_crc" | conditional(flags.pesCrcFlag, uint16) ) :: - ("extension" | conditional(flags.pesExtensionFlag, Codec[Extension]) ) - ) - }) // .removeElem[Flags](Generic[Flags].from(optionalFields.map(_.isDefined))) + constant(bin"10") ~> + ("pes_scrambling_control" | PesScramblingControl.codec) :: + ("pes_priority" | bool) :: + ("data_alignment_indicator" | bool) :: + ("copyright" | bool) :: + ("original_or_copy" | bool) :: + ("flags" | Codec[Flags]).flatPrepend { flags => + variableSizeBytes( + uint8, + ("pts" | conditional(flags.ptsFlag, tsCodec(bin"0011"))) :: + ("dts" | conditional(flags.dtsFlag, tsCodec(bin"0001"))) :: + ("escr" | conditional(flags.escrFlag, escrCodec)) :: + ("es_rate" | conditional(flags.esRateFlag, ignore(1) ~> uint(22) <~ ignore(1))) :: + ("dsm_trick_mode" | conditional(flags.dsmTrickModeFlag, bits(8))) :: + ("additional_copy_info" | conditional( + flags.additionalCopyInfoFlag, + ignore(1) ~> bits(7) + )) :: + ("pes_crc" | conditional(flags.pesCrcFlag, uint16)) :: + ("extension" | conditional(flags.pesExtensionFlag, Codec[Extension])) + ) + } // .removeElem[Flags](Generic[Flags].from(optionalFields.map(_.isDefined))) }.withContext("pes_packet_header").as[PesPacketHeader] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala index f4dcb36552..e9465f0f71 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeaderPrefix.scala @@ -30,10 +30,10 @@ case class PesPacketHeaderPrefix(streamId: Int, length: Int) object PesPacketHeaderPrefix { - implicit val codec: Codec[PesPacketHeaderPrefix] = { - fixedSizeBytes(3, - ("stream_id" | uint8 ) :: - ("pes_packet_length" | uint16) + implicit val codec: Codec[PesPacketHeaderPrefix] = + fixedSizeBytes( + 3, + ("stream_id" | uint8) :: + ("pes_packet_length" | uint16) ).as[PesPacketHeaderPrefix] - } } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala index dc869977de..b3c5e82d95 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesStreamId.scala @@ -27,28 +27,28 @@ import scodec.bits._ object PesStreamId { // format:off - val ProgramStreamMap = bin"1011 1100".toInt(signed = false) - val PrivateStream1 = bin"1011 1101".toInt(signed = false) - val PaddingStream = bin"1011 1110".toInt(signed = false) - val PrivateStream2 = bin"1011 1111".toInt(signed = false) - val AudioStreamMin = bin"1100 0000".toInt(signed = false) - val AudioStreamMax = bin"1101 1111".toInt(signed = false) - val VideoStreamMin = bin"1110 0000".toInt(signed = false) - val VideoStreamMax = bin"1110 1111".toInt(signed = false) - val ECM = bin"1111 0000".toInt(signed = false) - val EMM = bin"1111 0001".toInt(signed = false) - val DSMCC = bin"1111 0010".toInt(signed = false) - val `ISO/IEC 13522` = bin"1111 0011".toInt(signed = false) - val `ITU-T Rec. H.222.1 type A` = bin"1111 0100".toInt(signed = false) - val `ITU-T Rec. H.222.1 type B` = bin"1111 0101".toInt(signed = false) - val `ITU-T Rec. H.222.1 type C` = bin"1111 0110".toInt(signed = false) - val `ITU-T Rec. H.222.1 type D` = bin"1111 0111".toInt(signed = false) - val `ITU-T Rec. H.222.1 type E` = bin"1111 1000".toInt(signed = false) - val Ancillary = bin"1111 1001".toInt(signed = false) + val ProgramStreamMap = bin"1011 1100".toInt(signed = false) + val PrivateStream1 = bin"1011 1101".toInt(signed = false) + val PaddingStream = bin"1011 1110".toInt(signed = false) + val PrivateStream2 = bin"1011 1111".toInt(signed = false) + val AudioStreamMin = bin"1100 0000".toInt(signed = false) + val AudioStreamMax = bin"1101 1111".toInt(signed = false) + val VideoStreamMin = bin"1110 0000".toInt(signed = false) + val VideoStreamMax = bin"1110 1111".toInt(signed = false) + val ECM = bin"1111 0000".toInt(signed = false) + val EMM = bin"1111 0001".toInt(signed = false) + val DSMCC = bin"1111 0010".toInt(signed = false) + val `ISO/IEC 13522` = bin"1111 0011".toInt(signed = false) + val `ITU-T Rec. H.222.1 type A` = bin"1111 0100".toInt(signed = false) + val `ITU-T Rec. H.222.1 type B` = bin"1111 0101".toInt(signed = false) + val `ITU-T Rec. H.222.1 type C` = bin"1111 0110".toInt(signed = false) + val `ITU-T Rec. H.222.1 type D` = bin"1111 0111".toInt(signed = false) + val `ITU-T Rec. H.222.1 type E` = bin"1111 1000".toInt(signed = false) + val Ancillary = bin"1111 1001".toInt(signed = false) val `ISO/IEC14496-1 SL Packetized` = bin"1111 1010".toInt(signed = false) - val `ISO/IEC14496-1 FlexMux` = bin"1111 1011".toInt(signed = false) - val ReservedMin = bin"1111 1100".toInt(signed = false) - val ReservedMax = bin"1111 1110".toInt(signed = false) - val ProgramStreamDirectory = bin"1111 1111".toInt(signed = false) + val `ISO/IEC14496-1 FlexMux` = bin"1111 1011".toInt(signed = false) + val ReservedMin = bin"1111 1100".toInt(signed = false) + val ReservedMax = bin"1111 1110".toInt(signed = false) + val ProgramStreamDirectory = bin"1111 1111".toInt(signed = false) // format:on } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala index abb93f06f4..39efe62b12 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala @@ -24,20 +24,19 @@ package fs2.protocols.mpeg package transport -import scodec.{ Attempt, Codec, DecodeResult, SizeBound } +import scodec.{Attempt, Codec, DecodeResult, SizeBound} import scodec.bits.BitVector import scodec.codecs._ -/** - * Partial modelling of the adaptation field. - * The field extension, if present, is ignored upon decoding. - */ +/** Partial modelling of the adaptation field. + * The field extension, if present, is ignored upon decoding. + */ case class AdaptationField( - flags: Option[AdaptationFieldFlags], - pcr: Option[Clock27MHz], - opcr: Option[Clock27MHz], - spliceCountdown: Option[Int], - transportPrivateData: Option[BitVector] + flags: Option[AdaptationFieldFlags], + pcr: Option[Clock27MHz], + opcr: Option[Clock27MHz], + spliceCountdown: Option[Int], + transportPrivateData: Option[BitVector] ) object AdaptationField { @@ -46,41 +45,56 @@ object AdaptationField { implicit val codec: Codec[AdaptationField] = new Codec[AdaptationField] { private case class NonEmptyAF( - flags: AdaptationFieldFlags, - pcr: Option[Clock27MHz], - opcr: Option[Clock27MHz], - spliceCountdown: Option[Int], - transportPrivateData: Option[BitVector] - ) { def asAF: AdaptationField = AdaptationField(Some(flags), pcr, opcr, spliceCountdown, transportPrivateData) } + flags: AdaptationFieldFlags, + pcr: Option[Clock27MHz], + opcr: Option[Clock27MHz], + spliceCountdown: Option[Int], + transportPrivateData: Option[BitVector] + ) { + def asAF: AdaptationField = + AdaptationField(Some(flags), pcr, opcr, spliceCountdown, transportPrivateData) + } private val pcrCodec: Codec[Clock27MHz] = - ((ulong(33) <~ ignore(6)) :: uint(9)).xmap[Clock27MHz]({ case (base, ext) => - Clock27MHz(base * 300 + ext) - }, { clock => - val value = clock.value - val base = value / 300 - val ext = (value % 300).toInt - (base, ext) - }) + ((ulong(33) <~ ignore(6)) :: uint(9)).xmap[Clock27MHz]( + { case (base, ext) => + Clock27MHz(base * 300 + ext) + }, + { clock => + val value = clock.value + val base = value / 300 + val ext = (value % 300).toInt + (base, ext) + } + ) private val transportPrivateData: Codec[BitVector] = variableSizeBits(uint8, bits) - private val nonEmptyAFCodec: Codec[NonEmptyAF] = "adaptation_field" | { - variableSizeBytes(uint8, - ("adaptation_flags" | Codec[AdaptationFieldFlags] ).flatPrepend { flags => - ("pcr" | conditional(flags.pcrFlag, pcrCodec) ) :: - ("opcr" | conditional(flags.opcrFlag, pcrCodec) ) :: - ("splice_countdown" | conditional(flags.splicingPointFlag, int8) ) :: - ("transport_private_data" | conditional(flags.transportPrivateDataFlag, transportPrivateData)) - }) - }.as[NonEmptyAF] + private val nonEmptyAFCodec: Codec[NonEmptyAF] = "adaptation_field" | + variableSizeBytes( + uint8, + ("adaptation_flags" | Codec[AdaptationFieldFlags]).flatPrepend { flags => + ("pcr" | conditional(flags.pcrFlag, pcrCodec)) :: + ("opcr" | conditional(flags.opcrFlag, pcrCodec)) :: + ("splice_countdown" | conditional(flags.splicingPointFlag, int8)) :: + ("transport_private_data" | conditional( + flags.transportPrivateDataFlag, + transportPrivateData + )) + } + ) + .as[NonEmptyAF] def sizeBound: SizeBound = SizeBound.unknown def encode(af: AdaptationField): Attempt[BitVector] = af.flags.fold(uint8.encode(0)) { flags => - nonEmptyAFCodec.encode(NonEmptyAF(flags, af.pcr, af.opcr, af.spliceCountdown, af.transportPrivateData)) + nonEmptyAFCodec.encode( + NonEmptyAF(flags, af.pcr, af.opcr, af.spliceCountdown, af.transportPrivateData) + ) } - def decode(bv: BitVector): Attempt[DecodeResult[AdaptationField]] = uint8.decode(bv) flatMap { size => - if (size.value > 0) nonEmptyAFCodec.decode(bv).map(_.map(_.asAF)) else Attempt.successful(DecodeResult(Empty, size.remainder)) - } + def decode(bv: BitVector): Attempt[DecodeResult[AdaptationField]] = + uint8.decode(bv).flatMap { size => + if (size.value > 0) nonEmptyAFCodec.decode(bv).map(_.map(_.asAF)) + else Attempt.successful(DecodeResult(Empty, size.remainder)) + } } } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala index 6313cc3c4c..b6f9d5bb51 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationFieldFlags.scala @@ -29,24 +29,26 @@ import scodec.codecs._ /** Flags in the adaptation field. */ case class AdaptationFieldFlags( - discontinuity: Boolean, - randomAccess: Boolean, - priority: Boolean, - pcrFlag: Boolean, - opcrFlag: Boolean, - splicingPointFlag: Boolean, - transportPrivateDataFlag: Boolean, - adaptationFieldExtension: Boolean) + discontinuity: Boolean, + randomAccess: Boolean, + priority: Boolean, + pcrFlag: Boolean, + opcrFlag: Boolean, + splicingPointFlag: Boolean, + transportPrivateDataFlag: Boolean, + adaptationFieldExtension: Boolean +) object AdaptationFieldFlags { - implicit val codec: Codec[AdaptationFieldFlags] = "adaptation_field_flags" | fixedSizeBytes(1, - ("discontinuity" | bool ) :: - ("randomAccess" | bool ) :: - ("priority" | bool ) :: - ("pcrFlag" | bool ) :: - ("opcrFlag" | bool ) :: - ("splicingPointFlag" | bool ) :: - ("transportPrivateDataFlag" | bool ) :: - ("adaptationFieldExtension" | bool ) + implicit val codec: Codec[AdaptationFieldFlags] = "adaptation_field_flags" | fixedSizeBytes( + 1, + ("discontinuity" | bool) :: + ("randomAccess" | bool) :: + ("priority" | bool) :: + ("pcrFlag" | bool) :: + ("opcrFlag" | bool) :: + ("splicingPointFlag" | bool) :: + ("transportPrivateDataFlag" | bool) :: + ("adaptationFieldExtension" | bool) ).as[AdaptationFieldFlags] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala index 99028b3aa0..1b3e6e5030 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Clock27MHz.scala @@ -26,8 +26,9 @@ package fs2.protocols.mpeg.transport import scala.concurrent.duration.{FiniteDuration, MICROSECONDS} case class Clock27MHz(value: Long) { - def toDuration: FiniteDuration = FiniteDuration(((1000000d / 27000000) * value).toLong, MICROSECONDS) + def toDuration: FiniteDuration = + FiniteDuration(((1000000d / 27000000) * value).toLong, MICROSECONDS) def +(that: Clock27MHz): Clock27MHz = Clock27MHz(value + that.value) def -(that: Clock27MHz): Clock27MHz = Clock27MHz(value - that.value) -} \ No newline at end of file +} diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala index 702c7c0aec..ef1875dbf7 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Demultiplexer.scala @@ -26,12 +26,12 @@ package protocols package mpeg package transport -import scodec.{ Attempt, Codec, DecodeResult, Err } +import scodec.{Attempt, Codec, DecodeResult, Err} import scodec.Decoder import scodec.bits._ import scodec.codecs.fixedSizeBits -import fs2.protocols.mpeg.transport.psi.{ Section, SectionHeader, SectionCodec } +import fs2.protocols.mpeg.transport.psi.{Section, SectionCodec, SectionHeader} /** Supports depacketization of an MPEG transport stream, represented as a stream of `Packet`s. */ object Demultiplexer { @@ -40,18 +40,18 @@ object Demultiplexer { case class SectionResult(section: Section) extends Result case class PesPacketResult(body: PesPacket) extends Result - /** - * Indication that a header was decoded successfully and there was enough information on how to decode the body of the message. - * - * Upon receiving a result of this type, the demultiplexer will accumulate the number of bits specified by `neededBits` if that value - * is defined. If `neededBits` is undefined, the demultiplexer will accumulate all payload bits until the start of the next message (as - * indicated by the payload unit start indicator). When accumulation has completed, the specified decoder will be invoked to decode - * a message. - */ + /** Indication that a header was decoded successfully and there was enough information on how to decode the body of the message. + * + * Upon receiving a result of this type, the demultiplexer will accumulate the number of bits specified by `neededBits` if that value + * is defined. If `neededBits` is undefined, the demultiplexer will accumulate all payload bits until the start of the next message (as + * indicated by the payload unit start indicator). When accumulation has completed, the specified decoder will be invoked to decode + * a message. + */ case class DecodeBody[A](neededBits: Option[Long], decoder: Decoder[A]) /** Error that indicates any data accumulated by the demultiplexer should be dropped and no further decoding should occur until the next - * payload start. */ + * payload start. + */ case class ResetDecodeState(context: List[String]) extends Err { def message = "reset decode state" def pushContext(ctx: String) = ResetDecodeState(ctx :: context) @@ -62,114 +62,168 @@ object Demultiplexer { sealed trait DecodeState object DecodeState { - final case class AwaitingHeader(acc: BitVector, startedAtOffsetZero: Boolean) extends DecodeState + final case class AwaitingHeader(acc: BitVector, startedAtOffsetZero: Boolean) + extends DecodeState - final case class AwaitingBody[A](headerBits: BitVector, neededBits: Option[Long], bitsPostHeader: BitVector, decoder: Decoder[A]) extends DecodeState { + final case class AwaitingBody[A]( + headerBits: BitVector, + neededBits: Option[Long], + bitsPostHeader: BitVector, + decoder: Decoder[A] + ) extends DecodeState { def decode: Attempt[DecodeResult[A]] = decoder.decode(bitsPostHeader) - def accumulate(data: BitVector): AwaitingBody[A] = copy(bitsPostHeader = bitsPostHeader ++ data) + def accumulate(data: BitVector): AwaitingBody[A] = + copy(bitsPostHeader = bitsPostHeader ++ data) } } - private case class StepResult[+A](state: Option[DecodeState], output: Chunk[Either[DemultiplexerError, A]]) { - def ++[AA >: A](that: StepResult[AA]): StepResult[AA] = StepResult(that.state, Chunk.concat(List(output, that.output))) + private case class StepResult[+A]( + state: Option[DecodeState], + output: Chunk[Either[DemultiplexerError, A]] + ) { + def ++[AA >: A](that: StepResult[AA]): StepResult[AA] = + StepResult(that.state, Chunk.concat(List(output, that.output))) } private object StepResult { def noOutput(state: Option[DecodeState]): StepResult[Nothing] = apply(state, Chunk.empty) def state(state: DecodeState): StepResult[Nothing] = StepResult(Some(state), Chunk.empty) - def oneResult[A](state: Option[DecodeState], output: A): StepResult[A] = apply(state, Chunk.singleton(Right(output))) - def oneError(state: Option[DecodeState], err: DemultiplexerError): StepResult[Nothing] = apply(state, Chunk.singleton(Left(err))) + def oneResult[A](state: Option[DecodeState], output: A): StepResult[A] = + apply(state, Chunk.singleton(Right(output))) + def oneError(state: Option[DecodeState], err: DemultiplexerError): StepResult[Nothing] = + apply(state, Chunk.singleton(Left(err))) } - /** - * Stream transducer that converts packets in to sections and PES packets. - * - * The packets may span PID values. De-packetization is performed on each PID and as whole messages are received, - * reassembled messages are emitted. - * - * PES packets emitted by this method never include parsed headers -- that is, every emitted PES packet is of - * type `PesPacket.WithoutHeader`. To get PES packets with parsed headers, use `demultiplexWithPesHeaders`. - * - * Errors encountered while depacketizing are emitted. - * - * Upon noticing a PID discontinuity, an error is emitted and PID decoding state is discarded, resulting in any in-progress - * section decoding to be lost for that PID. - */ - def demultiplex[F[_]](sectionCodec: SectionCodec): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = - demultiplexSectionsAndPesPackets(sectionCodec.decoder, pph => Decoder(b => Attempt.successful(DecodeResult(PesPacket.WithoutHeader(pph.streamId, b), BitVector.empty)))) + /** Stream transducer that converts packets in to sections and PES packets. + * + * The packets may span PID values. De-packetization is performed on each PID and as whole messages are received, + * reassembled messages are emitted. + * + * PES packets emitted by this method never include parsed headers -- that is, every emitted PES packet is of + * type `PesPacket.WithoutHeader`. To get PES packets with parsed headers, use `demultiplexWithPesHeaders`. + * + * Errors encountered while depacketizing are emitted. + * + * Upon noticing a PID discontinuity, an error is emitted and PID decoding state is discarded, resulting in any in-progress + * section decoding to be lost for that PID. + */ + def demultiplex[F[_]]( + sectionCodec: SectionCodec + ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[ + Either[DemultiplexerError, Result] + ]] = + demultiplexSectionsAndPesPackets( + sectionCodec.decoder, + pph => + Decoder(b => + Attempt.successful( + DecodeResult(PesPacket.WithoutHeader(pph.streamId, b), BitVector.empty) + ) + ) + ) /** Variant of `demultiplex` that parses PES packet headers. */ - def demultiplexWithPesHeaders[F[_]](sectionCodec: SectionCodec): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = + def demultiplexWithPesHeaders[F[_]]( + sectionCodec: SectionCodec + ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[ + Either[DemultiplexerError, Result] + ]] = demultiplexSectionsAndPesPackets(sectionCodec.decoder, PesPacket.decoder) /** Variant of `demultiplex` that allows section and PES decoding to be explicitly specified. */ def demultiplexSectionsAndPesPackets[F[_]]( - decodeSectionBody: SectionHeader => Decoder[Section], - decodePesBody: PesPacketHeaderPrefix => Decoder[PesPacket]): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Result]]] = { + decodeSectionBody: SectionHeader => Decoder[Section], + decodePesBody: PesPacketHeaderPrefix => Decoder[PesPacket] + ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[ + Either[DemultiplexerError, Result] + ]] = { val stuffingByte = bin"11111111" - def decodeHeader(data: BitVector, startedAtOffsetZero: Boolean): Attempt[DecodeResult[DecodeBody[Result]]] = { + def decodeHeader( + data: BitVector, + startedAtOffsetZero: Boolean + ): Attempt[DecodeResult[DecodeBody[Result]]] = if (data.sizeLessThan(16)) { Attempt.failure(Err.InsufficientBits(16, data.size, Nil)) - } else if (data startsWith stuffingByte) { + } else if (data.startsWith(stuffingByte)) { Attempt.failure(ResetDecodeState(Nil)) } else { if (startedAtOffsetZero && data.take(16) == hex"0001".bits) { if (data.sizeLessThan(40)) { Attempt.failure(Err.InsufficientBits(40, data.size, Nil)) } else { - Codec[PesPacketHeaderPrefix].decode(data.drop(16)) map { _ map { header => + Codec[PesPacketHeaderPrefix].decode(data.drop(16)).map { + _.map { header => val neededBits = if (header.length == 0) None else Some(header.length * 8L) DecodeBody(neededBits, decodePesBody(header).map(PesPacketResult.apply)) - }} + } + } } } else { if (data.sizeLessThan(24)) { Attempt.failure(Err.InsufficientBits(24, data.size, Nil)) } else { - Codec[SectionHeader].decode(data) map { _ map { header => - DecodeBody(Some(header.length * 8L), decodeSectionBody(header).map(SectionResult.apply)) - }} + Codec[SectionHeader].decode(data).map { + _.map { header => + DecodeBody( + Some(header.length * 8L), + decodeSectionBody(header).map(SectionResult.apply) + ) + } + } } } } - } demultiplexGeneral(decodeHeader) } - /** - * Most general way to perform demultiplexing, allowing parsing of arbitrary headers and decoding of a specified output type. - * - * When processing the payload in a packet, the start of the payload is passed along to `decodeHeader`, which determines how to - * process the body of the message. - * - * In addition to the payload data, a flag is passed to `decodeHeader` -- true is passed when the payload data started at byte 0 of - * the packet and false is passed when the payload data started later in the packet. - * - * See the documentation on `DecodeBody` for more information. - */ + /** Most general way to perform demultiplexing, allowing parsing of arbitrary headers and decoding of a specified output type. + * + * When processing the payload in a packet, the start of the payload is passed along to `decodeHeader`, which determines how to + * process the body of the message. + * + * In addition to the payload data, a flag is passed to `decodeHeader` -- true is passed when the payload data started at byte 0 of + * the packet and false is passed when the payload data started later in the packet. + * + * See the documentation on `DecodeBody` for more information. + */ def demultiplexGeneral[F[_], Out]( - decodeHeader: (BitVector, Boolean) => Attempt[DecodeResult[DecodeBody[Out]]] - ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[Either[DemultiplexerError, Out]]] = { + decodeHeader: (BitVector, Boolean) => Attempt[DecodeResult[DecodeBody[Out]]] + ): Scan[(Map[Pid, ContinuityCounter], State), Packet, PidStamped[ + Either[DemultiplexerError, Out] + ]] = { - def processBody[A](awaitingBody: DecodeState.AwaitingBody[A], payloadUnitStartAfterData: Boolean): StepResult[Out] = { + def processBody[A]( + awaitingBody: DecodeState.AwaitingBody[A], + payloadUnitStartAfterData: Boolean + ): StepResult[Out] = { val haveFullBody = awaitingBody.neededBits match { - case None => payloadUnitStartAfterData + case None => payloadUnitStartAfterData case Some(needed) => awaitingBody.bitsPostHeader.size >= needed } if (haveFullBody) { awaitingBody.decode match { case Attempt.Successful(DecodeResult(body, remainder)) => - val decoded = StepResult.oneResult(None, body.asInstanceOf[Out]) // Safe cast b/c DecodeBody must provide a Decoder[Out] + val decoded = StepResult.oneResult( + None, + body.asInstanceOf[Out] + ) // Safe cast b/c DecodeBody must provide a Decoder[Out] decoded ++ processHeader(remainder, false, payloadUnitStartAfterData) case Attempt.Failure(err) => val out = { if (err.isInstanceOf[ResetDecodeState]) Chunk.empty - else Chunk.singleton(Left(DemultiplexerError.Decoding( - awaitingBody.headerBits ++ - awaitingBody.neededBits. - map { n => awaitingBody.bitsPostHeader.take(n) }. - getOrElse(awaitingBody.bitsPostHeader), err))) + else + Chunk.singleton( + Left( + DemultiplexerError.Decoding( + awaitingBody.headerBits ++ + awaitingBody.neededBits + .map(n => awaitingBody.bitsPostHeader.take(n)) + .getOrElse(awaitingBody.bitsPostHeader), + err + ) + ) + ) } val failure = StepResult(None, out) awaitingBody.neededBits match { @@ -184,7 +238,11 @@ object Demultiplexer { } } - def processHeader(acc: BitVector, startedAtOffsetZero: Boolean, payloadUnitStartAfterData: Boolean): StepResult[Out] = { + def processHeader( + acc: BitVector, + startedAtOffsetZero: Boolean, + payloadUnitStartAfterData: Boolean + ): StepResult[Out] = decodeHeader(acc, startedAtOffsetZero) match { case Attempt.Failure(e: Err.InsufficientBits) => StepResult.state(DecodeState.AwaitingHeader(acc, startedAtOffsetZero)) @@ -194,14 +252,20 @@ object Demultiplexer { StepResult.oneError(None, DemultiplexerError.Decoding(acc, e)) case Attempt.Successful(DecodeResult(DecodeBody(neededBits, decoder), bitsPostHeader)) => val guardedDecoder = neededBits match { - case None => decoder + case None => decoder case Some(n) => fixedSizeBits(n, decoder.decodeOnly) } - processBody(DecodeState.AwaitingBody(acc.take(24L), neededBits, bitsPostHeader, guardedDecoder), payloadUnitStartAfterData) + processBody( + DecodeState.AwaitingBody(acc.take(24L), neededBits, bitsPostHeader, guardedDecoder), + payloadUnitStartAfterData + ) } - } - def resume(state: DecodeState, newData: BitVector, payloadUnitStartAfterData: Boolean): StepResult[Out] = state match { + def resume( + state: DecodeState, + newData: BitVector, + payloadUnitStartAfterData: Boolean + ): StepResult[Out] = state match { case ah: DecodeState.AwaitingHeader => processHeader(ah.acc ++ newData, ah.startedAtOffsetZero, payloadUnitStartAfterData) @@ -209,15 +273,21 @@ object Demultiplexer { processBody(ab.accumulate(newData), payloadUnitStartAfterData) } - def handlePacket(state: Option[DecodeState], packet: Packet): StepResult[Out] = { + def handlePacket(state: Option[DecodeState], packet: Packet): StepResult[Out] = packet.payload match { case None => StepResult.noOutput(state) case Some(payload) => val currentResult = state match { case None => StepResult.noOutput(state) case Some(state) => - val currentData = packet.payloadUnitStart.map { start => payload.take(start.toLong * 8L) }.getOrElse(payload) - resume(state, currentData, payloadUnitStartAfterData = packet.payloadUnitStart.isDefined) + val currentData = packet.payloadUnitStart + .map(start => payload.take(start.toLong * 8L)) + .getOrElse(payload) + resume( + state, + currentData, + payloadUnitStartAfterData = packet.payloadUnitStart.isDefined + ) } packet.payloadUnitStart match { case None => @@ -227,24 +297,28 @@ object Demultiplexer { currentResult ++ nextResult } } - } - val demux = Scan.stateful[State, Either[PidStamped[DemultiplexerError.Discontinuity], Packet], PidStamped[Either[DemultiplexerError, Out]]](State(Map.empty)) { (state, event) => - event match { - case Right(packet) => - val pid = packet.header.pid - val oldStateForPid = state.byPid.get(pid) - val result = handlePacket(oldStateForPid, packet) - val newState = State(result.state.map { s => state.byPid.updated(pid, s) }.getOrElse(state.byPid - pid)) - val out = result.output.map { e => PidStamped(pid, e) } - (newState, out) - case Left(discontinuity) => - val newState = State(state.byPid - discontinuity.pid) - val out = Chunk.singleton(PidStamped(discontinuity.pid, Left(discontinuity.value))) - (newState, out) + val demux = + Scan.stateful[State, Either[PidStamped[DemultiplexerError.Discontinuity], Packet], PidStamped[ + Either[DemultiplexerError, Out] + ]](State(Map.empty)) { (state, event) => + event match { + case Right(packet) => + val pid = packet.header.pid + val oldStateForPid = state.byPid.get(pid) + val result = handlePacket(oldStateForPid, packet) + val newState = State( + result.state.map(s => state.byPid.updated(pid, s)).getOrElse(state.byPid - pid) + ) + val out = result.output.map(e => PidStamped(pid, e)) + (newState, out) + case Left(discontinuity) => + val newState = State(state.byPid - discontinuity.pid) + val out = Chunk.singleton(PidStamped(discontinuity.pid, Left(discontinuity.value))) + (newState, out) + } } - } - Packet.validateContinuity andThen demux + Packet.validateContinuity.andThen(demux) } } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala index 1053894599..562fb0ee4c 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/DemultiplexerError.scala @@ -33,8 +33,14 @@ sealed abstract class DemultiplexerError { object DemultiplexerError { - case class Discontinuity(last: ContinuityCounter, current: ContinuityCounter, adaptationFieldControl: Int) extends DemultiplexerError with MpegError { - def message = s"pid discontinuity: $last to $current with adaptation field control $adaptationFieldControl" + case class Discontinuity( + last: ContinuityCounter, + current: ContinuityCounter, + adaptationFieldControl: Int + ) extends DemultiplexerError + with MpegError { + def message = + s"pid discontinuity: $last to $current with adaptation field control $adaptationFieldControl" def toMpegError = this } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala index 30b6b5103e..3542774da3 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Packet.scala @@ -32,27 +32,44 @@ import scodec.codecs._ /** Transport stream packet. */ case class Packet( - header: TransportStreamHeader, - adaptationField: Option[AdaptationField], - payloadUnitStart: Option[Int], - payload: Option[BitVector] + header: TransportStreamHeader, + adaptationField: Option[AdaptationField], + payloadUnitStart: Option[Int], + payload: Option[BitVector] ) object Packet { - def packetize(pid: Pid, startingCountinuityCounter: ContinuityCounter, section: BitVector): Vector[Packet] = { + def packetize( + pid: Pid, + startingCountinuityCounter: ContinuityCounter, + section: BitVector + ): Vector[Packet] = { @annotation.tailrec - def go(first: Boolean, cc: ContinuityCounter, remaining: BitVector, acc: Vector[Packet]): Vector[Packet] = { + def go( + first: Boolean, + cc: ContinuityCounter, + remaining: BitVector, + acc: Vector[Packet] + ): Vector[Packet] = if (remaining.isEmpty) acc else { val (packetData, remData) = remaining.splitAt(8L * (if (first) 183 else 184)) - go(false, cc.next, remData, acc :+ payload(pid, cc, if (first) Some(0) else None, packetData)) + go( + false, + cc.next, + remData, + acc :+ payload(pid, cc, if (first) Some(0) else None, packetData) + ) } - } go(true, startingCountinuityCounter, section, Vector.empty) } - def packetizeMany(pid: Pid, startingCountinuityCounter: ContinuityCounter, sections: Vector[BitVector]): Vector[Packet] = { + def packetizeMany( + pid: Pid, + startingCountinuityCounter: ContinuityCounter, + sections: Vector[BitVector] + ): Vector[Packet] = { /* * Accumulates up to `n` bits from the specified bit vectors. @@ -61,9 +78,16 @@ object Packet { * - the left over bits of the last consumed input section * - the remaining unconsumed sections */ - def accumulateN(n: Long, sections: Vector[BitVector]): (BitVector, BitVector, Vector[BitVector]) = { + def accumulateN( + n: Long, + sections: Vector[BitVector] + ): (BitVector, BitVector, Vector[BitVector]) = { @annotation.tailrec - def go(needed: Long, remainingSections: Vector[BitVector], acc: BitVector): (BitVector, BitVector, Vector[BitVector]) = { + def go( + needed: Long, + remainingSections: Vector[BitVector], + acc: BitVector + ): (BitVector, BitVector, Vector[BitVector]) = if (remainingSections.isEmpty) (acc, BitVector.empty, Vector.empty) else { val (x, rem) = remainingSections.head.splitAt(needed) @@ -72,31 +96,41 @@ object Packet { if (left == 0) (newAcc, rem, remainingSections.tail) else go(left, remainingSections.tail, newAcc) } - } go(n, sections, BitVector.empty) } @annotation.tailrec - def go(cc: ContinuityCounter, remaining: BitVector, remainingSections: Vector[BitVector], acc: Vector[Packet]): Vector[Packet] = { + def go( + cc: ContinuityCounter, + remaining: BitVector, + remainingSections: Vector[BitVector], + acc: Vector[Packet] + ): Vector[Packet] = if (remaining.isEmpty && remainingSections.isEmpty) acc else { - val (packetData, overflow, remSections) = accumulateN(184 * 8, remaining +: remainingSections) + val (packetData, overflow, remSections) = + accumulateN(184 * 8, remaining +: remainingSections) val payloadUnitStart = { if (remSections.size < remainingSections.size) Some((remaining.size / 8).toInt) else None } val (adjPacketData, adjOverflow) = { - if (payloadUnitStart.isDefined) (packetData.take(183 * 8), packetData.drop(183 * 8) ++ overflow) + if (payloadUnitStart.isDefined) + (packetData.take(183 * 8), packetData.drop(183 * 8) ++ overflow) else (packetData, overflow) } val packet = payload(pid, cc, payloadUnitStart, adjPacketData) go(cc.next, adjOverflow, remSections, acc :+ packet) } - } go(startingCountinuityCounter, BitVector.empty, sections, Vector.empty) } - def payload(pid: Pid, continuityCounter: ContinuityCounter, payloadUnitStart: Option[Int], payload: BitVector): Packet = { + def payload( + pid: Pid, + continuityCounter: ContinuityCounter, + payloadUnitStart: Option[Int], + payload: BitVector + ): Packet = { val thisPid = pid val thisContinuityCounter = continuityCounter val thisPayloadUnitStart = payloadUnitStart @@ -115,34 +149,56 @@ object Packet { ), adaptationField = None, payloadUnitStart = thisPayloadUnitStart, - payload = Some(thisPayload)) + payload = Some(thisPayload) + ) } implicit def codec(implicit adaptationField: Codec[AdaptationField]): Codec[Packet] = - "packet" | fixedSizeBytes(188, - ("header" | Codec[TransportStreamHeader] ).flatPrepend { hdr => - ("adaptation_field" | conditional(hdr.adaptationFieldIncluded, adaptationField) ) :: - ("payload_start_ind" | conditional(hdr.payloadUnitStartIndicator, uint8) ) :: - ("payload" | conditional(hdr.payloadIncluded, bits) ) - }).as[Packet] + "packet" | fixedSizeBytes( + 188, + ("header" | Codec[TransportStreamHeader]).flatPrepend { hdr => + ("adaptation_field" | conditional(hdr.adaptationFieldIncluded, adaptationField)) :: + ("payload_start_ind" | conditional(hdr.payloadUnitStartIndicator, uint8)) :: + ("payload" | conditional(hdr.payloadIncluded, bits)) + } + ).as[Packet] - def validateContinuity: Scan[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[DemultiplexerError.Discontinuity], Packet]] = - Scan.stateful[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[DemultiplexerError.Discontinuity], Packet]](Map.empty) { (state, packet) => + def validateContinuity: Scan[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[ + DemultiplexerError.Discontinuity + ], Packet]] = + Scan.stateful[Map[Pid, ContinuityCounter], Packet, Either[PidStamped[ + DemultiplexerError.Discontinuity + ], Packet]](Map.empty) { (state, packet) => val pid = packet.header.pid val currentContinuityCounter = packet.header.continuityCounter - val err = state.get(pid).map { lastContinuityCounter => - val expectedContinuityCounter = - if (packet.header.adaptationFieldControl == 0 || packet.header.adaptationFieldControl == 2) lastContinuityCounter else lastContinuityCounter.next - if (expectedContinuityCounter == currentContinuityCounter) { - None - } else { - val err: Either[PidStamped[DemultiplexerError.Discontinuity], Packet] = - Left(PidStamped(pid, DemultiplexerError.Discontinuity(lastContinuityCounter, currentContinuityCounter, packet.header.adaptationFieldControl))) - Some(err) + val err = state + .get(pid) + .map { lastContinuityCounter => + val expectedContinuityCounter = + if ( + packet.header.adaptationFieldControl == 0 || packet.header.adaptationFieldControl == 2 + ) lastContinuityCounter + else lastContinuityCounter.next + if (expectedContinuityCounter == currentContinuityCounter) { + None + } else { + val err: Either[PidStamped[DemultiplexerError.Discontinuity], Packet] = + Left( + PidStamped( + pid, + DemultiplexerError.Discontinuity( + lastContinuityCounter, + currentContinuityCounter, + packet.header.adaptationFieldControl + ) + ) + ) + Some(err) + } } - }.getOrElse(None) + .getOrElse(None) val newState = state + (pid -> currentContinuityCounter) - val out = err.map { e => Chunk(e, Right(packet)) }.getOrElse(Chunk.singleton(Right(packet))) + val out = err.map(e => Chunk(e, Right(packet))).getOrElse(Chunk.singleton(Right(packet))) (newState, out) } } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala index 218c3b01e9..f6b5d8421e 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/Pid.scala @@ -46,10 +46,9 @@ case class PidStamped[+A](pid: Pid, value: A) { object PidStamped { - /** - * Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, PidStamped[I], PidStamped[O]]` such that - * pidstamps are preserved on elements that flow through the stream. - */ + /** Combinator that converts a `Scan[S, I, O]` in to a `Scan[S, PidStamped[I], PidStamped[O]]` such that + * pidstamps are preserved on elements that flow through the stream. + */ def preserve[S, I, O](t: Scan[S, I, O]): Scan[S, PidStamped[I], PidStamped[O]] = t.lens(_.value, (psi, o) => psi.copy(value = o)) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala index 4ad40b4a46..eab4d92d83 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/TransportStreamHeader.scala @@ -28,27 +28,28 @@ import scodec.Codec import scodec.codecs._ case class TransportStreamHeader( - transportErrorIndicator: Boolean, - payloadUnitStartIndicator: Boolean, - transportPriority: Boolean, - pid: Pid, - scramblingControl: Int, - adaptationFieldControl: Int, - continuityCounter: ContinuityCounter + transportErrorIndicator: Boolean, + payloadUnitStartIndicator: Boolean, + transportPriority: Boolean, + pid: Pid, + scramblingControl: Int, + adaptationFieldControl: Int, + continuityCounter: ContinuityCounter ) { def adaptationFieldIncluded: Boolean = adaptationFieldControl >= 2 def payloadIncluded: Boolean = adaptationFieldControl == 1 || adaptationFieldControl == 3 } object TransportStreamHeader { - implicit val codec: Codec[TransportStreamHeader] = "transport_stream_header" | fixedSizeBytes(4, - ("syncByte" | constant(0x47) ) ~> - ("transportErrorIndicator" | bool ) :: - ("payloadUnitStartIndicator" | bool ) :: - ("transportPriority" | bool ) :: - ("pid" | Codec[Pid] ) :: - ("scramblingControl" | uint2 ) :: - ("adaptationFieldControl" | uint2 ) :: - ("continuityCounter" | Codec[ContinuityCounter]) + implicit val codec: Codec[TransportStreamHeader] = "transport_stream_header" | fixedSizeBytes( + 4, + ("syncByte" | constant(0x47)) ~> + ("transportErrorIndicator" | bool) :: + ("payloadUnitStartIndicator" | bool) :: + ("transportPriority" | bool) :: + ("pid" | Codec[Pid]) :: + ("scramblingControl" | uint2) :: + ("adaptationFieldControl" | uint2) :: + ("continuityCounter" | Codec[ContinuityCounter]) ).as[TransportStreamHeader] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala index 0181337898..779102622a 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ConditionalAccessTable.scala @@ -30,12 +30,13 @@ import scodec.bits._ import scodec.codecs._ case class ConditionalAccessTable( - version: Int, - current: Boolean, - descriptors: List[ConditionalAccessDescriptor] + version: Int, + current: Boolean, + descriptors: List[ConditionalAccessDescriptor] ) extends Table { def tableId = ConditionalAccessSection.TableId - def toSections: GroupedSections[ConditionalAccessSection] = ConditionalAccessTable.toSections(this) + def toSections: GroupedSections[ConditionalAccessSection] = + ConditionalAccessTable.toSections(this) } object ConditionalAccessTable { @@ -44,19 +45,31 @@ object ConditionalAccessTable { val grouped = groupBasedOnSize(cat.descriptors.toVector) val lastSection = grouped.size - 1 val sections = grouped.zipWithIndex.map { case (ds, idx) => - ConditionalAccessSection(SectionExtension(65535, cat.version, cat.current, idx, lastSection), ds.toList) + ConditionalAccessSection( + SectionExtension(65535, cat.version, cat.current, idx, lastSection), + ds.toList + ) } if (sections.isEmpty) - GroupedSections(ConditionalAccessSection(SectionExtension(65535, cat.version, cat.current, 0, 0), Nil)) + GroupedSections( + ConditionalAccessSection(SectionExtension(65535, cat.version, cat.current, 0, 0), Nil) + ) else GroupedSections(sections.head, sections.tail.toList) } - private def groupBasedOnSize(sections: Vector[ConditionalAccessDescriptor]): Vector[Vector[ConditionalAccessDescriptor]] = { + private def groupBasedOnSize( + sections: Vector[ConditionalAccessDescriptor] + ): Vector[Vector[ConditionalAccessDescriptor]] = { val MaxBitsLeft = (1024 - 12) * 8L def sizeOf(c: ConditionalAccessDescriptor): Long = (6 * 8) + c.privateData.size @annotation.tailrec - def go(remaining: Vector[ConditionalAccessDescriptor], cur: Vector[ConditionalAccessDescriptor], bitsLeft: Long, acc: Vector[Vector[ConditionalAccessDescriptor]]): Vector[Vector[ConditionalAccessDescriptor]] = { + def go( + remaining: Vector[ConditionalAccessDescriptor], + cur: Vector[ConditionalAccessDescriptor], + bitsLeft: Long, + acc: Vector[Vector[ConditionalAccessDescriptor]] + ): Vector[Vector[ConditionalAccessDescriptor]] = if (remaining.isEmpty) acc :+ cur else { val next = remaining.head @@ -67,11 +80,12 @@ object ConditionalAccessTable { go(remaining, Vector.empty, MaxBitsLeft, acc :+ cur) } } - } go(sections, Vector.empty, MaxBitsLeft, Vector.empty) } - def fromSections(sections: GroupedSections[ConditionalAccessSection]): Either[String, ConditionalAccessTable] = { + def fromSections( + sections: GroupedSections[ConditionalAccessSection] + ): Either[String, ConditionalAccessTable] = { def extract[A](name: String, f: ConditionalAccessSection => A): Either[String, A] = { val extracted = sections.list.map(f).distinct if (extracted.size == 1) Right(extracted.head) @@ -80,7 +94,7 @@ object ConditionalAccessTable { for { version <- extract("versions", _.extension.version) } yield { - val current = sections.list.foldLeft(false) { (acc, s) => acc || s.extension.current } + val current = sections.list.foldLeft(false)((acc, s) => acc || s.extension.current) ConditionalAccessTable( version, current, @@ -92,17 +106,20 @@ object ConditionalAccessTable { } } - implicit val tableSupport: TableSupport[ConditionalAccessTable] = new TableSupport[ConditionalAccessTable] { - def tableId = ConditionalAccessSection.TableId - def toTable(gs: GroupedSections[Section]) = - gs.narrow[ConditionalAccessSection].toRight(s"Not CAT sections").flatMap { sections => fromSections(sections) } - def toSections(cat: ConditionalAccessTable) = ConditionalAccessTable.toSections(cat) - } + implicit val tableSupport: TableSupport[ConditionalAccessTable] = + new TableSupport[ConditionalAccessTable] { + def tableId = ConditionalAccessSection.TableId + def toTable(gs: GroupedSections[Section]) = + gs.narrow[ConditionalAccessSection].toRight(s"Not CAT sections").flatMap { sections => + fromSections(sections) + } + def toSections(cat: ConditionalAccessTable) = ConditionalAccessTable.toSections(cat) + } } case class ConditionalAccessSection( - extension: SectionExtension, - descriptors: List[ConditionalAccessDescriptor] + extension: SectionExtension, + descriptors: List[ConditionalAccessDescriptor] ) extends ExtendedSection { def tableId = ConditionalAccessSection.TableId } @@ -130,12 +147,13 @@ object ConditionalAccessDescriptor { implicit val codec: Codec[ConditionalAccessDescriptor] = { constant(Tag) ~> - variableSizeBytes(uint8, - ("ca_system_id" | uint16) :: - (reserved(3) ~> - ("ca_pid" | Codec[Pid])) :: - bits - ) + variableSizeBytes( + uint8, + ("ca_system_id" | uint16) :: + (reserved(3) ~> + ("ca_pid" | Codec[Pid])) :: + bits + ) }.as[ConditionalAccessDescriptor] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala index db30aacaf4..47a3d6b090 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupedSections.scala @@ -30,11 +30,10 @@ import scala.reflect.ClassTag import fs2._ -/** - * Group of sections that make up a logical message. - * - * Intermediate representation between sections and tables. All sections must share the same table id. - */ +/** Group of sections that make up a logical message. + * + * Intermediate representation between sections and tables. All sections must share the same table id. + */ sealed abstract class GroupedSections[+A <: Section] { def tableId: Int @@ -46,14 +45,17 @@ sealed abstract class GroupedSections[+A <: Section] { object GroupedSections { implicit class InvariantOps[A <: Section](val self: GroupedSections[A]) extends AnyVal { - def narrow[B <: A : ClassTag]: Option[GroupedSections[B]] = { - val matched = self.list.foldLeft(true) { (acc, s) => s match { case _: B => true; case _ => false } } + def narrow[B <: A: ClassTag]: Option[GroupedSections[B]] = { + val matched = self.list.foldLeft(true) { (acc, s) => + s match { case _: B => true; case _ => false } + } if (matched) Some(self.asInstanceOf[GroupedSections[B]]) else None } } - private case class DefaultGroupedSections[A <: Section](head: A, tail: List[A]) extends GroupedSections[A] { + private case class DefaultGroupedSections[A <: Section](head: A, tail: List[A]) + extends GroupedSections[A] { val tableId = head.tableId val list = head :: tail } @@ -62,18 +64,28 @@ object GroupedSections { DefaultGroupedSections[A](head, tail) final case class ExtendedTableId(tableId: Int, tableIdExtension: Int) - final case class ExtendedSectionGrouperState[A <: ExtendedSection](accumulatorByIds: Map[ExtendedTableId, SectionAccumulator[A]]) - - def groupExtendedSections[A <: ExtendedSection]: Scan[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]] = { - def toKey(section: A): ExtendedTableId = ExtendedTableId(section.tableId, section.extension.tableIdExtension) - Scan.stateful[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]](ExtendedSectionGrouperState(Map.empty)) { (state, section) => + final case class ExtendedSectionGrouperState[A <: ExtendedSection]( + accumulatorByIds: Map[ExtendedTableId, SectionAccumulator[A]] + ) + + def groupExtendedSections[A <: ExtendedSection] + : Scan[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]] = { + def toKey(section: A): ExtendedTableId = + ExtendedTableId(section.tableId, section.extension.tableIdExtension) + Scan.stateful[ExtendedSectionGrouperState[A], A, Either[GroupingError, GroupedSections[A]]]( + ExtendedSectionGrouperState(Map.empty) + ) { (state, section) => val key = toKey(section) val (err, acc) = state.accumulatorByIds.get(key) match { case None => (None, SectionAccumulator(section)) case Some(acc) => acc.add(section) match { case Right(acc) => (None, acc) - case Left(err) => (Some(GroupingError(section.tableId, section.extension.tableIdExtension, err)), SectionAccumulator(section)) + case Left(err) => + ( + Some(GroupingError(section.tableId, section.extension.tableIdExtension, err)), + SectionAccumulator(section) + ) } } @@ -93,51 +105,67 @@ object GroupedSections { def noGrouping: Scan[Unit, Section, Either[GroupingError, GroupedSections[Section]]] = Scan.lift(s => Right(GroupedSections(s))) - /** - * Groups sections in to groups. - * - * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. - * Non-extended sections are emitted as singleton groups. - */ - def group: Scan[ExtendedSectionGrouperState[ExtendedSection], Section, Either[GroupingError, GroupedSections[Section]]] = { + /** Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. + * Non-extended sections are emitted as singleton groups. + */ + def group: Scan[ExtendedSectionGrouperState[ExtendedSection], Section, Either[ + GroupingError, + GroupedSections[Section] + ]] = groupGeneral((), noGrouping).imapState(_._2)(s => ((), s)) - } - /** - * Groups sections in to groups. - * - * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. - * The specified `nonExtended` process is used to handle non-extended sections. - */ + /** Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled. + * The specified `nonExtended` process is used to handle non-extended sections. + */ def groupGeneral[NonExtendedState]( - initialNonExtendedState: NonExtendedState, - nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]] - ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]] = { + initialNonExtendedState: NonExtendedState, + nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]] + ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[ + GroupingError, + GroupedSections[Section] + ]] = groupGeneralConditionally(initialNonExtendedState, nonExtended, _ => true) - } - /** - * Groups sections in to groups. - * - * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled if `true` is returned from the - * `groupExtended` function when applied with the section in question. - * - * The specified `nonExtended` transducer is used to handle non-extended sections. - */ + /** Groups sections in to groups. + * + * Extended sections, aka sections with the section syntax indicator set to true, are automatically handled if `true` is returned from the + * `groupExtended` function when applied with the section in question. + * + * The specified `nonExtended` transducer is used to handle non-extended sections. + */ def groupGeneralConditionally[NonExtendedState]( - initialNonExtendedState: NonExtendedState, - nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]], - groupExtended: ExtendedSection => Boolean = _ => true - ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]] = { - Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[GroupingError, GroupedSections[Section]]]((initialNonExtendedState, ExtendedSectionGrouperState(Map.empty)))({ case ((nonExtendedState, extendedState), section) => - section match { - case s: ExtendedSection if groupExtended(s) => - val (s2, out) = groupExtendedSections.transform(extendedState, s) - (nonExtendedState -> s2, out) - case s: Section => - val (s2, out) = nonExtended.transform(nonExtendedState, s) - (s2 -> extendedState, out) + initialNonExtendedState: NonExtendedState, + nonExtended: Scan[NonExtendedState, Section, Either[GroupingError, GroupedSections[Section]]], + groupExtended: ExtendedSection => Boolean = _ => true + ): Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[ + GroupingError, + GroupedSections[Section] + ]] = + Scan[(NonExtendedState, ExtendedSectionGrouperState[ExtendedSection]), Section, Either[ + GroupingError, + GroupedSections[Section] + ]]((initialNonExtendedState, ExtendedSectionGrouperState(Map.empty)))( + { case ((nonExtendedState, extendedState), section) => + section match { + case s: ExtendedSection if groupExtended(s) => + val (s2, out) = groupExtendedSections.transform(extendedState, s) + (nonExtendedState -> s2, out) + case s: Section => + val (s2, out) = nonExtended.transform(nonExtendedState, s) + (s2 -> extendedState, out) + } + }, + { case (nonExtendedState, extendedState) => + Chunk.concat( + List( + nonExtended.onComplete(nonExtendedState), + groupExtendedSections.onComplete(extendedState) + ) + ) } - }, { case (nonExtendedState, extendedState) => Chunk.concat(List(nonExtended.onComplete(nonExtendedState), groupExtendedSections.onComplete(extendedState))) }) - } + ) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala index 7356b65201..bd76c8081c 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/GroupingError.scala @@ -25,7 +25,8 @@ package fs2.protocols.mpeg package transport package psi -case class GroupingError(tableId: Int, tableIdExtension: Option[Int], message: String) extends MpegError +case class GroupingError(tableId: Int, tableIdExtension: Option[Int], message: String) + extends MpegError object GroupingError { def apply(tableId: Int, tableIdExtension: Int, message: String): GroupingError = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala index 50b1775d32..f786257113 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala @@ -29,13 +29,14 @@ import scodec.Codec import scodec.codecs._ case class ProgramAssociationTable( - tsid: TransportStreamId, - version: Int, - current: Boolean, - programByPid: Map[ProgramNumber, Pid] + tsid: TransportStreamId, + version: Int, + current: Boolean, + programByPid: Map[ProgramNumber, Pid] ) extends Table { def tableId = ProgramAssociationSection.TableId - def toSections: GroupedSections[ProgramAssociationSection] = ProgramAssociationTable.toSections(this) + def toSections: GroupedSections[ProgramAssociationSection] = + ProgramAssociationTable.toSections(this) } object ProgramAssociationTable { @@ -47,15 +48,25 @@ object ProgramAssociationTable { val groupedEntries = entries.grouped(MaxProgramsPerSection).toVector val lastSection = groupedEntries.size - 1 val sections = groupedEntries.zipWithIndex.map { case (es, idx) => - ProgramAssociationSection(SectionExtension(pat.tsid.value, pat.version, pat.current, idx, lastSection), es) + ProgramAssociationSection( + SectionExtension(pat.tsid.value, pat.version, pat.current, idx, lastSection), + es + ) } if (sections.isEmpty) - GroupedSections(ProgramAssociationSection(SectionExtension(pat.tsid.value, pat.version, pat.current, 0, 0), Vector.empty)) + GroupedSections( + ProgramAssociationSection( + SectionExtension(pat.tsid.value, pat.version, pat.current, 0, 0), + Vector.empty + ) + ) else GroupedSections(sections.head, sections.tail.toList) } - def fromSections(sections: GroupedSections[ProgramAssociationSection]): Either[String, ProgramAssociationTable] = { + def fromSections( + sections: GroupedSections[ProgramAssociationSection] + ): Either[String, ProgramAssociationTable] = { def extract[A](name: String, f: ProgramAssociationSection => A): Either[String, A] = { val extracted = sections.list.map(f).distinct if (extracted.size == 1) Right(extracted.head) @@ -65,7 +76,7 @@ object ProgramAssociationTable { tsid <- extract("TSIDs", _.tsid) version <- extract("versions", _.extension.version) } yield { - val current = sections.list.foldLeft(false) { (acc, s) => acc || s.extension.current } + val current = sections.list.foldLeft(false)((acc, s) => acc || s.extension.current) ProgramAssociationTable( tsid, version, @@ -73,21 +84,25 @@ object ProgramAssociationTable { (for { section <- sections.list pidMapping <- section.pidMappings - } yield pidMapping).toMap) + } yield pidMapping).toMap + ) } } - implicit val tableSupport: TableSupport[ProgramAssociationTable] = new TableSupport[ProgramAssociationTable] { - def tableId = ProgramAssociationSection.TableId - def toTable(gs: GroupedSections[Section]) = - gs.narrow[ProgramAssociationSection].toRight("Not PAT sections").flatMap { sections => fromSections(sections) } - def toSections(pat: ProgramAssociationTable) = ProgramAssociationTable.toSections(pat) - } + implicit val tableSupport: TableSupport[ProgramAssociationTable] = + new TableSupport[ProgramAssociationTable] { + def tableId = ProgramAssociationSection.TableId + def toTable(gs: GroupedSections[Section]) = + gs.narrow[ProgramAssociationSection].toRight("Not PAT sections").flatMap { sections => + fromSections(sections) + } + def toSections(pat: ProgramAssociationTable) = ProgramAssociationTable.toSections(pat) + } } case class ProgramAssociationSection( - extension: SectionExtension, - pidMappings: Vector[(ProgramNumber, Pid)] + extension: SectionExtension, + pidMappings: Vector[(ProgramNumber, Pid)] ) extends ExtendedSection { def tableId = ProgramAssociationSection.TableId def tsid: TransportStreamId = TransportStreamId(extension.tableIdExtension) @@ -98,13 +113,12 @@ object ProgramAssociationSection { private type Fragment = Vector[(ProgramNumber, Pid)] - private val fragmentCodec: Codec[Fragment] = { + private val fragmentCodec: Codec[Fragment] = vector { ("program_number" | Codec[ProgramNumber]) :: - (reserved(3) ~> - ("pid" | Codec[Pid])) + (reserved(3) ~> + ("pid" | Codec[Pid])) } - } implicit val sectionFragmentCodec: SectionFragmentCodec[ProgramAssociationSection] = SectionFragmentCodec.psi[ProgramAssociationSection, Vector[(ProgramNumber, Pid)]]( diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala index aff15c5c56..e8c7c726a8 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala @@ -31,31 +31,35 @@ import scodec.codecs._ import Descriptor._ case class ProgramMapTable( - programNumber: ProgramNumber, - version: Int, - current: Boolean, - pcrPid: Pid, - programInfoDescriptors: List[Descriptor], - componentStreamMapping: Map[StreamType, List[ProgramMapRecord]] + programNumber: ProgramNumber, + version: Int, + current: Boolean, + pcrPid: Pid, + programInfoDescriptors: List[Descriptor], + componentStreamMapping: Map[StreamType, List[ProgramMapRecord]] ) extends Table { def tableId = ProgramMapSection.TableId } object ProgramMapTable { - def toSection(pmt: ProgramMapTable): ProgramMapSection = { + def toSection(pmt: ProgramMapTable): ProgramMapSection = ProgramMapSection( SectionExtension(pmt.programNumber.value, pmt.version, pmt.current, 0, 0), pmt.pcrPid, pmt.programInfoDescriptors, - (for ((st, pmrs) <- pmt.componentStreamMapping.toVector; pmr <- pmrs) yield (st, pmr)).sortBy { case (k, v) => (k.value, v.pid.value) } + (for { + (st, pmrs) <- pmt.componentStreamMapping.toVector + pmr <- pmrs + } yield (st, pmr)).sortBy { case (k, v) => (k.value, v.pid.value) } ) - } def fromSection(section: ProgramMapSection): ProgramMapTable = { - val componentStreamMapping = section.componentStreamMapping.foldLeft(Map.empty[StreamType, List[ProgramMapRecord]]) { case (acc, (st, pmr)) => - acc.updated(st, acc.get(st).fold(List(pmr))(existing => pmr :: existing)) - }.map { case (k, v) => (k, v.reverse) } + val componentStreamMapping = section.componentStreamMapping + .foldLeft(Map.empty[StreamType, List[ProgramMapRecord]]) { case (acc, (st, pmr)) => + acc.updated(st, acc.get(st).fold(List(pmr))(existing => pmr :: existing)) + } + .map { case (k, v) => (k, v.reverse) } ProgramMapTable( section.programNumber, section.extension.version, @@ -72,7 +76,7 @@ object ProgramMapTable { gs.narrow[ProgramMapSection].toRight("Not PMT sections").flatMap { sections => if (sections.tail.isEmpty) Right(fromSection(sections.head)) else Left(s"PMT supports only 1 section but got ${sections.list.size}") - } + } def toSections(pmt: ProgramMapTable) = GroupedSections(ProgramMapTable.toSection(pmt)) } } @@ -84,10 +88,10 @@ object ProgramMapRecord { } case class ProgramMapSection( - extension: SectionExtension, - pcrPid: Pid, - programInfoDescriptors: List[Descriptor], - componentStreamMapping: Vector[(StreamType, ProgramMapRecord)] + extension: SectionExtension, + pcrPid: Pid, + programInfoDescriptors: List[Descriptor], + componentStreamMapping: Vector[(StreamType, ProgramMapRecord)] ) extends ExtendedSection { def tableId = ProgramMapSection.TableId def programNumber: ProgramNumber = ProgramNumber(extension.tableIdExtension) @@ -105,18 +109,20 @@ object ProgramMapSection { (("pid" | pid) :: ("es_descriptors" | descriptors)).as[ProgramMapRecord] ("pcr_pid" | pid) :: - ("program_info_descriptors" | descriptors) :: - vector { - ("stream_type" | uint8.as[StreamType]) :: programMapRecord - } + ("program_info_descriptors" | descriptors) :: + vector { + ("stream_type" | uint8.as[StreamType]) :: programMapRecord + } } implicit val sectionSubCodec: SectionFragmentCodec[ProgramMapSection] = SectionFragmentCodec.psi[ProgramMapSection, Fragment]( TableId, - (ext, fragment) => fragment match { - case (pcrPid, descriptors, mapping) => ProgramMapSection(ext, pcrPid, descriptors, mapping) - }, + (ext, fragment) => + fragment match { + case (pcrPid, descriptors, mapping) => + ProgramMapSection(ext, pcrPid, descriptors, mapping) + }, pmt => (pmt.extension, (pmt.pcrPid, pmt.programInfoDescriptors, pmt.componentStreamMapping)) )(fragmentCodec) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala index 676329bf9f..7743a930a4 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Section.scala @@ -37,20 +37,20 @@ trait ExtendedSection extends Section { } case class SectionExtension( - tableIdExtension: Int, - version: Int, - current: Boolean, - sectionNumber: Int, - lastSectionNumber: Int + tableIdExtension: Int, + version: Int, + current: Boolean, + sectionNumber: Int, + lastSectionNumber: Int ) object SectionExtension { implicit val codec: Codec[SectionExtension] = { - ("table_id_extension" | uint16) :: - (reserved(2) ~> - ("version_number" | uint(5))) :: - ("current_next_indicator" | bool) :: - ("section_number" | uint8) :: - ("last_section_number" | uint8) + ("table_id_extension" | uint16) :: + (reserved(2) ~> + ("version_number" | uint(5))) :: + ("current_next_indicator" | bool) :: + ("section_number" | uint8) :: + ("last_section_number" | uint8) }.as[SectionExtension] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala index 05a83d955e..79e8f3b61c 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionAccumulator.scala @@ -26,7 +26,10 @@ package transport package psi /** Accumulates sections of the same table id and table id extension. */ -private[psi] class SectionAccumulator[A <: ExtendedSection] private (val sections: GroupedSections[A], sectionByNumber: Map[Int, A]) { +private[psi] class SectionAccumulator[A <: ExtendedSection] private ( + val sections: GroupedSections[A], + sectionByNumber: Map[Int, A] +) { def add(section: A): Either[String, SectionAccumulator[A]] = { def validate(err: => String)(f: Boolean): Either[String, Unit] = @@ -41,19 +44,26 @@ private[psi] class SectionAccumulator[A <: ExtendedSection] private (val section _ <- checkEquality("table id extensions")(_.extension.tableIdExtension) _ <- checkEquality("versions")(_.extension.version) _ <- checkEquality("last section numbers")(_.extension.lastSectionNumber) - _ <- validate("invalid section number")(sectionNumber <= sections.head.extension.lastSectionNumber) + _ <- validate("invalid section number")( + sectionNumber <= sections.head.extension.lastSectionNumber + ) _ <- validate("duplicate section number")(!sectionByNumber.contains(sectionNumber)) - } yield new SectionAccumulator(GroupedSections(section, sections.list), sectionByNumber + (section.extension.sectionNumber -> section)) + } yield new SectionAccumulator( + GroupedSections(section, sections.list), + sectionByNumber + (section.extension.sectionNumber -> section) + ) } def complete: Option[GroupedSections[A]] = - if (sectionByNumber.size == (sections.head.extension.lastSectionNumber + 1)) Some(sections) else None + if (sectionByNumber.size == (sections.head.extension.lastSectionNumber + 1)) Some(sections) + else None } private[psi] object SectionAccumulator { def apply[A <: ExtendedSection](section: A): SectionAccumulator[A] = - new SectionAccumulator(GroupedSections(section), Map(section.extension.sectionNumber -> section)) + new SectionAccumulator( + GroupedSections(section), + Map(section.extension.sectionNumber -> section) + ) } - - diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala index cc7daa0ee5..e71784ab23 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala @@ -25,11 +25,14 @@ package fs2.protocols.mpeg package transport package psi -import scodec.{ Attempt, Codec, Decoder, DecodeResult, Err, SizeBound } +import scodec.{Attempt, Codec, DecodeResult, Decoder, Err, SizeBound} import scodec.bits._ import scodec.codecs._ -class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section]]], verifyCrc: Boolean = true) extends Codec[Section] { +class SectionCodec private ( + cases: Map[Int, List[SectionCodec.Case[Any, Section]]], + verifyCrc: Boolean = true +) extends Codec[Section] { import SectionCodec._ def supporting[A <: Section](implicit c: SectionFragmentCodec[A]): SectionCodec = { @@ -63,8 +66,13 @@ class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section] } for { - cs <- Attempt.fromOption(cases.get(section.tableId), Err(s"unsupported table id ${section.tableId}")) - enc <- cs.dropRight(1).foldRight(tryEncode(cs.last)) { (next, res) => res orElse tryEncode(next) } + cs <- Attempt.fromOption( + cases.get(section.tableId), + Err(s"unsupported table id ${section.tableId}") + ) + enc <- cs.dropRight(1).foldRight(tryEncode(cs.last)) { (next, res) => + res.orElse(tryEncode(next)) + } } yield enc } @@ -76,12 +84,10 @@ class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section] def decodeSection(header: SectionHeader)(bits: BitVector): Attempt[DecodeResult[Section]] = decoder(header).decode(bits) - def decoder(header: SectionHeader): Decoder[Section] = { + def decoder(header: SectionHeader): Decoder[Section] = decoder(header, Codec.encode(header).require) - } def decoder(header: SectionHeader, headerBits: BitVector): Decoder[Section] = Decoder { bits => - def ensureCrcMatches(actual: Int, expected: Int) = if (actual == expected) { Attempt.successful(()) } else Attempt.failure(Err(s"CRC mismatch: calculated $expected does not equal $actual")) @@ -89,7 +95,9 @@ class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section] def generateCrc: Int = crc32mpeg(headerBits ++ bits.take((header.length.toLong - 4) * 8)).toInt() - def decodeExtended(c: SectionCodec.Case[Any, Section]): Decoder[(Option[SectionExtension], Any)] = for { + def decodeExtended( + c: SectionCodec.Case[Any, Section] + ): Decoder[(Option[SectionExtension], Any)] = for { ext <- Codec[SectionExtension] data <- fixedSizeBytes(header.length.toLong - 9, c.codec(header, verifyCrc)) actualCrc <- int32 @@ -97,18 +105,23 @@ class SectionCodec private (cases: Map[Int, List[SectionCodec.Case[Any, Section] _ <- Decoder.liftAttempt(ensureCrcMatches(actualCrc, expectedCrc)) } yield Some(ext) -> data - def decodeStandard(c: SectionCodec.Case[Any, Section]): Decoder[(Option[SectionExtension], Any)] = for { + def decodeStandard( + c: SectionCodec.Case[Any, Section] + ): Decoder[(Option[SectionExtension], Any)] = for { data <- fixedSizeBytes(header.length.toLong, c.codec(header, verifyCrc)) } yield None -> data def attemptDecode(c: SectionCodec.Case[Any, Section]): Attempt[DecodeResult[Section]] = for { - result <- ( if (header.extendedSyntax) decodeExtended(c) else decodeStandard(c) ).decode(bits) + result <- (if (header.extendedSyntax) decodeExtended(c) else decodeStandard(c)).decode(bits) DecodeResult((ext, data), remainder) = result section <- c.toSection(header.privateBits, ext, data) } yield DecodeResult(section, remainder) - val cs = cases.getOrElse(header.tableId, List(unknownSectionCase(header.tableId).asInstanceOf[Case[Any, Section]])) - cs.foldRight(attemptDecode(cs.head)) { (next, res) => res orElse attemptDecode(next) } + val cs = cases.getOrElse( + header.tableId, + List(unknownSectionCase(header.tableId).asInstanceOf[Case[Any, Section]]) + ) + cs.foldRight(attemptDecode(cs.head))((next, res) => res.orElse(attemptDecode(next))) } } @@ -116,41 +129,52 @@ object SectionCodec { def empty: SectionCodec = new SectionCodec(Map.empty) - def supporting[S <: Section : SectionFragmentCodec]: SectionCodec = + def supporting[S <: Section: SectionFragmentCodec]: SectionCodec = empty.supporting[S] def psi: SectionCodec = - supporting[ProgramAssociationSection]. - supporting[ProgramMapSection]. - supporting[ConditionalAccessSection] + supporting[ProgramAssociationSection] + .supporting[ProgramMapSection] + .supporting[ConditionalAccessSection] sealed trait UnknownSection extends Section - case class UnknownNonExtendedSection(tableId: Int, privateBits: BitVector, data: ByteVector) extends UnknownSection - case class UnknownExtendedSection(tableId: Int, privateBits: BitVector, extension: SectionExtension, data: ByteVector) extends UnknownSection with ExtendedSection + case class UnknownNonExtendedSection(tableId: Int, privateBits: BitVector, data: ByteVector) + extends UnknownSection + case class UnknownExtendedSection( + tableId: Int, + privateBits: BitVector, + extension: SectionExtension, + data: ByteVector + ) extends UnknownSection + with ExtendedSection private case class Case[A, B <: Section]( - codec: (SectionHeader, Boolean) => Codec[A], - toSection: (BitVector, Option[SectionExtension], A) => Attempt[B], - fromSection: B => (BitVector, Option[SectionExtension], A)) + codec: (SectionHeader, Boolean) => Codec[A], + toSection: (BitVector, Option[SectionExtension], A) => Attempt[B], + fromSection: B => (BitVector, Option[SectionExtension], A) + ) private object Case { - def fromSectionFragmentCodec[A <: Section](c: SectionFragmentCodec[A]): Case[Any, Section] = { + def fromSectionFragmentCodec[A <: Section](c: SectionFragmentCodec[A]): Case[Any, Section] = Case[Any, Section]( (hdr, verifyCrc) => c.subCodec(hdr, verifyCrc).asInstanceOf[Codec[Any]], - (privateBits, extension, data) => c.toSection(privateBits, extension, data.asInstanceOf[c.Repr]), + (privateBits, extension, data) => + c.toSection(privateBits, extension, data.asInstanceOf[c.Repr]), section => c.fromSection(section.asInstanceOf[A]) ) - } } private def unknownSectionCase(tableId: Int): Case[BitVector, UnknownSection] = Case( (hdr, verifyCrc) => bits, - (privateBits, ext, bits) => Attempt.successful(ext match { - case Some(e) => UnknownExtendedSection(tableId, privateBits, e, bits.bytes) - case None => UnknownNonExtendedSection(tableId, privateBits, bits.bytes) + (privateBits, ext, bits) => + Attempt.successful(ext match { + case Some(e) => UnknownExtendedSection(tableId, privateBits, e, bits.bytes) + case None => UnknownNonExtendedSection(tableId, privateBits, bits.bytes) }), - section => section match { - case u: UnknownExtendedSection => (u.privateBits, Some(u.extension), u.data.bits) - case u: UnknownNonExtendedSection => (u.privateBits, None, u.data.bits) - }) + section => + section match { + case u: UnknownExtendedSection => (u.privateBits, Some(u.extension), u.data.bits) + case u: UnknownNonExtendedSection => (u.privateBits, None, u.data.bits) + } + ) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala index 5cd1b1aa54..68f7056c43 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionFragmentCodec.scala @@ -26,7 +26,7 @@ package transport package psi import scodec.bits._ -import scodec.{ Attempt, Codec, Err } +import scodec.{Attempt, Codec, Err} trait SectionFragmentCodec[A] { type Repr @@ -39,10 +39,22 @@ trait SectionFragmentCodec[A] { object SectionFragmentCodec { private val PsiPrivateBits = bin"011" - def psi[A, R: Codec](tableId: Int, toSection: (SectionExtension, R) => A, fromSection: A => (SectionExtension, R)): SectionFragmentCodec[A] = - extended[A, R](tableId, (_, ext, r) => toSection(ext, r), s => { val (ext, r) = fromSection(s); (PsiPrivateBits, ext, r) }) + def psi[A, R: Codec]( + tableId: Int, + toSection: (SectionExtension, R) => A, + fromSection: A => (SectionExtension, R) + ): SectionFragmentCodec[A] = + extended[A, R]( + tableId, + (_, ext, r) => toSection(ext, r), + s => { val (ext, r) = fromSection(s); (PsiPrivateBits, ext, r) } + ) - def extended[A, R: Codec](tableId: Int, toSection: (BitVector, SectionExtension, R) => A, fromSection: A => (BitVector, SectionExtension, R)): SectionFragmentCodec[A] = { + def extended[A, R: Codec]( + tableId: Int, + toSection: (BitVector, SectionExtension, R) => A, + fromSection: A => (BitVector, SectionExtension, R) + ): SectionFragmentCodec[A] = { val tid = tableId val build = toSection val extract = fromSection @@ -51,13 +63,21 @@ object SectionFragmentCodec { def tableId = tid def subCodec(header: SectionHeader, verifyCrc: Boolean) = Codec[Repr] def toSection(privateBits: BitVector, extension: Option[SectionExtension], data: Repr) = - Attempt.fromOption(extension.map { ext => build(privateBits, ext, data) }, Err("extended section missing expected section extension")) + Attempt.fromOption( + extension.map(ext => build(privateBits, ext, data)), + Err("extended section missing expected section extension") + ) def fromSection(section: A) = extract(section) match { case (privateBits, ext, data) => (privateBits, Some(ext), data) } } } - def nonExtended[A, R](tableId: Int, toCodec: SectionHeader => Codec[R], toSection: (BitVector, R) => A, fromSection: A => (BitVector, R)): SectionFragmentCodec[A] = { + def nonExtended[A, R]( + tableId: Int, + toCodec: SectionHeader => Codec[R], + toSection: (BitVector, R) => A, + fromSection: A => (BitVector, R) + ): SectionFragmentCodec[A] = { val tid = tableId val codec = toCodec val build = toSection @@ -75,7 +95,12 @@ object SectionFragmentCodec { } } - def nonExtendedWithCrc[A, R](tableId: Int, toCodec: (SectionHeader, Boolean) => Codec[R], toSection: (BitVector, R) => A, fromSection: A => (BitVector, R)): SectionFragmentCodec[A] = { + def nonExtendedWithCrc[A, R]( + tableId: Int, + toCodec: (SectionHeader, Boolean) => Codec[R], + toSection: (BitVector, R) => A, + fromSection: A => (BitVector, R) + ): SectionFragmentCodec[A] = { val tid = tableId val codec = toCodec val build = toSection @@ -93,9 +118,25 @@ object SectionFragmentCodec { } } - def nonExtendedIdentity[A](tableId: Int, toCodec: SectionHeader => Codec[A]): SectionFragmentCodec[A] = - SectionFragmentCodec.nonExtended[A, A](tableId, sHdr => toCodec(sHdr), (bits, a) => a, a => (BitVector.empty, a)) + def nonExtendedIdentity[A]( + tableId: Int, + toCodec: SectionHeader => Codec[A] + ): SectionFragmentCodec[A] = + SectionFragmentCodec.nonExtended[A, A]( + tableId, + sHdr => toCodec(sHdr), + (bits, a) => a, + a => (BitVector.empty, a) + ) - def nonExtendedIdentityWithCrc[A](tableId: Int, toCodec: (SectionHeader, Boolean) => Codec[A]): SectionFragmentCodec[A] = - SectionFragmentCodec.nonExtendedWithCrc[A, A](tableId, (sHdr, verifyCrc) => toCodec(sHdr, verifyCrc), (bits, a) => a, a => (BitVector.empty, a)) + def nonExtendedIdentityWithCrc[A]( + tableId: Int, + toCodec: (SectionHeader, Boolean) => Codec[A] + ): SectionFragmentCodec[A] = + SectionFragmentCodec.nonExtendedWithCrc[A, A]( + tableId, + (sHdr, verifyCrc) => toCodec(sHdr, verifyCrc), + (bits, a) => a, + a => (BitVector.empty, a) + ) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala index 57c9c40d1f..0c36dc3650 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionHeader.scala @@ -29,18 +29,14 @@ import scodec.Codec import scodec.bits.BitVector import scodec.codecs._ -case class SectionHeader( - tableId: Int, - extendedSyntax: Boolean, - privateBits: BitVector, - length: Int) +case class SectionHeader(tableId: Int, extendedSyntax: Boolean, privateBits: BitVector, length: Int) object SectionHeader { implicit val codec: Codec[SectionHeader] = { - ("table_id" | uint8 ) :: - ("section_syntax_indicator" | bool ) :: - ("private_bits" | bits(3) ) :: - ("length" | uint(12) ) + ("table_id" | uint8) :: + ("section_syntax_indicator" | bool) :: + ("private_bits" | bits(3)) :: + ("length" | uint(12)) }.as[SectionHeader] } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala index 8f4adafa70..728cfc0619 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/Table.scala @@ -25,13 +25,12 @@ package fs2.protocols.mpeg package transport package psi -/** - * Indicates the implementor can be treated as a message delivered in an MPEG transport stream. - * - * This library differentiates tables from sections. Sections are the actual messages delivered - * in the transport stream whereas tables are the result of grouping multiple related sections - * together in to a single logical message. - */ +/** Indicates the implementor can be treated as a message delivered in an MPEG transport stream. + * + * This library differentiates tables from sections. Sections are the actual messages delivered + * in the transport stream whereas tables are the result of grouping multiple related sections + * together in to a single logical message. + */ trait Table { def tableId: Int } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala index 6745d20b83..7545f1919e 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala @@ -35,28 +35,29 @@ class TableBuilder private (cases: Map[Int, List[TableSupport[_]]]) { new TableBuilder(cases + (ts.tableId -> newCases)) } - def build(gs: GroupedSections[Section]): Either[TableBuildingError, Table] = { + def build(gs: GroupedSections[Section]): Either[TableBuildingError, Table] = cases.get(gs.tableId) match { case None | Some(Nil) => Left(TableBuildingError(gs.tableId, "Unknown table id")) case Some(list) => - list.dropRight(1).foldRight[Either[String, _]](list.last.toTable(gs)) { (next, res) => res.fold(_ => next.toTable(gs), Right(_)) } match { + list.dropRight(1).foldRight[Either[String, _]](list.last.toTable(gs)) { (next, res) => + res.fold(_ => next.toTable(gs), Right(_)) + } match { case Right(table) => Right(table.asInstanceOf[Table]) - case Left(err) => Left(TableBuildingError(gs.tableId, err)) + case Left(err) => Left(TableBuildingError(gs.tableId, err)) } } - } } object TableBuilder { def empty: TableBuilder = new TableBuilder(Map.empty) - def supporting[T <: Table : TableSupport] = empty.supporting[T] + def supporting[T <: Table: TableSupport] = empty.supporting[T] def psi: TableBuilder = - supporting[ProgramAssociationTable]. - supporting[ProgramMapTable]. - supporting[ConditionalAccessTable] + supporting[ProgramAssociationTable] + .supporting[ProgramMapTable] + .supporting[ConditionalAccessTable] } trait TableSupport[T <: Table] { @@ -67,7 +68,7 @@ trait TableSupport[T <: Table] { object TableSupport { - def singleton[A <: Section with Table : reflect.ClassTag](tableId: Int): TableSupport[A] = { + def singleton[A <: Section with Table: reflect.ClassTag](tableId: Int): TableSupport[A] = { val tid = tableId new TableSupport[A] { def tableId = tid diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala index 8f1dee002f..0757e94314 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamEvent.scala @@ -30,7 +30,7 @@ package psi import cats.data.Chain import scodec.bits.BitVector -import psi.{ Table => TableMessage } +import psi.{Table => TableMessage} abstract class TransportStreamEvent @@ -43,27 +43,33 @@ object TransportStreamEvent { def pes(pid: Pid, pes: PesPacket): TransportStreamEvent = Pes(pid, pes) def table(pid: Pid, table: TableMessage): TransportStreamEvent = Table(pid, table) - def scrambledPayload(pid: Pid, content: BitVector): TransportStreamEvent = ScrambledPayload(pid, content) + def scrambledPayload(pid: Pid, content: BitVector): TransportStreamEvent = + ScrambledPayload(pid, content) def metadata[A](md: A): TransportStreamEvent = Metadata(None, md) def metadata[A](pid: Pid, md: A): TransportStreamEvent = Metadata(Some(pid), md) def error(pid: Pid, e: MpegError): TransportStreamEvent = Error(Some(pid), e) def error(pid: Option[Pid], e: MpegError): TransportStreamEvent = Error(pid, e) private def sectionsToTables[S]( - group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], tableBuilder: TableBuilder - ): Scan[(Map[Pid, S], TransportStreamIndex), PidStamped[Either[MpegError, Section]], TransportStreamEvent] = { + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], + tableBuilder: TableBuilder + ): Scan[(Map[Pid, S], TransportStreamIndex), PidStamped[ + Either[MpegError, Section] + ], TransportStreamEvent] = { import MpegError._ val sectionsToTablesForPid: Scan[S, Section, Either[MpegError, TableMessage]] = group.map { - case Left(e) => Left(e) + case Left(e) => Left(e) case Right(gs) => tableBuilder.build(gs) } - val sectionsToTables: Scan[Map[Pid, S], PidStamped[Either[MpegError, Section]], PidStamped[Either[MpegError, TableMessage]]] = - Scan(Map.empty[Pid, S])({ - case (state, PidStamped(pid, e)) => + val sectionsToTables: Scan[Map[Pid, S], PidStamped[Either[MpegError, Section]], PidStamped[ + Either[MpegError, TableMessage] + ]] = + Scan(Map.empty[Pid, S])( + { case (state, PidStamped(pid, e)) => e match { case Right(section) => val groupingState = state.getOrElse(pid, group.initial) @@ -72,35 +78,50 @@ object TransportStreamEvent { case Left(err) => (state, Chunk.singleton(PidStamped(pid, Left(err)))) } - }, { state => - Chunk.concat(state.foldLeft(Chain.empty[Chunk[PidStamped[Either[MpegError, TableMessage]]]]) { case (acc, (pid, gs)) => - acc :+ sectionsToTablesForPid.onComplete(gs).map(PidStamped(pid, _)) - }.toList) - }) + }, + state => + Chunk.concat( + state + .foldLeft(Chain.empty[Chunk[PidStamped[Either[MpegError, TableMessage]]]]) { + case (acc, (pid, gs)) => + acc :+ sectionsToTablesForPid.onComplete(gs).map(PidStamped(pid, _)) + } + .toList + ) + ) - sectionsToTables.andThen(PidStamped.preserve(passErrors(TransportStreamIndex.build))).map { case PidStamped(pid, value) => - value match { - case Left(e) => error(pid, e) - case Right(Left(tsi)) => metadata(tsi) - case Right(Right(tbl)) => table(pid, tbl) - } + sectionsToTables.andThen(PidStamped.preserve(passErrors(TransportStreamIndex.build))).map { + case PidStamped(pid, value) => + value match { + case Left(e) => error(pid, e) + case Right(Left(tsi)) => metadata(tsi) + case Right(Right(tbl)) => table(pid, tbl) + } } } def fromPacketStream[S]( - sectionCodec: SectionCodec, - group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], - tableBuilder: TableBuilder - ): Scan[((Map[Pid, ContinuityCounter], Demultiplexer.State), (Map[Pid, S], TransportStreamIndex)), Packet, TransportStreamEvent] = { - val demuxed = { - Demultiplexer.demultiplex(sectionCodec).andThen( - sectionsToTables(group, tableBuilder).semipass[PidStamped[Either[DemultiplexerError, Demultiplexer.Result]], TransportStreamEvent]( - { - case PidStamped(pid, Right(Demultiplexer.SectionResult(section))) => Right(PidStamped(pid, Right(section))) + sectionCodec: SectionCodec, + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], + tableBuilder: TableBuilder + ): Scan[ + ((Map[Pid, ContinuityCounter], Demultiplexer.State), (Map[Pid, S], TransportStreamIndex)), + Packet, + TransportStreamEvent + ] = { + val demuxed = + Demultiplexer + .demultiplex(sectionCodec) + .andThen( + sectionsToTables(group, tableBuilder).semipass[PidStamped[ + Either[DemultiplexerError, Demultiplexer.Result] + ], TransportStreamEvent]({ + case PidStamped(pid, Right(Demultiplexer.SectionResult(section))) => + Right(PidStamped(pid, Right(section))) case PidStamped(pid, Right(Demultiplexer.PesPacketResult(p))) => Left(pes(pid, p)) - case PidStamped(pid, Left(e)) => Right(PidStamped(pid, Left(e.toMpegError))) - })) - } + case PidStamped(pid, Left(e)) => Right(PidStamped(pid, Left(e.toMpegError))) + }) + ) demuxed.semipass[Packet, TransportStreamEvent]({ case Packet(header, _, _, Some(payload)) if header.scramblingControl != 0 => Left(scrambledPayload(header.pid, payload)) @@ -110,8 +131,8 @@ object TransportStreamEvent { } def fromSectionStream[S]( - group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], - tableBuilder: TableBuilder + group: Scan[S, Section, Either[GroupingError, GroupedSections[Section]]], + tableBuilder: TableBuilder ): Scan[(Map[Pid, S], TransportStreamIndex), PidStamped[Section], TransportStreamEvent] = sectionsToTables(group, tableBuilder).contramap[PidStamped[Section]](_.map(Right(_))) } diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala index e1897bffa1..9b382a4393 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala @@ -36,7 +36,10 @@ sealed abstract class TransportStreamIndex { def pmt(prg: ProgramNumber): Either[LookupError, ProgramMapTable] - def programMapRecords(program: ProgramNumber, streamType: StreamType): Either[LookupError, List[ProgramMapRecord]] = + def programMapRecords( + program: ProgramNumber, + streamType: StreamType + ): Either[LookupError, List[ProgramMapRecord]] = for { p <- pat.toRight(LookupError.MissingProgramAssociation) _ <- p.programByPid.get(program).toRight(LookupError.UnknownProgram) @@ -44,15 +47,17 @@ sealed abstract class TransportStreamIndex { pmrs <- q.componentStreamMapping.get(streamType).toRight(LookupError.UnknownStreamType) } yield pmrs - def programManRecord(program: ProgramNumber, streamType: StreamType): Either[LookupError, ProgramMapRecord] = - programMapRecords(program, streamType).map { _.head } + def programManRecord( + program: ProgramNumber, + streamType: StreamType + ): Either[LookupError, ProgramMapRecord] = + programMapRecords(program, streamType).map(_.head) def withPat(pat: ProgramAssociationTable): TransportStreamIndex def withPmt(pmt: ProgramMapTable): TransportStreamIndex def withCat(cat: ConditionalAccessTable): TransportStreamIndex } - object TransportStreamIndex { sealed abstract class LookupError @@ -64,9 +69,9 @@ object TransportStreamIndex { } private case class DefaultTransportStreamIndex( - pat: Option[ProgramAssociationTable], - cat: Option[ConditionalAccessTable], - pmts: Map[ProgramNumber, ProgramMapTable] + pat: Option[ProgramAssociationTable], + cat: Option[ConditionalAccessTable], + pmts: Map[ProgramNumber, ProgramMapTable] ) extends TransportStreamIndex { def pmt(prg: ProgramNumber): Either[LookupError, ProgramMapTable] = @@ -77,9 +82,8 @@ object TransportStreamIndex { copy(pat = Some(pat), pmts = pmts.view.filterKeys(programs).toMap) } - def withPmt(pmt: ProgramMapTable): TransportStreamIndex = { + def withPmt(pmt: ProgramMapTable): TransportStreamIndex = copy(pmts = pmts + (pmt.programNumber -> pmt)) - } def withCat(cat: ConditionalAccessTable): TransportStreamIndex = copy(cat = Some(cat)) @@ -87,22 +91,23 @@ object TransportStreamIndex { def empty: TransportStreamIndex = DefaultTransportStreamIndex(None, None, Map.empty) - def build: Scan[TransportStreamIndex, Table, Either[TransportStreamIndex, Table]] = Scan.stateful(empty) { (tsi, section) => - val updatedTsi = section match { - case pat: ProgramAssociationTable => - Some(tsi.withPat(pat)) - case pmt: ProgramMapTable => - Some(tsi.withPmt(pmt)) - case cat: ConditionalAccessTable => - Some(tsi.withCat(cat)) - case other => None + def build: Scan[TransportStreamIndex, Table, Either[TransportStreamIndex, Table]] = + Scan.stateful(empty) { (tsi, section) => + val updatedTsi = section match { + case pat: ProgramAssociationTable => + Some(tsi.withPat(pat)) + case pmt: ProgramMapTable => + Some(tsi.withPmt(pmt)) + case cat: ConditionalAccessTable => + Some(tsi.withCat(cat)) + case other => None + } + val out = updatedTsi match { + case Some(newTsi) if newTsi != tsi => + Chunk(Right(section), Left(newTsi)) + case _ => + Chunk(Right(section)) + } + (updatedTsi.getOrElse(tsi), out) } - val out = updatedTsi match { - case Some(newTsi) if newTsi != tsi => - Chunk(Right(section), Left(newTsi)) - case _ => - Chunk(Right(section)) - } - (updatedTsi.getOrElse(tsi), out) - } } diff --git a/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala b/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala index eedc7d1526..0a068ba4f3 100644 --- a/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala +++ b/protocols/shared/src/test/scala/fs2/protocols/PcapMpegExample.scala @@ -30,18 +30,21 @@ import fs2.interop.scodec.StreamDecoder import fs2.io.file.{Files, Path} import fs2.timeseries.TimeStamped -import pcap.{ CaptureFile, LinkType } +import pcap.{CaptureFile, LinkType} -/** - * Example of decoding a PCAP file that contains: - * - captured ethernet frames - * - of IPv4 packets - * - of UDP datagrams - * - containing MPEG transport stream packets - */ +/** Example of decoding a PCAP file that contains: + * - captured ethernet frames + * - of IPv4 packets + * - of UDP datagrams + * - containing MPEG transport stream packets + */ object PcapMpegExample extends IOApp.Simple { - case class CapturedPacket(source: SocketAddress[Ipv4Address], destination: SocketAddress[Ipv4Address], packet: mpeg.transport.Packet) + case class CapturedPacket( + source: SocketAddress[Ipv4Address], + destination: SocketAddress[Ipv4Address], + packet: mpeg.transport.Packet + ) val run: IO[Unit] = { val decoder: StreamDecoder[TimeStamped[CapturedPacket]] = CaptureFile.payloadStreamDecoderPF { @@ -54,12 +57,14 @@ object PcapMpegExample extends IOApp.Simple { CapturedPacket( SocketAddress(ipHeader.sourceIp, udpDatagram.sourcePort), SocketAddress(ipHeader.destinationIp, udpDatagram.destinationPort), - p) + p + ) } } yield packets } - Files[IO].readAll(Path("path/to/pcap")) + Files[IO] + .readAll(Path("path/to/pcap")) .through(decoder.toPipeByte) .map(_.toString) .foreach(IO.println) diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala index a5049f945a..c4fe25a42d 100644 --- a/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/DescriptorTest.scala @@ -32,7 +32,7 @@ import Descriptor._ import scodec._ import scodec.bits._ -import org.scalacheck.{ Arbitrary, Gen, Prop } +import org.scalacheck.{Arbitrary, Gen, Prop} class DescriptorTest extends Fs2Suite { import DescriptorTestData._ @@ -42,7 +42,7 @@ class DescriptorTest extends Fs2Suite { .withMinSuccessfulTests(1000) test("support relevant descriptors which handles decoding and encoding for valid values") { - Prop.forAll { (d: Descriptor) => roundtrip(Descriptor.codec, d) } + Prop.forAll((d: Descriptor) => roundtrip(Descriptor.codec, d)) } private def roundtrip[A](codec: Codec[A], value: A) = { @@ -71,7 +71,14 @@ object DescriptorTestData { constrainedParameter <- Gen.oneOf(true, false) stillPictureFlag <- Gen.oneOf(true, false) mpeg1Only <- genMpeg1Only(mpeg1OnlyFlag) - } yield VideoStreamDescriptor(multipleFrameRateFlag, frameRateCode, mpeg1OnlyFlag, constrainedParameter, stillPictureFlag, mpeg1Only) + } yield VideoStreamDescriptor( + multipleFrameRateFlag, + frameRateCode, + mpeg1OnlyFlag, + constrainedParameter, + stillPictureFlag, + mpeg1Only + ) lazy val genAudioStreamDescriptor: Gen[AudioStreamDescriptor] = for { freeFormatFlag <- Gen.oneOf(true, false) @@ -89,27 +96,38 @@ object DescriptorTestData { HierarchyType.PrivateStream, HierarchyType.MultiViewProfile, HierarchyType.Reserved(0), - HierarchyType.BaseLayer) + HierarchyType.BaseLayer + ) lazy val genHierarchyDescriptor: Gen[HierarchyDescriptor] = for { hierarchyType <- genHierarchyType hierarchyLayerIndex <- Gen.chooseNum(0, 63) hierarchyEmbeddedLayerIndex <- Gen.chooseNum(0, 63) hierarchyChannel <- Gen.chooseNum(0, 63) - } yield HierarchyDescriptor(hierarchyType, hierarchyLayerIndex, hierarchyEmbeddedLayerIndex, hierarchyChannel) + } yield HierarchyDescriptor( + hierarchyType, + hierarchyLayerIndex, + hierarchyEmbeddedLayerIndex, + hierarchyChannel + ) lazy val genRegistrationDescriptor: Gen[RegistrationDescriptor] = for { length <- Gen.chooseNum(4, 255) formatIdentifier <- Gen.listOfN(4, Gen.chooseNum(0, 255)) additionalIdentificationInfo <- Gen.listOfN(length - 4, Gen.chooseNum(0, 255)) - } yield RegistrationDescriptor(ByteVector(formatIdentifier: _*), ByteVector(additionalIdentificationInfo: _*)) + } yield RegistrationDescriptor( + ByteVector(formatIdentifier: _*), + ByteVector(additionalIdentificationInfo: _*) + ) lazy val genDataStreamAlignmentDescriptor: Gen[DataStreamAlignmentDescriptor] = for { - alignmentType <- Gen.oneOf(AlignmentType.Reserved(0), + alignmentType <- Gen.oneOf( + AlignmentType.Reserved(0), AlignmentType.SliceOrVideoAccessUnit, AlignmentType.VideoAccessUnit, AlignmentType.GopOrSeq, - AlignmentType.Seq) + AlignmentType.Seq + ) } yield DataStreamAlignmentDescriptor(alignmentType) lazy val genTargetBackgroundGridDescriptor: Gen[TargetBackgroundGridDescriptor] = for { @@ -132,8 +150,14 @@ object DescriptorTestData { } yield CADescriptor(caSystemId, Pid(caPid), ByteVector(privateData: _*)) lazy val genLanguageField: Gen[LanguageField] = for { - iso639LanguageCode <- Gen.listOfN(3, Gen.alphaChar) - audioType <- Gen.oneOf(AudioType.Undefined, AudioType.CleanEffects, AudioType.HearingImpaired, AudioType.VisualImpairedCommentary, AudioType.Reserved(4)) + iso639LanguageCode <- Gen.listOfN(3, Gen.alphaChar) + audioType <- Gen.oneOf( + AudioType.Undefined, + AudioType.CleanEffects, + AudioType.HearingImpaired, + AudioType.VisualImpairedCommentary, + AudioType.Reserved(4) + ) } yield LanguageField(iso639LanguageCode.mkString, audioType) lazy val genIso639LanguageDescriptor: Gen[Iso639LanguageDescriptor] = for { @@ -143,22 +167,34 @@ object DescriptorTestData { } yield Iso639LanguageDescriptor(languageFields.toVector) lazy val genSystemClockDescriptor: Gen[SystemClockDescriptor] = for { - externalClockReferenceIndicator <- Gen.oneOf(true, false) - clockAccuracyInteger <- Gen.oneOf(0, 63) - clockAccuracyExponent <- Gen.oneOf(0, 7) - } yield SystemClockDescriptor(externalClockReferenceIndicator, clockAccuracyInteger, clockAccuracyExponent) - - lazy val genMultiplexBufferUtilizationDescriptor: Gen[MultiplexBufferUtilizationDescriptor] = for { - boundValidFlag <- Gen.oneOf(true, false) - ltwOffsetLowerBound <- Gen.oneOf(0, 32767) - ltwOffsetUpperBound <- Gen.oneOf(0, 16383) - } yield MultiplexBufferUtilizationDescriptor(boundValidFlag, ltwOffsetLowerBound, ltwOffsetUpperBound) + externalClockReferenceIndicator <- Gen.oneOf(true, false) + clockAccuracyInteger <- Gen.oneOf(0, 63) + clockAccuracyExponent <- Gen.oneOf(0, 7) + } yield SystemClockDescriptor( + externalClockReferenceIndicator, + clockAccuracyInteger, + clockAccuracyExponent + ) + + lazy val genMultiplexBufferUtilizationDescriptor: Gen[MultiplexBufferUtilizationDescriptor] = + for { + boundValidFlag <- Gen.oneOf(true, false) + ltwOffsetLowerBound <- Gen.oneOf(0, 32767) + ltwOffsetUpperBound <- Gen.oneOf(0, 16383) + } yield MultiplexBufferUtilizationDescriptor( + boundValidFlag, + ltwOffsetLowerBound, + ltwOffsetUpperBound + ) lazy val genCopyrightDescriptor: Gen[CopyrightDescriptor] = for { length <- Gen.chooseNum(4, 255) copyrightIdentifier <- Gen.listOfN(4, Gen.chooseNum(0, 255)) additionalCopyrightInfo <- Gen.listOfN(length - 4, Gen.chooseNum(0, 255)) - } yield CopyrightDescriptor(ByteVector(copyrightIdentifier: _*), ByteVector(additionalCopyrightInfo: _*)) + } yield CopyrightDescriptor( + ByteVector(copyrightIdentifier: _*), + ByteVector(additionalCopyrightInfo: _*) + ) lazy val genMaximumBitrateDescriptor: Gen[MaximumBitrateDescriptor] = for { maximumBitrate <- Gen.chooseNum(0, 4194303) @@ -177,16 +213,20 @@ object DescriptorTestData { for { leakValidFlag <- Gen.oneOf(true, false) } yield StdDescriptor(leakValidFlag) lazy val genIbpDescriptor: Gen[IbpDescriptor] = for { - closedGopFlag <- Gen.oneOf(true, false) - identicalGopFlag <- Gen.oneOf(true, false) - maxGopLength <- Gen.chooseNum(0, 16383) + closedGopFlag <- Gen.oneOf(true, false) + identicalGopFlag <- Gen.oneOf(true, false) + maxGopLength <- Gen.chooseNum(0, 16383) } yield IbpDescriptor(closedGopFlag, identicalGopFlag, maxGopLength) lazy val genMpeg4VideoDescriptor: Gen[Mpeg4VideoDescriptor] = - for { mpeg4VisualProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4VideoDescriptor(mpeg4VisualProfileAndLevel.toByte) + for { mpeg4VisualProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4VideoDescriptor( + mpeg4VisualProfileAndLevel.toByte + ) lazy val genMpeg4AudioDescriptor: Gen[Mpeg4AudioDescriptor] = - for { mpeg4AudioProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4AudioDescriptor(mpeg4AudioProfileAndLevel.toByte) + for { mpeg4AudioProfileAndLevel <- Gen.chooseNum(0, 255) } yield Mpeg4AudioDescriptor( + mpeg4AudioProfileAndLevel.toByte + ) lazy val genIodDescriptor: Gen[IodDescriptor] = for { scopeOfIodLabel <- Gen.chooseNum(0, 255) @@ -251,7 +291,8 @@ object DescriptorTestData { genExternalEsIdDescriptor, genMuxCodeDescriptor, genFmxBufferSizeDescriptor, - genMultiplexBufferDescriptor) + genMultiplexBufferDescriptor + ) lazy val genUnknownDescriptor: Gen[UnknownDescriptor] = for { tag <- Gen.chooseNum(36, 255) @@ -259,10 +300,11 @@ object DescriptorTestData { data <- Gen.listOfN(length, Gen.chooseNum(0, 255)) } yield UnknownDescriptor(tag, length, ByteVector(data: _*)) - lazy val genDescriptor: Gen[Descriptor] = Gen.oneOf(genKnownDescriptor, genUnknownDescriptor).map { - case known: KnownDescriptor => Right(known) - case unknown: UnknownDescriptor => Left(unknown) - } + lazy val genDescriptor: Gen[Descriptor] = + Gen.oneOf(genKnownDescriptor, genUnknownDescriptor).map { + case known: KnownDescriptor => Right(known) + case unknown: UnknownDescriptor => Left(unknown) + } implicit lazy val arbitraryDescriptor: Arbitrary[Descriptor] = Arbitrary(genDescriptor) } diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala index 09b513f714..0ff48e0ebc 100644 --- a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/PacketTest.scala @@ -37,18 +37,36 @@ class PacketTest extends Fs2Suite { val c = ByteVector.fill(10)(2).bits val sections = Vector(a, b, c) val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), sections) - assertEquals(packets, Vector(Packet.payload(Pid(0), ContinuityCounter(0), Some(0), a ++ b ++ c ++ BitVector.fill((183 * 8) - a.size - b.size - c.size)(true)))) + assertEquals( + packets, + Vector( + Packet.payload( + Pid(0), + ContinuityCounter(0), + Some(0), + a ++ b ++ c ++ BitVector.fill((183 * 8) - a.size - b.size - c.size)(true) + ) + ) + ) } test("support packetizing multiple sections across multiple packets") { - val sections = (0 until 256).map { x => ByteVector.fill(10)(x).bits }.toVector + val sections = (0 until 256).map(x => ByteVector.fill(10)(x).bits).toVector val data = sections.foldLeft(BitVector.empty)(_ ++ _) val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), sections) packets.zipWithIndex.foreach { case (packet, idx) => val payloadOffset = if (idx == 0) 0 else 10 * ((idx * 183) / 10 + 1) - (idx * 183) val offset = 183L * 8 * idx - assertEquals(packets(idx), Packet.payload(Pid(0), ContinuityCounter(idx), Some(payloadOffset), data.drop(offset).take(183 * 8))) + assertEquals( + packets(idx), + Packet.payload( + Pid(0), + ContinuityCounter(idx), + Some(payloadOffset), + data.drop(offset).take(183 * 8) + ) + ) } } } diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala index e376157c8c..b335d5d0c9 100644 --- a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/GroupingTest.scala @@ -32,24 +32,29 @@ class GroupingTest extends Fs2Suite { val des = GroupedSections.groupExtendedSections[ProgramAssociationSection].toPipe[Pure] val pat3: ProgramAssociationTable = { - val pidMap = (0 until ProgramAssociationTable.MaxProgramsPerSection * 3).map { n => ProgramNumber(n) -> Pid(n) }.toMap + val pidMap = (0 until ProgramAssociationTable.MaxProgramsPerSection * 3).map { n => + ProgramNumber(n) -> Pid(n) + }.toMap ProgramAssociationTable(TransportStreamId(5), 1, true, pidMap) } test("handles stream of a specific table id and extension") { val p = Stream.emits(pat3.toSections.list).through(des).map { case Right(sections) => ProgramAssociationTable.fromSections(sections) - case l @ Left(_) => l + case l @ Left(_) => l } assertEquals(p.toList, List(Right(pat3))) } test("handles stream containing sections for the same table id but differing extension ids") { val patA = pat3 - val patB = pat3.copy(tsid = TransportStreamId(pat3.tsid.value + 1), programByPid = pat3.programByPid.map { case (prg, Pid(n)) => prg -> Pid(n + 1)} ) + val patB = pat3.copy( + tsid = TransportStreamId(pat3.tsid.value + 1), + programByPid = pat3.programByPid.map { case (prg, Pid(n)) => prg -> Pid(n + 1) } + ) - val sections = Stream.emits(patA.toSections.list) interleave Stream.emits(patB.toSections.list) - val p = sections.through(des).map { _.flatMap(ProgramAssociationTable.fromSections) } + val sections = Stream.emits(patA.toSections.list).interleave(Stream.emits(patB.toSections.list)) + val p = sections.through(des).map(_.flatMap(ProgramAssociationTable.fromSections)) assertEquals(p.toList, List(Right(patA), Right(patB))) } } diff --git a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala index d85140b94b..af3fe4146a 100644 --- a/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala +++ b/protocols/shared/src/test/scala/fs2/protocols/mpeg/transport/psi/SectionCodecTest.scala @@ -37,38 +37,78 @@ class SectionCodecTest extends Fs2Suite { val sectionCodec = SectionCodec.supporting[ProgramAssociationSection] - test("handles case where section starts at beginning of packet and is fully contained within packet") { - val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + test( + "handles case where section starts at beginning of packet and is fully contained within packet" + ) { + val pas = ProgramAssociationTable + .toSections( + ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2))) + ) + .head val pasEnc = sectionCodec.encode(pas).require val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), pasEnc) - val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe - assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + val p = Stream.emit(packet).through(Demultiplexer.demultiplex(sectionCodec).toPipe) + assertEquals( + p.toList, + List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s)))) + ) } test("handles case where section starts at beginning of packet and spans multiple packets") { - val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, - (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) - yield ProgramNumber(i) -> Pid(i)).toMap - )).head + val pas = ProgramAssociationTable + .toSections( + ProgramAssociationTable( + TransportStreamId(1), + 15, + true, + (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) + yield ProgramNumber(i) -> Pid(i)).toMap + ) + ) + .head val pasEnc = sectionCodec.encode(pas).require val packets = Packet.packetize(Pid(0), ContinuityCounter(0), pasEnc) - val p = Stream.emits(packets) through Demultiplexer.demultiplex(sectionCodec).toPipe - assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + val p = Stream.emits(packets).through(Demultiplexer.demultiplex(sectionCodec).toPipe) + assertEquals( + p.toList, + List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s)))) + ) } test("checks packet continuity") { - val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, - (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) - yield ProgramNumber(i) -> Pid(i)).toMap - )).head + val pas = ProgramAssociationTable + .toSections( + ProgramAssociationTable( + TransportStreamId(1), + 15, + true, + (for (i <- 0 until ProgramAssociationTable.MaxProgramsPerSection) + yield ProgramNumber(i) -> Pid(i)).toMap + ) + ) + .head val pasEnc = sectionCodec.encode(pas).require val packets = Packet.packetize(Pid(0), ContinuityCounter(1), pasEnc) - val withDiscontinuity = packets.updated(0, packets.head.copy(header = packets.head.header.copy(continuityCounter = ContinuityCounter(15)))) + val withDiscontinuity = packets.updated( + 0, + packets.head.copy(header = + packets.head.header.copy(continuityCounter = ContinuityCounter(15)) + ) + ) - val p = Stream.emits(withDiscontinuity) through Demultiplexer.demultiplex(sectionCodec).toPipe - assertEquals(p.toList, List(PidStamped(Pid(0), Left(DemultiplexerError.Discontinuity(ContinuityCounter(15), ContinuityCounter(2), 1))))) + val p = + Stream.emits(withDiscontinuity).through(Demultiplexer.demultiplex(sectionCodec).toPipe) + assertEquals( + p.toList, + List( + PidStamped( + Pid(0), + Left(DemultiplexerError.Discontinuity(ContinuityCounter(15), ContinuityCounter(2), 1)) + ) + ) + ) } test("upon decoding failure of a section, remaining sections in packet are decoded") { @@ -76,41 +116,84 @@ class SectionCodecTest extends Fs2Suite { val sections = List(SmallSection(0), SmallSection(1)) implicit val sfc: SectionFragmentCodec[SmallSection] = - SectionFragmentCodec.nonExtended[SmallSection, Int](0, h => (constant(bin"0") ~> uint(7)), (p, i) => SmallSection(i), ss => (bin"010", ss.x)) + SectionFragmentCodec.nonExtended[SmallSection, Int]( + 0, + h => (constant(bin"0") ~> uint(7)), + (p, i) => SmallSection(i), + ss => (bin"010", ss.x) + ) val sc = SectionCodec.supporting[SmallSection] - val encodedSections = sections.toVector map { s => sc.encode(s).require } + val encodedSections = sections.toVector.map(s => sc.encode(s).require) val ss0 = encodedSections(0).bytes val ss1 = encodedSections(1).bytes - val indexOfInt = ss0.toIndexedSeq.zipWithIndex.find { case (x, idx) => ss1(idx.toLong) != x }.map { case (x, idx) => idx }.get + val indexOfInt = ss0.toIndexedSeq.zipWithIndex + .find { case (x, idx) => ss1(idx.toLong) != x } + .map { case (x, idx) => idx } + .get val ss255 = ss0.update(indexOfInt.toLong, 255.toByte) - val packets = Packet.packetizeMany(Pid(0), ContinuityCounter(0), ss255.bits +: encodedSections) - val p = Stream.emits(packets) through Demultiplexer.demultiplex(sc).toPipe - - assertEquals(p.toList, - PidStamped(Pid(0), Left(DemultiplexerError.Decoding(hex"002001ff".bits, Err("expected constant BitVector(1 bits, 0x0) but got BitVector(1 bits, 0x8)")))) +: - sections.map { x => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(x))) } + val packets = + Packet.packetizeMany(Pid(0), ContinuityCounter(0), ss255.bits +: encodedSections) + val p = Stream.emits(packets).through(Demultiplexer.demultiplex(sc).toPipe) + + assertEquals( + p.toList, + PidStamped( + Pid(0), + Left( + DemultiplexerError.Decoding( + hex"002001ff".bits, + Err("expected constant BitVector(1 bits, 0x0) but got BitVector(1 bits, 0x8)") + ) + ) + ) +: + sections.map(x => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(x)))) ) } test("reports invalid CRC") { - val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + val pas = ProgramAssociationTable + .toSections( + ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2))) + ) + .head val pasEnc = sectionCodec.encode(pas).require val corruptedSection = pasEnc.dropRight(32) ++ (~pasEnc.takeRight(32)) val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), corruptedSection) - val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe - assertEquals(p.toList, List(PidStamped(Pid(0), Left(DemultiplexerError.Decoding(corruptedSection, Err("CRC mismatch: calculated 18564404 does not equal -18564405")))))) + val p = Stream.emit(packet).through(Demultiplexer.demultiplex(sectionCodec).toPipe) + assertEquals( + p.toList, + List( + PidStamped( + Pid(0), + Left( + DemultiplexerError.Decoding( + corruptedSection, + Err("CRC mismatch: calculated 18564404 does not equal -18564405") + ) + ) + ) + ) + ) } test("does not report invalid CRC when verifyCrc is disabled") { - val sectionCodec = SectionCodec.psi.disableCrcVerification.supporting[ProgramAssociationSection] - val pas = ProgramAssociationTable.toSections(ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2)))).head + val sectionCodec = + SectionCodec.psi.disableCrcVerification.supporting[ProgramAssociationSection] + val pas = ProgramAssociationTable + .toSections( + ProgramAssociationTable(TransportStreamId(1), 15, true, Map(ProgramNumber(1) -> Pid(2))) + ) + .head val pasEnc = sectionCodec.encode(pas).require val corruptedSection = pasEnc.dropRight(32) ++ (~pasEnc.dropRight(32)) val packet = Packet.payload(Pid(0), ContinuityCounter(0), Some(0), corruptedSection) - val p = Stream.emit(packet) through Demultiplexer.demultiplex(sectionCodec).toPipe - assertEquals(p.toList, List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s))))) + val p = Stream.emit(packet).through(Demultiplexer.demultiplex(sectionCodec).toPipe) + assertEquals( + p.toList, + List(pas).map(s => PidStamped(Pid(0), Right(Demultiplexer.SectionResult(s)))) + ) } } } From ad742e924928ab4856647b195604add525524af6 Mon Sep 17 00:00:00 2001 From: mpilquist Date: Fri, 22 Oct 2021 14:58:01 -0400 Subject: [PATCH 28/33] Fix 2.x compilation --- .../main/scala/fs2/protocols/mpeg/PesPacketHeader.scala | 9 +++++---- .../fs2/protocols/mpeg/transport/AdaptationField.scala | 5 +++-- .../mpeg/transport/psi/ProgramAssociationTable.scala | 4 +--- .../protocols/mpeg/transport/psi/ProgramMapTable.scala | 4 ++-- .../fs2/protocols/mpeg/transport/psi/SectionCodec.scala | 2 +- .../fs2/protocols/mpeg/transport/psi/TableBuilder.scala | 6 +++--- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala index 91e432b2a2..0656ded654 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/PesPacketHeader.scala @@ -26,6 +26,7 @@ package fs2.protocols.mpeg import scodec.bits._ import scodec.Codec import scodec.codecs._ +import scodec.compat._ sealed abstract class PesScramblingControl object PesScramblingControl { @@ -151,10 +152,10 @@ object PesPacketHeader { private def tsCodec(prefix: BitVector) = (constant(prefix) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker).dropUnits .xmap[Long]( - { case (a, b, c) => (a ++ b ++ c).toLong() }, + { case a *: b *: c *: EmptyTuple => (a ++ b ++ c).toLong() }, l => { val b = BitVector.fromLong(l).drop(31) - (b.take(3), b.drop(3).take(15), b.drop(18)) + b.take(3) *: b.drop(3).take(15) *: b.drop(18) *: EmptyTuple } ) @@ -162,7 +163,7 @@ object PesPacketHeader { (ignore(2) :: bits(3) :: marker :: bits(15) :: marker :: bits(15) :: marker :: uint( 9 ) :: marker).dropUnits.xmap[Long]( - { case (a, b, c, ext) => + { case a *: b *: c *: ext *: EmptyTuple => val base = (a ++ b ++ c).toLong() base * 300 + ext }, @@ -170,7 +171,7 @@ object PesPacketHeader { val base = (l / 300) % (2L << 32) val b = BitVector.fromLong(base).drop(31) val ext = (l % 300).toInt - (b.take(3), b.drop(3).take(15), b.drop(18), ext) + b.take(3) *: b.drop(3).take(15) *: b.drop(18) *: ext *: EmptyTuple } ) diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala index 39efe62b12..4d53804bb1 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/AdaptationField.scala @@ -27,6 +27,7 @@ package transport import scodec.{Attempt, Codec, DecodeResult, SizeBound} import scodec.bits.BitVector import scodec.codecs._ +import scodec.compat._ /** Partial modelling of the adaptation field. * The field extension, if present, is ignored upon decoding. @@ -57,14 +58,14 @@ object AdaptationField { private val pcrCodec: Codec[Clock27MHz] = ((ulong(33) <~ ignore(6)) :: uint(9)).xmap[Clock27MHz]( - { case (base, ext) => + { case base *: ext *: EmptyTuple => Clock27MHz(base * 300 + ext) }, { clock => val value = clock.value val base = value / 300 val ext = (value % 300).toInt - (base, ext) + base *: ext *: EmptyTuple } ) private val transportPrivateData: Codec[BitVector] = variableSizeBits(uint8, bits) diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala index f786257113..f4f895cba9 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala @@ -115,9 +115,7 @@ object ProgramAssociationSection { private val fragmentCodec: Codec[Fragment] = vector { - ("program_number" | Codec[ProgramNumber]) :: - (reserved(3) ~> - ("pid" | Codec[Pid])) + (("program_number" | Codec[ProgramNumber]) :: (reserved(3) ~> ("pid" | Codec[Pid]))).as[(ProgramNumber, Pid)] } implicit val sectionFragmentCodec: SectionFragmentCodec[ProgramAssociationSection] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala index e8c7c726a8..5aaa2c4d9f 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala @@ -111,9 +111,9 @@ object ProgramMapSection { ("pcr_pid" | pid) :: ("program_info_descriptors" | descriptors) :: vector { - ("stream_type" | uint8.as[StreamType]) :: programMapRecord + (("stream_type" | uint8.as[StreamType]) :: programMapRecord).as[(StreamType, ProgramMapRecord)] } - } + }.as[Fragment] implicit val sectionSubCodec: SectionFragmentCodec[ProgramMapSection] = SectionFragmentCodec.psi[ProgramMapSection, Fragment]( diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala index e71784ab23..cac67ac932 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/SectionCodec.scala @@ -78,7 +78,7 @@ class SectionCodec private ( def decode(bits: BitVector) = (for { header <- Codec[SectionHeader] - section <- Decoder(decodeSection(header)) + section <- Decoder(decodeSection(header)(_)) } yield section).decode(bits) def decodeSection(header: SectionHeader)(bits: BitVector): Attempt[DecodeResult[Section]] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala index 7545f1919e..b151a5f52b 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala @@ -68,14 +68,14 @@ trait TableSupport[T <: Table] { object TableSupport { - def singleton[A <: Section with Table: reflect.ClassTag](tableId: Int): TableSupport[A] = { + def singleton[A <: Section with Table](tableId: Int)(implicit ct: reflect.ClassTag[A]): TableSupport[A] = { val tid = tableId new TableSupport[A] { def tableId = tid def toTable(gs: GroupedSections[Section]) = - gs.narrow[A].toRight(s"Not a ${reflect.ClassTag[A]}").flatMap { sections => + gs.narrow[A].toRight(s"Not a $ct").flatMap { sections => if (sections.tail.isEmpty) Right(sections.head) - else Left(s"${reflect.ClassTag[A]} supports only 1 section but got ${sections.list.size}") + else Left(s"$ct supports only 1 section but got ${sections.list.size}") } def toSections(table: A) = GroupedSections(table) } From 5ce75bed273e0c6ca59138f566b6d800a5fdfef8 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 22 Oct 2021 20:28:29 -0400 Subject: [PATCH 29/33] Site --- build.sbt | 2 +- site/_sidebar.md | 1 + site/api-reference.md | 18 ++++++++++-------- site/getstarted/install.md | 10 +++++++--- site/scodec.md | 27 +++++++++++++++++++++++++++ 5 files changed, 46 insertions(+), 12 deletions(-) create mode 100644 site/scodec.md diff --git a/build.sbt b/build.sbt index 4b242a4abf..a444068118 100644 --- a/build.sbt +++ b/build.sbt @@ -407,7 +407,7 @@ lazy val microsite = project githubWorkflowArtifactUpload := false, fatalWarningsInCI := false ) - .dependsOn(coreJVM, io.jvm, reactiveStreams) + .dependsOn(coreJVM, io.jvm, reactiveStreams, scodec.jvm) .enablePlugins(MdocPlugin, NoPublishPlugin) ThisBuild / githubWorkflowBuildPostamble ++= List( diff --git a/site/_sidebar.md b/site/_sidebar.md index 7b0809cd3b..8ee22f5871 100644 --- a/site/_sidebar.md +++ b/site/_sidebar.md @@ -5,6 +5,7 @@ - [Guide](guide.md) - [Concurrency Primitives](concurrency-primitives.md) - [I/O](io.md) +- [Scodec](scodec.md) - [API Reference](api-reference.md) - [FAQ](faq.md) - [Documentation](documentation.md) diff --git a/site/api-reference.md b/site/api-reference.md index d5ffeadfea..571fbe317e 100644 --- a/site/api-reference.md +++ b/site/api-reference.md @@ -1,21 +1,23 @@ # API Reference -### FS2 3.1.1 (Cats Effect 3) +### FS2 3.2.0 (Cats Effect 3) * [fs2-core][core-api-v3] * [fs2-io][io-api-v3] * [fs2-reactive-streams][rx-api-v3] +* [fs2-scodec][scodec-api-v3] -[core-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-core_2.13/3.1.1/fs2-core_2.13-3.1.1-javadoc.jar/!/fs2/index.html -[io-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-io_2.13/3.1.1/fs2-io_2.13-3.1.1-javadoc.jar/!/fs2/io/index.html -[rx-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-reactive-streams_2.13/3.1.1/fs2-reactive-streams_2.13-3.1.1-javadoc.jar/!/fs2/interop/reactivestreams/index.html +[core-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-core_2.13/3.2.0/fs2-core_2.13-3.2.0-javadoc.jar/!/fs2/index.html +[io-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-io_2.13/3.2.0/fs2-io_2.13-3.2.0-javadoc.jar/!/fs2/io/index.html +[rx-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-reactive-streams_2.13/3.2.0/fs2-reactive-streams_2.13-3.2.0-javadoc.jar/!/fs2/interop/reactivestreams/index.html +[scodec-api-v3]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-scodec_2.13/3.2.0/fs2-scodec_2.13-3.2.0-javadoc.jar/!/fs2/interop/scodec/index.html -### FS2 2.5.9 (Cats Effect 2) +### FS2 2.5.10 (Cats Effect 2) * [fs2-core][core-api-v2] * [fs2-io][io-api-v2] * [fs2-reactive-streams][rx-api-v2] -[core-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-core_2.13/2.5.9/fs2-core_2.13-2.5.9-javadoc.jar/!/fs2/index.html -[io-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-io_2.13/2.5.9/fs2-io_2.13-2.5.9-javadoc.jar/!/fs2/io/index.html -[rx-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-reactive-streams_2.13/2.5.9/fs2-reactive-streams_2.13-2.5.9-javadoc.jar/!/fs2/interop/reactivestreams/index.html +[core-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-core_2.13/2.5.10/fs2-core_2.13-2.5.10-javadoc.jar/!/fs2/index.html +[io-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-io_2.13/2.5.10/fs2-io_2.13-2.5.10-javadoc.jar/!/fs2/io/index.html +[rx-api-v2]: https://oss.sonatype.org/service/local/repositories/releases/archive/co/fs2/fs2-reactive-streams_2.13/2.5.10/fs2-reactive-streams_2.13-2.5.10-javadoc.jar/!/fs2/interop/reactivestreams/index.html diff --git a/site/getstarted/install.md b/site/getstarted/install.md index 4141a7d0b0..f015b772f6 100644 --- a/site/getstarted/install.md +++ b/site/getstarted/install.md @@ -1,8 +1,8 @@ # Install -The latest version for Cats Effect 3 is `3.1.1`, which supports Cats Effect 3 and is cross built for Scala 2.12, 2.13, and 3.0. +The latest version for Cats Effect 3 is `3.2.0`, which supports Cats Effect 3 and is cross built for Scala 2.12, 2.13, and 3.0. -The latest version for Cats Effect 2 is `2.5.9`, which supports Cats Effect 2 and is similarly cross built for various Scala versions. +The latest version for Cats Effect 2 is `2.5.10`, which supports Cats Effect 2 and is similarly cross built for various Scala versions. ### Dependencies @@ -15,13 +15,17 @@ libraryDependencies += "co.fs2" %% "fs2-io" % "" // optional reactive streams interop libraryDependencies += "co.fs2" %% "fs2-reactive-streams" % "" + +// optional scodec interop +libraryDependencies += "co.fs2" %% "fs2-scodec" % "" ``` -The fs2-core as well as fs2-io libraries are also supported on Scala.js: +The fs2-core as well as fs2-io and fs2-scodec libraries are also supported on Scala.js: ``` libraryDependencies += "co.fs2" %%% "fs2-core" % "" libraryDependencies += "co.fs2" %%% "fs2-io" % "" // Node.js only +libraryDependencies += "co.fs2" %%% "fs2-scodec" % "" ``` Release notes for each release are available on [Github](https://github.com/typelevel/fs2/releases/). diff --git a/site/scodec.md b/site/scodec.md new file mode 100644 index 0000000000..bbfbea8d16 --- /dev/null +++ b/site/scodec.md @@ -0,0 +1,27 @@ +# Scodec + +The `fs2-scodec` library provides the ability to do streaming binary encoding and decoding, powered by [scodec](https://github.com/scodec/scodec). It was originally called [scodec-stream](https://github.com/scodec/scodec-stream) and released independently for many years before being imported in to fs2. + +```scala mdoc +import cats.effect.{IO, IOApp} +import scodec.bits._ +import scodec.codecs._ +import fs2.Stream +import fs2.interop.scodec._ +import fs2.io.file.{Files, Path} + +object Decode extends IOApp.Simple { + + def run = { + val frames: StreamDecoder[ByteVector] = + StreamDecoder.many(int32).flatMap { numBytes => StreamDecoder.once(bytes(numBytes)) } + + val filePath = Path("path/to/file") + + val s: Stream[IO, ByteVector] = + Files[IO].readAll(filePath).through(frames.toPipeByte) + + s.compile.count.flatMap(cnt => IO.println(s"Read $cnt frames.")) + } +} +``` \ No newline at end of file From 910feb34eea383d19e46313039fa64dd96670868 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 22 Oct 2021 20:38:31 -0400 Subject: [PATCH 30/33] Scalafmt --- .../mpeg/transport/psi/ProgramAssociationTable.scala | 3 ++- .../fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala | 3 ++- .../scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala | 4 +++- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala index f4f895cba9..c12a3b0f62 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramAssociationTable.scala @@ -115,7 +115,8 @@ object ProgramAssociationSection { private val fragmentCodec: Codec[Fragment] = vector { - (("program_number" | Codec[ProgramNumber]) :: (reserved(3) ~> ("pid" | Codec[Pid]))).as[(ProgramNumber, Pid)] + (("program_number" | Codec[ProgramNumber]) :: (reserved(3) ~> ("pid" | Codec[Pid]))) + .as[(ProgramNumber, Pid)] } implicit val sectionFragmentCodec: SectionFragmentCodec[ProgramAssociationSection] = diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala index 5aaa2c4d9f..bcefd1b70b 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/ProgramMapTable.scala @@ -111,7 +111,8 @@ object ProgramMapSection { ("pcr_pid" | pid) :: ("program_info_descriptors" | descriptors) :: vector { - (("stream_type" | uint8.as[StreamType]) :: programMapRecord).as[(StreamType, ProgramMapRecord)] + (("stream_type" | uint8.as[StreamType]) :: programMapRecord) + .as[(StreamType, ProgramMapRecord)] } }.as[Fragment] diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala index b151a5f52b..5c24a3eb08 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TableBuilder.scala @@ -68,7 +68,9 @@ trait TableSupport[T <: Table] { object TableSupport { - def singleton[A <: Section with Table](tableId: Int)(implicit ct: reflect.ClassTag[A]): TableSupport[A] = { + def singleton[A <: Section with Table]( + tableId: Int + )(implicit ct: reflect.ClassTag[A]): TableSupport[A] = { val tid = tableId new TableSupport[A] { def tableId = tid From 0441d8d11f774e1005a46b450f294b32adb1e6e9 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Fri, 22 Oct 2021 20:58:27 -0400 Subject: [PATCH 31/33] Fix 2.12 compilation --- .../fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala index 9b382a4393..7026346784 100644 --- a/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala +++ b/protocols/shared/src/main/scala/fs2/protocols/mpeg/transport/psi/TransportStreamIndex.scala @@ -79,7 +79,7 @@ object TransportStreamIndex { def withPat(pat: ProgramAssociationTable): TransportStreamIndex = { val programs = pat.programByPid.keys.toSet - copy(pat = Some(pat), pmts = pmts.view.filterKeys(programs).toMap) + copy(pat = Some(pat), pmts = pmts.view.filter { case (k, v) => programs(k) }.toMap) } def withPmt(pmt: ProgramMapTable): TransportStreamIndex = From 788482801288a52e88819907032cb6d4bcc2bc94 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sat, 23 Oct 2021 11:14:41 -0400 Subject: [PATCH 32/33] Add docs on TimeSeries and scodec --- .../scala/fs2/timeseries/TimeSeries.scala | 2 +- .../scala/fs2/timeseries/TimeStamped.scala | 6 +- site/_sidebar.md | 1 + site/scodec.md | 8 +- site/timeseries.md | 128 ++++++++++++++++++ 5 files changed, 141 insertions(+), 4 deletions(-) create mode 100644 site/timeseries.md diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala index 644b4b3c15..5f4d269a9c 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeSeries.scala @@ -87,7 +87,7 @@ object TimeSeries { private def timeTicks[F[_]: Temporal](tickPeriod: FiniteDuration): Stream[F, TimeStamped[Unit]] = Stream.awakeEvery[F](tickPeriod).map(_ => TimeStamped.unsafeNow(())) - /** Stream transducer that converts a stream of timestamped values with monotonically increasing timestamps in + /** Pipe that converts a stream of timestamped values with monotonically increasing timestamps in * to a stream of timestamped ticks or values, where a tick is emitted every `tickPeriod`. * Ticks are emitted between values from the source stream. */ diff --git a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala index d4a0a10827..0cbf1dbbda 100644 --- a/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala +++ b/core/shared/src/main/scala/fs2/timeseries/TimeStamped.scala @@ -121,7 +121,7 @@ object TimeStamped { * Every incoming `A` is echoed to the output. * * For example, the emitted bits per second of a `Stream[F, ByteVector]` can be calculated - * using `rate(1.0)(_.size * 8)`, which yields a stream of the emitted bits per second. + * using `rate(1.second)(_.size * 8)`, which yields a stream of the emitted bits per second. * * @param over time period over which to calculate * @param f function which extracts a feature of `A` @@ -359,4 +359,8 @@ object TimeStamped { TimeStamped(d, value) } } + + implicit val functor: Functor[TimeStamped[*]] = new Functor[TimeStamped[*]] { + def map[A, B](fa: TimeStamped[A])(f: A => B): TimeStamped[B] = fa.map(f) + } } diff --git a/site/_sidebar.md b/site/_sidebar.md index 8ee22f5871..77d8ad7d9f 100644 --- a/site/_sidebar.md +++ b/site/_sidebar.md @@ -5,6 +5,7 @@ - [Guide](guide.md) - [Concurrency Primitives](concurrency-primitives.md) - [I/O](io.md) +- [Time Series](timeseries.md) - [Scodec](scodec.md) - [API Reference](api-reference.md) - [FAQ](faq.md) diff --git a/site/scodec.md b/site/scodec.md index bbfbea8d16..b3240bc424 100644 --- a/site/scodec.md +++ b/site/scodec.md @@ -16,7 +16,7 @@ object Decode extends IOApp.Simple { val frames: StreamDecoder[ByteVector] = StreamDecoder.many(int32).flatMap { numBytes => StreamDecoder.once(bytes(numBytes)) } - val filePath = Path("path/to/file") + val filePath = Path("largefile.bin") val s: Stream[IO, ByteVector] = Files[IO].readAll(filePath).through(frames.toPipeByte) @@ -24,4 +24,8 @@ object Decode extends IOApp.Simple { s.compile.count.flatMap(cnt => IO.println(s"Read $cnt frames.")) } } -``` \ No newline at end of file +``` + +When run, this program will incrementally read chunks from "largefile.bin", then decode a stream of frames, where each frame is expected to begin with a number of bytes specified as a 32-bit signed int (the `int32` codec), followed by a frame payload of that many bytes. + +This library provides two main types: `StreamDecoder` and `StreamEncoder`. Each have various constructors and combinators to build instances up from regular scodec `Decoder` and `Encoder` values. Once built, they are typically converted to pipes via the `toPipeByte` method. In the example above, the `frames` decoder is converted to a `Pipe[IO, Byte, ByteVector]`. \ No newline at end of file diff --git a/site/timeseries.md b/site/timeseries.md new file mode 100644 index 0000000000..80e8777c4c --- /dev/null +++ b/site/timeseries.md @@ -0,0 +1,128 @@ +# Timeseries + +The `fs2.timeseries` package provides various utilities for working with time stamped values. + +The `TimeStamped` type associates a time stamp, represented as a Unix epoch, with a value: + +```scala +case class TimeStamped[+A](time: FiniteDuration, value: A) +``` + +There's a bunch of interesting things we can do with streams of time stamped values. For example, we can compute bitrates of binary streams: + +```scala mdoc +import fs2.Stream +import fs2.timeseries.TimeStamped +import scodec.bits.ByteVector + +def withBitrate[F[_]](input: Stream[F, TimeStamped[ByteVector]]): Stream[F, TimeStamped[Either[Long, ByteVector]]] = + TimeStamped.withPerSecondRate[ByteVector, Long](_.size * 8).toPipe(input) +``` + +The `TimeStamped.withPerSecondRate` function let's us aggregate a "feature" of a sequence of values received in each one second period. In this case, our feature is the number of bits, resulting in a bitrate. These computed bitrates are emitted in the output stream. + +Our `withBitrate` combinator requires a `Stream[F, TimeStamped[ByteVector]]` argument, whereas we'd normally have a `Stream[F, Byte]` when working with binary -- e.g., when reading network sockets. + +```scala mdoc +def withReceivedBitrate[F[_]](input: Stream[F, Byte]): Stream[F, TimeStamped[Either[Long, ByteVector]]] = + input.chunks.map(c => TimeStamped.unsafeNow(c.toByteVector)).through(withBitrate) +``` + +Each emitted sample is the sum of bits received during each one second period. Let's compute an average of that value over the last 10 seconds. We can do this via `mapAccumulate` along with a `scala.collection.immutable.Queue`: + +```scala mdoc +import scala.collection.immutable.Queue + +def withAverageBitrate[F[_]](input: Stream[F, Byte]): Stream[F, TimeStamped[Either[Long, ByteVector]]] = + withReceivedBitrate(input).mapAccumulate(Queue.empty[Long]) { + case (q, tsv @ TimeStamped(_, Right(_))) => (q, tsv) + case (q, TimeStamped(t, Left(sample))) => + val q2 = (sample +: q).take(10) + val average = q2.sum / q2.size + (q, TimeStamped(t, Left(average))) + }.map(_._2) +``` + +We can then store the computed bitrates in a `Ref` for later viewing / logging / etc. + +```scala mdoc +import cats.effect.Ref +import fs2.Chunk + +def measureAverageBitrate[F[_]](store: Ref[F, Long], input: Stream[F, Byte]): Stream[F, Byte] = + withAverageBitrate(input).flatMap { + case TimeStamped(_, Left(bitrate)) => Stream.exec(store.set(bitrate)) + case TimeStamped(_, Right(bytes)) => Stream.chunk(Chunk.byteVector(bytes)) + } +``` + +## Time Series + +All good, right? But wait, what happens if we stop receiving data for a while? There will be no input to `TimeStamped.withPerSecondRate`, which means there will be no output as well, and that means our `Ref` will not be updated -- it will be frozen with the last value! We need to `Ref` to accurately reflect the loss of data, which means we need some way to all `withPerSecondRate` to emit zero values as time passes. + +To accomplish this, we can modify our bitrate calculation to operate on values of type `TimeStamped[Option[ByteVector]]`, where a timestamped `None` represents a "tick" of the clock. A stream of timestamped options is referred to as a "time series" and the `fs2.timeseries.TimeSeries` object defines various ways to build such streams. + +First, let's adjust `withBitrate` so that it operates on `TimeStamped[Option[ByteVector]]` values: + +```scala mdoc:reset +import fs2.Stream +import fs2.timeseries.{TimeStamped, TimeSeries} +import scodec.bits.ByteVector + +def withBitrate[F[_]](input: Stream[F, TimeStamped[Option[ByteVector]]]): Stream[F, TimeStamped[Either[Long, Option[ByteVector]]]] = + TimeStamped.withPerSecondRate[Option[ByteVector], Long](_.map(_.size).getOrElse(0L) * 8).toPipe(input) +``` + +We then need to adjust `withReceivedBitrate` to convert our source stream to a time series: + +```scala mdoc +import scala.concurrent.duration._ +import cats.effect.Temporal + +def withReceivedBitrate[F[_]: Temporal](input: Stream[F, Byte]): Stream[F, TimeStamped[Either[Long, Option[ByteVector]]]] = + TimeSeries.timePulled(input.chunks.map(_.toByteVector), 1.second, 1.second).through(withBitrate) +``` + +We've used the `timePulled` operation to build a time series out of a regular stream -- as the name indicates, it time stamps each received value with the wall clock time it was pulled from the source stream. It also has a configurable tick period - in this case, we set it to one second. + +The remaining parts of our bitrate computation are unimpacted by this conversion to a time series (besides some slight signature changes). + +## Scans + +In the previous section, we wrote various stream transformations which incrementally built up a pipe that monitored the bitrate flowing through a stream. Each of these transformations, up until the final storage of the computed bitrate in a ref, were pure tranformations -- no effects were evaluated and no resources were opened. This type of processing is very common when working with time series. + +The `fs2.Scan` type allows us to express these types of pure, stateful computations in a way which gives us a quite a bit more compositionality than `Pipe`. Rewriting our above example using `Scan` gives us the following: + +```scala mdoc:reset +import fs2.{Stream, Chunk, Scan} +import fs2.timeseries.{TimeStamped, TimeSeries} +import cats.effect.{Ref, Temporal} +import scodec.bits.ByteVector +import scala.collection.immutable.Queue +import scala.concurrent.duration._ + +def bitrate = + TimeStamped.withPerSecondRate[Option[ByteVector], Long](_.map(_.size).getOrElse(0L) * 8) + +def averageBitrate = + bitrate.andThen(Scan.stateful1(Queue.empty[Long]) { + case (q, tsv @ TimeStamped(_, Right(_))) => (q, tsv) + case (q, TimeStamped(t, Left(sample))) => + val q2 = (sample +: q).take(10) + val average = q2.sum / q2.size + (q, TimeStamped(t, Left(average))) + }) + +def measureAverageBitrate[F[_]: Temporal](store: Ref[F, Long], input: Stream[F, Byte]): Stream[F, Byte] = + TimeSeries.timePulled(input.chunks.map(_.toByteVector), 1.second, 1.second) + .through(averageBitrate.toPipe) + .flatMap { + case TimeStamped(_, Left(bitrate)) => Stream.exec(store.set(bitrate)) + case TimeStamped(_, Right(Some(bytes))) => Stream.chunk(Chunk.byteVector(bytes)) + case TimeStamped(_, Right(None)) => Stream.empty + } +``` + +In this example, both `bitrate` and `averageBitrate` are instances of `Scan`. The `averageBitrate` function is able to directly reuse `bitrate` via `andThen` to construct a "bigger" scan. After we've finished composing scans, we convert the scan to a pipe and apply it to the input stream (in `measureAverageBitrate`). + +There are lots of combinators on `TimeStamped` and `TimeSeries` which provide scans. The `Scan` type also provides many generic combinators. \ No newline at end of file From 64ea6ade131e069b45c5ae8e13c01d9447937d68 Mon Sep 17 00:00:00 2001 From: Michael Pilquist Date: Sat, 23 Oct 2021 11:35:17 -0400 Subject: [PATCH 33/33] Bump ip4s version --- build.sbt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.sbt b/build.sbt index a7d8af6834..0ae7911c00 100644 --- a/build.sbt +++ b/build.sbt @@ -268,7 +268,7 @@ lazy val io = crossProject(JVMPlatform, JSPlatform) .jsConfigure(_.enablePlugins(ScalaJSBundlerPlugin)) .settings( name := "fs2-io", - libraryDependencies += "com.comcast" %%% "ip4s-core" % "3.0-27-0b113c0", + libraryDependencies += "com.comcast" %%% "ip4s-core" % "3.1.0", OsgiKeys.exportPackage := Seq("fs2.io.*"), OsgiKeys.privatePackage := Seq(), OsgiKeys.importPackage := {