From 9f75074e0095e0606f26b3053795df4877b62c79 Mon Sep 17 00:00:00 2001 From: Kalra Date: Sun, 19 Sep 2021 17:40:21 +0800 Subject: [PATCH 1/5] Add support for IO, SyncIO serialization --- .../main/scala/cats/effect/IOPlatform.scala | 2 +- .../src/main/scala/cats/effect/SyncIO.scala | 2 +- .../effect/kernel/testkit/Generators.scala | 4 +- .../cats/effect/IOPlatformSpecification.scala | 40 +++++++++++++++ .../scala/cats/effect/SerializationUtil.scala | 50 +++++++++++++++++++ 5 files changed, 94 insertions(+), 4 deletions(-) create mode 100644 tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala diff --git a/core/jvm/src/main/scala/cats/effect/IOPlatform.scala b/core/jvm/src/main/scala/cats/effect/IOPlatform.scala index e5e676109e..98c9b9682a 100644 --- a/core/jvm/src/main/scala/cats/effect/IOPlatform.scala +++ b/core/jvm/src/main/scala/cats/effect/IOPlatform.scala @@ -22,7 +22,7 @@ import scala.concurrent.duration._ import java.util.concurrent.{CompletableFuture, CountDownLatch, TimeUnit} -abstract private[effect] class IOPlatform[+A] { self: IO[A] => +abstract private[effect] class IOPlatform[+A] extends Serializable { self: IO[A] => /** * Produces the result by running the encapsulated effects as impure side effects. diff --git a/core/shared/src/main/scala/cats/effect/SyncIO.scala b/core/shared/src/main/scala/cats/effect/SyncIO.scala index 5934579121..0a7e3650ec 100644 --- a/core/shared/src/main/scala/cats/effect/SyncIO.scala +++ b/core/shared/src/main/scala/cats/effect/SyncIO.scala @@ -38,7 +38,7 @@ import scala.util.control.NonFatal * the JVM blocks the calling thread while the async part of the computation is run and doing so * on Scala.js is not supported. */ -sealed abstract class SyncIO[+A] private () { +sealed abstract class SyncIO[+A] private () extends Serializable { private[effect] def tag: Byte diff --git a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala index 0927000e94..f36483bd02 100644 --- a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala +++ b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala @@ -26,12 +26,12 @@ import scala.collection.immutable.SortedMap import scala.concurrent.ExecutionContext import scala.concurrent.duration.FiniteDuration -trait GenK[F[_]] { +trait GenK[F[_]] extends Serializable { def apply[A: Arbitrary: Cogen]: Gen[F[A]] } // Generators for * -> * kinded types -trait Generators1[F[_]] { +trait Generators1[F[_]] extends Serializable { protected val maxDepth: Int = 10 //todo: uniqueness based on... names, I guess. Have to solve the diamond problem somehow diff --git a/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala b/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala index 59405e66d9..01cfd3be31 100644 --- a/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala +++ b/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala @@ -29,6 +29,8 @@ import scala.concurrent.duration._ import java.util.concurrent.{CancellationException, CountDownLatch, Executors} import java.util.concurrent.CompletableFuture +import cats.effect.testkit.TestException + trait IOPlatformSpecification { self: BaseSpec with ScalaCheck => def platformSpecs = { @@ -255,6 +257,44 @@ trait IOPlatformSpecification { self: BaseSpec with ScalaCheck => } must completeAs(true) } + "serialise IO when not including evalOn" in ticked { implicit ticker => + import SerializationUtil._ + + def testSerialization[A](io: IO[A]): IO[A] = + IO.fromTry(serialize(io) >>= deserialize[IO[A]]).flatten + + val ls = List( + IO.pure(42), + IO.raiseError(TestException(42)), + IO.delay(42), + IO.realTime, + IO.monotonic, + IO.executionContext, + IO.pure(42).map(_ + 1), + IO.pure(42).flatMap(_ => IO.pure(42)), + IO.pure(42).attempt, + IO.raiseError(new RuntimeException("Err")).handleErrorWith(_ => IO.pure(42)), + IO.canceled, + IO.pure(42).onCancel(IO.pure(42)), + IO.pure(42).uncancelable, + IO.pure(42).start, + IO.racePair(IO.pure(42), IO.pure(42)), + IO.sleep(0.second), + IO.trace + ).map(_.as(42)).zipWithIndex + + forall(ls){ case (io, _) => io.attempt eqv testSerialization(io).attempt } + } + + "serialize SyncIO" in { + import SerializationUtil._ + + def testSerialization[A](io: SyncIO[A]): SyncIO[A] = + SyncIO.fromTry(serialize(io) >>= deserialize[SyncIO[A]]).flatten + + forAll { (io: SyncIO[Int]) => io.attempt eqv testSerialization(io).attempt } + } + } } } diff --git a/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala b/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala new file mode 100644 index 0000000000..3d519ecc04 --- /dev/null +++ b/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala @@ -0,0 +1,50 @@ +/* + * Copyright 2020-2021 Typelevel + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package cats.effect + +import scala.util.Try + +object SerializationUtil { + import java.io._ + import java.util.Base64 + + def serialize(`object`: Serializable): Try[String] = { + val baos = new ByteArrayOutputStream + val oos = new ObjectOutputStream(baos) + + val result = Try { + oos.writeObject(`object`) + Base64.getEncoder.encodeToString(baos.toByteArray) + } + + baos.close() + oos.close() + + result + } + + def deserialize[T <: Serializable](objectAsString: String): Try[T] = { + val data = Base64.getDecoder.decode(objectAsString) + val ois = new ObjectInputStream(new ByteArrayInputStream(data)) + + val result = Try(ois.readObject.asInstanceOf[T]) + + ois.close() + + result + } +} From fad95bdfdf4f5471801856720008b71d8f0f7d27 Mon Sep 17 00:00:00 2001 From: Kalra Date: Wed, 22 Sep 2021 00:26:48 +0800 Subject: [PATCH 2/5] Addressed review feedback #1, added property check for IO serialization, switched to Serializable laws --- .../cats/effect/tracing/TracingEvent.scala | 2 +- .../effect/kernel/testkit/Generators.scala | 21 ++++++++ .../cats/effect/kernel/testkit/pure.scala | 3 +- .../cats/effect/kernel/AsyncPlatform.scala | 2 +- .../cats/effect/kernel/ClockPlatform.scala | 2 +- .../cats/effect/kernel/ResourcePlatform.scala | 2 +- .../main/scala/cats/effect/kernel/Clock.scala | 2 +- .../main/scala/cats/effect/kernel/Cont.scala | 2 +- .../scala/cats/effect/kernel/Deferred.scala | 4 +- .../main/scala/cats/effect/kernel/Fiber.scala | 2 +- .../cats/effect/kernel/MiniSemaphore.scala | 2 +- .../scala/cats/effect/kernel/ParallelF.scala | 2 +- .../main/scala/cats/effect/kernel/Ref.scala | 4 +- .../scala/cats/effect/kernel/Resource.scala | 2 +- .../scala/cats/effect/kernel/Unique.scala | 4 +- .../cats/effect/testkit/TestInstances.scala | 17 +++++++ .../cats/effect/IOPlatformSpecification.scala | 40 --------------- .../scala/cats/effect/SerializationUtil.scala | 50 ------------------- .../src/test/scala/cats/effect/IOSpec.scala | 16 ++++-- .../test/scala/cats/effect/SyncIOSpec.scala | 5 ++ 20 files changed, 73 insertions(+), 111 deletions(-) delete mode 100644 tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala diff --git a/core/shared/src/main/scala/cats/effect/tracing/TracingEvent.scala b/core/shared/src/main/scala/cats/effect/tracing/TracingEvent.scala index c5cb2d8cee..99d8d6cc12 100644 --- a/core/shared/src/main/scala/cats/effect/tracing/TracingEvent.scala +++ b/core/shared/src/main/scala/cats/effect/tracing/TracingEvent.scala @@ -16,7 +16,7 @@ package cats.effect.tracing -private[effect] sealed trait TracingEvent +private[effect] sealed trait TracingEvent extends Serializable private[effect] object TracingEvent { final class StackTrace extends Throwable with TracingEvent diff --git a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala index f36483bd02..eb7db720f4 100644 --- a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala +++ b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala @@ -308,6 +308,27 @@ trait AsyncGenerators[F[_]] extends GenTemporalGenerators[F, Throwable] with Syn } yield F.evalOn(fa, ec) } + +trait AsyncGeneratorsWithoutEvalShift[F[_]] extends GenTemporalGenerators[F, Throwable] with SyncGenerators[F] { + implicit val F: Async[F] + implicit protected val cogenFU: Cogen[F[Unit]] = Cogen[Unit].contramap(_ => ()) + + override protected def recursiveGen[A: Arbitrary: Cogen](deeper: GenK[F]) = + ("async" -> genAsync[A](deeper)) :: super.recursiveGen[A](deeper) + + private def genAsync[A: Arbitrary](deeper: GenK[F]) = + for { + result <- arbitrary[Either[Throwable, A]] + + fo <- deeper[Option[F[Unit]]]( + Arbitrary(Gen.option[F[Unit]](deeper[Unit])), + Cogen.cogenOption(cogenFU)) + } yield F + .async[A](k => F.delay(k(result)) >> fo) + .flatMap(F.pure(_)) + .handleErrorWith(F.raiseError(_)) +} + trait ParallelFGenerators { implicit def arbitraryParallelF[F[_], A]( implicit ArbF: Arbitrary[F[A]]): Arbitrary[ParallelF[F, A]] = diff --git a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala index 5d1102fdea..1ac7ea226a 100644 --- a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala +++ b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala @@ -361,8 +361,9 @@ object pure { ft.mapK(fk) } + // todo: MVar is not Serializable, release then update here final class PureFiber[E, A](val state0: MVar[Outcome[PureConc[E, *], E, A]]) - extends Fiber[PureConc[E, *], E, A] { + extends Fiber[PureConc[E, *], E, A] with Serializable { private[this] val state = state0[PureConc[E, *]] diff --git a/kernel/jvm/src/main/scala/cats/effect/kernel/AsyncPlatform.scala b/kernel/jvm/src/main/scala/cats/effect/kernel/AsyncPlatform.scala index c953b5428c..b2e715bade 100644 --- a/kernel/jvm/src/main/scala/cats/effect/kernel/AsyncPlatform.scala +++ b/kernel/jvm/src/main/scala/cats/effect/kernel/AsyncPlatform.scala @@ -19,7 +19,7 @@ package cats.effect.kernel import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletionException -private[kernel] trait AsyncPlatform[F[_]] { this: Async[F] => +private[kernel] trait AsyncPlatform[F[_]] extends Serializable { this: Async[F] => /** * Suspend a [[java.util.concurrent.CompletableFuture]] into the `F[_]` context. diff --git a/kernel/jvm/src/main/scala/cats/effect/kernel/ClockPlatform.scala b/kernel/jvm/src/main/scala/cats/effect/kernel/ClockPlatform.scala index 82a49fa931..a1ed998a93 100644 --- a/kernel/jvm/src/main/scala/cats/effect/kernel/ClockPlatform.scala +++ b/kernel/jvm/src/main/scala/cats/effect/kernel/ClockPlatform.scala @@ -18,7 +18,7 @@ package cats.effect.kernel import java.time.Instant -private[effect] trait ClockPlatform[F[_]] { self: Clock[F] => +private[effect] trait ClockPlatform[F[_]] extends Serializable { self: Clock[F] => def realTimeInstant: F[Instant] = { self.applicative.map(self.realTime)(d => Instant.ofEpochMilli(d.toMillis)) } diff --git a/kernel/jvm/src/main/scala/cats/effect/kernel/ResourcePlatform.scala b/kernel/jvm/src/main/scala/cats/effect/kernel/ResourcePlatform.scala index 99ef7363c7..0c0deb6640 100644 --- a/kernel/jvm/src/main/scala/cats/effect/kernel/ResourcePlatform.scala +++ b/kernel/jvm/src/main/scala/cats/effect/kernel/ResourcePlatform.scala @@ -21,7 +21,7 @@ import javax.security.auth.Destroyable /** * JVM-specific Resource methods */ -private[effect] trait ResourcePlatform { +private[effect] trait ResourcePlatform extends Serializable { /** * Creates a [[Resource]] by wrapping a Java diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Clock.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Clock.scala index 5f18ea7c2f..94670450fd 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Clock.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Clock.scala @@ -28,7 +28,7 @@ import cats.{Defer, Monad} * A typeclass which encodes various notions of time. Analogous to some of the time functions * exposed by [[java.lang.System]]. */ -trait Clock[F[_]] extends ClockPlatform[F] { +trait Clock[F[_]] extends ClockPlatform[F] with Serializable { def applicative: Applicative[F] diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Cont.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Cont.scala index 0e73eb0b5c..d401f39c84 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Cont.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Cont.scala @@ -59,7 +59,7 @@ import cats.~> * override `Async[F].async` with your implementation, and use `Async.defaultCont` to implement * `Async[F].cont`. */ -trait Cont[F[_], K, R] { +trait Cont[F[_], K, R] extends Serializable { def apply[G[_]]( implicit G: MonadCancel[G, Throwable]): (Either[Throwable, K] => Unit, G[K], F ~> G) => G[R] diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Deferred.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Deferred.scala index 450b781299..9d8a8e4216 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Deferred.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Deferred.scala @@ -213,7 +213,7 @@ object Deferred { } } -trait DeferredSource[F[_], A] { +trait DeferredSource[F[_], A] extends Serializable { /** * Obtains the value of the `Deferred`, or waits until it has been completed. The returned @@ -240,7 +240,7 @@ object DeferredSource { } } -trait DeferredSink[F[_], A] { +trait DeferredSink[F[_], A] extends Serializable { /** * If this `Deferred` is empty, sets the current value to `a`, and notifies any and all diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Fiber.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Fiber.scala index cd0242efb9..677a5fa7d7 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Fiber.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Fiber.scala @@ -25,7 +25,7 @@ import cats.syntax.all._ * @see * [[GenSpawn]] documentation for more detailed information on the concurrency of fibers. */ -trait Fiber[F[_], E, A] { +trait Fiber[F[_], E, A] extends Serializable { /** * Requests the cancelation of the fiber bound to this `Fiber` handle and awaits its diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/MiniSemaphore.scala b/kernel/shared/src/main/scala/cats/effect/kernel/MiniSemaphore.scala index d4db4cd1ab..621bd2b05b 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/MiniSemaphore.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/MiniSemaphore.scala @@ -25,7 +25,7 @@ import scala.collection.immutable.{Queue => ScalaQueue} /** * A cut-down version of semaphore used to implement parTraverseN */ -private[kernel] abstract class MiniSemaphore[F[_]] { +private[kernel] abstract class MiniSemaphore[F[_]] extends Serializable { /** * Sequence an action while holding a permit diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/ParallelF.scala b/kernel/shared/src/main/scala/cats/effect/kernel/ParallelF.scala index 6be70e99fa..8c1d430210 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/ParallelF.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/ParallelF.scala @@ -18,7 +18,7 @@ package cats.effect.kernel //See https://failex.blogspot.com/2017/04/the-high-cost-of-anyval-subclasses.html object Par { - sealed abstract class ParallelFImpl { + sealed abstract class ParallelFImpl extends Serializable { type T[F[_], A] def apply[F[_], A](fa: F[A]): T[F, A] def value[F[_], A](t: T[F, A]): F[A] diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Ref.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Ref.scala index 0b13617704..fc05e5e0b5 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Ref.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Ref.scala @@ -464,7 +464,7 @@ object Ref { } } -trait RefSource[F[_], A] { +trait RefSource[F[_], A] extends Serializable { /** * Obtains the current value. @@ -486,7 +486,7 @@ object RefSource { } } -trait RefSink[F[_], A] { +trait RefSink[F[_], A] extends Serializable { /** * Sets the current value to `a`. diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Resource.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Resource.scala index 576f8b91ef..c68c8a4353 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Resource.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Resource.scala @@ -147,7 +147,7 @@ import scala.concurrent.duration.FiniteDuration * @tparam A * the type of resource */ -sealed abstract class Resource[F[_], +A] { +sealed abstract class Resource[F[_], +A] extends Serializable { import Resource._ private[effect] def fold[B]( diff --git a/kernel/shared/src/main/scala/cats/effect/kernel/Unique.scala b/kernel/shared/src/main/scala/cats/effect/kernel/Unique.scala index ba1a496837..babe045cbb 100644 --- a/kernel/shared/src/main/scala/cats/effect/kernel/Unique.scala +++ b/kernel/shared/src/main/scala/cats/effect/kernel/Unique.scala @@ -18,7 +18,7 @@ package cats.effect.kernel import cats.{Applicative, Hash} -trait Unique[F[_]] { +trait Unique[F[_]] extends Serializable { def applicative: Applicative[F] def unique: F[Unique.Token] } @@ -27,7 +27,7 @@ object Unique { def apply[F[_]](implicit F: Unique[F]): F.type = F - final class Token + final class Token extends Serializable object Token { implicit val tokenHash: Hash[Token] = diff --git a/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala b/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala index b5b8ee3644..57c45e42aa 100644 --- a/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala +++ b/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala @@ -20,6 +20,7 @@ package testkit import cats.{~>, Applicative, Eq, Id, Order, Show} import cats.effect.kernel.testkit.{ AsyncGenerators, + AsyncGeneratorsWithoutEvalShift, GenK, OutcomeGenerators, ParallelFGenerators, @@ -76,6 +77,22 @@ trait TestInstances extends ParallelFGenerators with OutcomeGenerators with Sync Arbitrary(generators.generators[A]) } + def arbitraryIOWithoutContextShift[A: Arbitrary: Cogen]: Arbitrary[IO[A]] = { + val generators = new AsyncGeneratorsWithoutEvalShift[IO] { + override implicit val F: Async[IO] = IO.asyncForIO + override implicit protected val arbitraryFD: Arbitrary[FiniteDuration] = outer.arbitraryFiniteDuration + override implicit val arbitraryE: Arbitrary[Throwable] = outer.arbitraryThrowable + override val cogenE: Cogen[Throwable] = Cogen[Throwable] + + override def recursiveGen[B: Arbitrary: Cogen](deeper: GenK[IO]) = + super + .recursiveGen[B](deeper) + .filterNot(x =>x._1 == "evalOn" || x._1 == "racePair") // todo: enable racePair after MVar been made serialization compatible + } + + Arbitrary(generators.generators[A]) + } + implicit def arbitrarySyncIO[A: Arbitrary: Cogen]: Arbitrary[SyncIO[A]] = { val generators = new SyncGenerators[SyncIO] { val arbitraryE: Arbitrary[Throwable] = diff --git a/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala b/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala index 01cfd3be31..59405e66d9 100644 --- a/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala +++ b/tests/jvm/src/test/scala/cats/effect/IOPlatformSpecification.scala @@ -29,8 +29,6 @@ import scala.concurrent.duration._ import java.util.concurrent.{CancellationException, CountDownLatch, Executors} import java.util.concurrent.CompletableFuture -import cats.effect.testkit.TestException - trait IOPlatformSpecification { self: BaseSpec with ScalaCheck => def platformSpecs = { @@ -257,44 +255,6 @@ trait IOPlatformSpecification { self: BaseSpec with ScalaCheck => } must completeAs(true) } - "serialise IO when not including evalOn" in ticked { implicit ticker => - import SerializationUtil._ - - def testSerialization[A](io: IO[A]): IO[A] = - IO.fromTry(serialize(io) >>= deserialize[IO[A]]).flatten - - val ls = List( - IO.pure(42), - IO.raiseError(TestException(42)), - IO.delay(42), - IO.realTime, - IO.monotonic, - IO.executionContext, - IO.pure(42).map(_ + 1), - IO.pure(42).flatMap(_ => IO.pure(42)), - IO.pure(42).attempt, - IO.raiseError(new RuntimeException("Err")).handleErrorWith(_ => IO.pure(42)), - IO.canceled, - IO.pure(42).onCancel(IO.pure(42)), - IO.pure(42).uncancelable, - IO.pure(42).start, - IO.racePair(IO.pure(42), IO.pure(42)), - IO.sleep(0.second), - IO.trace - ).map(_.as(42)).zipWithIndex - - forall(ls){ case (io, _) => io.attempt eqv testSerialization(io).attempt } - } - - "serialize SyncIO" in { - import SerializationUtil._ - - def testSerialization[A](io: SyncIO[A]): SyncIO[A] = - SyncIO.fromTry(serialize(io) >>= deserialize[SyncIO[A]]).flatten - - forAll { (io: SyncIO[Int]) => io.attempt eqv testSerialization(io).attempt } - } - } } } diff --git a/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala b/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala deleted file mode 100644 index 3d519ecc04..0000000000 --- a/tests/jvm/src/test/scala/cats/effect/SerializationUtil.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright 2020-2021 Typelevel - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package cats.effect - -import scala.util.Try - -object SerializationUtil { - import java.io._ - import java.util.Base64 - - def serialize(`object`: Serializable): Try[String] = { - val baos = new ByteArrayOutputStream - val oos = new ObjectOutputStream(baos) - - val result = Try { - oos.writeObject(`object`) - Base64.getEncoder.encodeToString(baos.toByteArray) - } - - baos.close() - oos.close() - - result - } - - def deserialize[T <: Serializable](objectAsString: String): Try[T] = { - val data = Base64.getDecoder.decode(objectAsString) - val ois = new ObjectInputStream(new ByteArrayInputStream(data)) - - val result = Try(ois.readObject.asInstanceOf[T]) - - ois.close() - - result - } -} diff --git a/tests/shared/src/test/scala/cats/effect/IOSpec.scala b/tests/shared/src/test/scala/cats/effect/IOSpec.scala index ea7ceab028..2e9154892f 100644 --- a/tests/shared/src/test/scala/cats/effect/IOSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/IOSpec.scala @@ -17,15 +17,15 @@ package cats.effect import cats.kernel.laws.discipline.MonoidTests -import cats.laws.discipline.{AlignTests, SemigroupKTests} +import cats.kernel.laws.SerializableLaws.serializable +import cats.laws.discipline.{AlignTests, SemigroupKTests, SerializableTests} import cats.laws.discipline.arbitrary._ - import cats.effect.implicits._ import cats.effect.laws.AsyncTests import cats.effect.testkit.TestContext import cats.syntax.all._ - -import org.scalacheck.Prop, Prop.forAll +import org.scalacheck.Prop +import Prop.forAll // import org.scalacheck.rng.Seed // import org.specs2.scalacheck.Parameters @@ -1329,6 +1329,14 @@ class IOSpec extends BaseSpec with Discipline with IOPlatformSpecification { } yield res } + "serialize" in { + forAll { (io: IO[Int]) => serializable(io) }( + implicitly, + arbitraryIOWithoutContextShift, + implicitly, + implicitly) + } + platformSpecs } diff --git a/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala b/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala index 7722742d96..a65f4e44ba 100644 --- a/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala @@ -17,6 +17,7 @@ package cats.effect import cats.kernel.laws.discipline.MonoidTests +import cats.kernel.laws.SerializableLaws.serializable import cats.laws.discipline.AlignTests import cats.laws.discipline.arbitrary._ import cats.effect.laws.SyncTests @@ -220,6 +221,10 @@ class SyncIOSpec extends BaseSpec with Discipline with SyncIOPlatformSpecificati res <- IO.delay(res1 mustEqual res2) } yield res } + + "serialize" in { + forAll { (io: SyncIO[Int]) => serializable(io) } + } } { From 1a1be8f26f225b3351928f7cade01bd21f6b3bd1 Mon Sep 17 00:00:00 2001 From: Gagandeep Kalra Date: Mon, 22 Nov 2021 00:51:14 +0800 Subject: [PATCH 3/5] fixed formatting --- .scalafmt.conf | 2 +- .../main/scala/cats/effect/kernel/testkit/Generators.scala | 5 +++-- .../src/main/scala/cats/effect/kernel/testkit/pure.scala | 3 ++- .../src/main/scala/cats/effect/testkit/TestInstances.scala | 6 ++++-- 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/.scalafmt.conf b/.scalafmt.conf index b48e8f7dd7..00588bbdec 100644 --- a/.scalafmt.conf +++ b/.scalafmt.conf @@ -5,7 +5,7 @@ runner.dialect = Scala213Source3 project.excludeFilters = [ "scalafix/*" ] - +lineEndings = preserve maxColumn = 96 includeCurlyBraceInSelectChains = true diff --git a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala index 4f9c44a03c..621e7e6fdd 100644 --- a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala +++ b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/Generators.scala @@ -308,8 +308,9 @@ trait AsyncGenerators[F[_]] extends GenTemporalGenerators[F, Throwable] with Syn } yield F.evalOn(fa, ec) } - -trait AsyncGeneratorsWithoutEvalShift[F[_]] extends GenTemporalGenerators[F, Throwable] with SyncGenerators[F] { +trait AsyncGeneratorsWithoutEvalShift[F[_]] + extends GenTemporalGenerators[F, Throwable] + with SyncGenerators[F] { implicit val F: Async[F] implicit protected val cogenFU: Cogen[F[Unit]] = Cogen[Unit].contramap(_ => ()) diff --git a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala index 5182a01ff2..0f6ebc3a67 100644 --- a/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala +++ b/kernel-testkit/shared/src/main/scala/cats/effect/kernel/testkit/pure.scala @@ -363,7 +363,8 @@ object pure { // todo: MVar is not Serializable, release then update here final class PureFiber[E, A](val state0: MVar[Outcome[PureConc[E, *], E, A]]) - extends Fiber[PureConc[E, *], E, A] with Serializable { + extends Fiber[PureConc[E, *], E, A] + with Serializable { private[this] val state = state0[PureConc[E, *]] diff --git a/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala b/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala index 834ef2ce51..95a3b41ea7 100644 --- a/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala +++ b/testkit/shared/src/main/scala/cats/effect/testkit/TestInstances.scala @@ -80,14 +80,16 @@ trait TestInstances extends ParallelFGenerators with OutcomeGenerators with Sync def arbitraryIOWithoutContextShift[A: Arbitrary: Cogen]: Arbitrary[IO[A]] = { val generators = new AsyncGeneratorsWithoutEvalShift[IO] { override implicit val F: Async[IO] = IO.asyncForIO - override implicit protected val arbitraryFD: Arbitrary[FiniteDuration] = outer.arbitraryFiniteDuration + override implicit protected val arbitraryFD: Arbitrary[FiniteDuration] = + outer.arbitraryFiniteDuration override implicit val arbitraryE: Arbitrary[Throwable] = outer.arbitraryThrowable override val cogenE: Cogen[Throwable] = Cogen[Throwable] override def recursiveGen[B: Arbitrary: Cogen](deeper: GenK[IO]) = super .recursiveGen[B](deeper) - .filterNot(x =>x._1 == "evalOn" || x._1 == "racePair") // todo: enable racePair after MVar been made serialization compatible + .filterNot(x => + x._1 == "evalOn" || x._1 == "racePair") // todo: enable racePair after MVar been made serialization compatible } Arbitrary(generators.generators[A]) From f6603be128dfc9196b433a12ebf6293b73192418 Mon Sep 17 00:00:00 2001 From: Gagandeep Kalra Date: Mon, 22 Nov 2021 20:41:55 +0800 Subject: [PATCH 4/5] fixed formatting --- tests/shared/src/test/scala/cats/effect/IOSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/shared/src/test/scala/cats/effect/IOSpec.scala b/tests/shared/src/test/scala/cats/effect/IOSpec.scala index d05fc5d472..202be3c42f 100644 --- a/tests/shared/src/test/scala/cats/effect/IOSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/IOSpec.scala @@ -18,7 +18,7 @@ package cats.effect import cats.kernel.laws.discipline.MonoidTests import cats.kernel.laws.SerializableLaws.serializable -import cats.laws.discipline.{AlignTests, SemigroupKTests, SerializableTests} +import cats.laws.discipline.{AlignTests, SemigroupKTests} import cats.laws.discipline.arbitrary._ import cats.effect.implicits._ import cats.effect.laws.AsyncTests From 9ab6666ba92f77a9a1e7291db7d4ec89c1d95d3a Mon Sep 17 00:00:00 2001 From: Daniel Spiewak Date: Wed, 16 Mar 2022 22:15:53 -0600 Subject: [PATCH 5/5] Moved `SyncIO` serialization tests to avoid Scala 3 --- .../SyncIOScalaVersionSpecification.scala | 30 +++++++++++++++++++ .../SyncIOScalaVersionSpecification.scala | 25 ++++++++++++++++ .../test/scala/cats/effect/SyncIOSpec.scala | 11 +++---- 3 files changed, 61 insertions(+), 5 deletions(-) create mode 100644 tests/shared/src/test/scala-2/cats/effect/SyncIOScalaVersionSpecification.scala create mode 100644 tests/shared/src/test/scala-3/cats/effect/SyncIOScalaVersionSpecification.scala diff --git a/tests/shared/src/test/scala-2/cats/effect/SyncIOScalaVersionSpecification.scala b/tests/shared/src/test/scala-2/cats/effect/SyncIOScalaVersionSpecification.scala new file mode 100644 index 0000000000..87ef34a0e6 --- /dev/null +++ b/tests/shared/src/test/scala-2/cats/effect/SyncIOScalaVersionSpecification.scala @@ -0,0 +1,30 @@ +/* + * Copyright 2020-2022 Typelevel + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package cats.effect + +import cats.kernel.laws.SerializableLaws.serializable + +import org.scalacheck.Prop.forAll +import org.typelevel.discipline.specs2.mutable.Discipline + +// collapse this back into SyncIOSpec once we're on a release with lampepfl/dotty#14686 +trait SyncIOScalaVersionSpecification extends BaseSpec with Discipline { + def scalaVersionSpecs = + "serialize" in { + forAll { (io: SyncIO[Int]) => serializable(io) } + } +} diff --git a/tests/shared/src/test/scala-3/cats/effect/SyncIOScalaVersionSpecification.scala b/tests/shared/src/test/scala-3/cats/effect/SyncIOScalaVersionSpecification.scala new file mode 100644 index 0000000000..ef2861067f --- /dev/null +++ b/tests/shared/src/test/scala-3/cats/effect/SyncIOScalaVersionSpecification.scala @@ -0,0 +1,25 @@ +/* + * Copyright 2020-2022 Typelevel + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package cats.effect + +import org.specs2.mutable.SpecificationLike +import org.specs2.specification.core.Fragments + +// collapse this back into SyncIOSpec once we're on a release with lampepfl/dotty#14686 +trait SyncIOScalaVersionSpecification extends SpecificationLike { + def scalaVersionSpecs = Fragments.empty +} diff --git a/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala b/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala index 0ed119d153..d09f671732 100644 --- a/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala +++ b/tests/shared/src/test/scala/cats/effect/SyncIOSpec.scala @@ -17,7 +17,6 @@ package cats.effect import cats.effect.laws.SyncTests -import cats.kernel.laws.SerializableLaws.serializable import cats.kernel.laws.discipline.MonoidTests import cats.laws.discipline.AlignTests import cats.laws.discipline.arbitrary._ @@ -26,7 +25,11 @@ import cats.syntax.all._ import org.scalacheck.Prop.forAll import org.typelevel.discipline.specs2.mutable.Discipline -class SyncIOSpec extends BaseSpec with Discipline with SyncIOPlatformSpecification { +class SyncIOSpec + extends BaseSpec + with Discipline + with SyncIOPlatformSpecification + with SyncIOScalaVersionSpecification { "sync io monad" should { "produce a pure value when run" in { @@ -222,9 +225,7 @@ class SyncIOSpec extends BaseSpec with Discipline with SyncIOPlatformSpecificati } yield res } - "serialize" in { - forAll { (io: SyncIO[Int]) => serializable(io) } - } + scalaVersionSpecs } {