Rename sbt akka modules
Co-authored-by: Sean Glover <sean@seanglover.com>
This commit is contained in:
parent
b92b749946
commit
24c03cde19
2930 changed files with 1466 additions and 1462 deletions
|
|
@ -0,0 +1,27 @@
|
|||
/*
|
||||
* Copyright (C) 2017-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.util.concurrent.{ Flow => JavaFlow }
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.NotUsed
|
||||
import pekko.stream.scaladsl.{ Flow, JavaFlowSupport, Sink, Source }
|
||||
import org.reactivestreams._
|
||||
|
||||
class IterablePublisherViaJavaFlowPublisherTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
override def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val sourceViaJavaFlowPublisher: JavaFlow.Publisher[Int] = Source(iterable(elements))
|
||||
.runWith(JavaFlowSupport.Sink.asPublisher(fanout = false))
|
||||
|
||||
val javaFlowPublisherIntoAkkaSource: Source[Int, NotUsed] =
|
||||
JavaFlowSupport.Source.fromPublisher(sourceViaJavaFlowPublisher)
|
||||
|
||||
javaFlowPublisherIntoAkkaSource
|
||||
.runWith(Sink.asPublisher(false)) // back as RS Publisher
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.testng.annotations.AfterClass
|
||||
import org.testng.annotations.BeforeClass
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.actor.ActorSystem
|
||||
import pekko.actor.ActorSystemImpl
|
||||
import pekko.event.Logging
|
||||
import pekko.testkit.PekkoSpec
|
||||
import pekko.testkit.EventFilter
|
||||
import pekko.testkit.TestEvent
|
||||
|
||||
trait ActorSystemLifecycle {
|
||||
|
||||
protected var _system: ActorSystem = _
|
||||
|
||||
implicit final def system: ActorSystem = _system
|
||||
|
||||
def additionalConfig: Config = ConfigFactory.empty()
|
||||
|
||||
def shutdownTimeout: FiniteDuration = 10.seconds
|
||||
|
||||
@BeforeClass
|
||||
def createActorSystem(): Unit = {
|
||||
_system = ActorSystem(Logging.simpleName(getClass), additionalConfig.withFallback(PekkoSpec.testConf))
|
||||
_system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception")))
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
def shutdownActorSystem(): Unit = {
|
||||
try {
|
||||
Await.ready(system.terminate(), shutdownTimeout)
|
||||
} catch {
|
||||
case _: TimeoutException =>
|
||||
val msg = "Failed to stop [%s] within [%s] \n%s".format(
|
||||
system.name,
|
||||
shutdownTimeout,
|
||||
system.asInstanceOf[ActorSystemImpl].printTree)
|
||||
throw new RuntimeException(msg)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.tck.SubscriberBlackboxVerification
|
||||
import org.reactivestreams.tck.SubscriberWhiteboxVerification
|
||||
import org.reactivestreams.tck.TestEnvironment
|
||||
import org.scalatestplus.testng.TestNGSuiteLike
|
||||
|
||||
import org.apache.pekko.actor.ActorSystem
|
||||
|
||||
abstract class PekkoSubscriberBlackboxVerification[T](env: TestEnvironment)
|
||||
extends SubscriberBlackboxVerification[T](env)
|
||||
with TestNGSuiteLike
|
||||
with PekkoSubscriberVerificationLike
|
||||
with ActorSystemLifecycle {
|
||||
|
||||
def this(printlnDebug: Boolean) =
|
||||
this(new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug))
|
||||
|
||||
def this() = this(false)
|
||||
}
|
||||
|
||||
abstract class PekkoSubscriberWhiteboxVerification[T](env: TestEnvironment)
|
||||
extends SubscriberWhiteboxVerification[T](env)
|
||||
with TestNGSuiteLike
|
||||
with PekkoSubscriberVerificationLike {
|
||||
|
||||
def this(printlnDebug: Boolean) =
|
||||
this(new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug))
|
||||
|
||||
def this() = this(false)
|
||||
}
|
||||
|
||||
trait PekkoSubscriberVerificationLike {
|
||||
implicit def system: ActorSystem
|
||||
}
|
||||
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Subscriber
|
||||
import org.testng.SkipException
|
||||
|
||||
import org.apache.pekko.stream.scaladsl._
|
||||
|
||||
class CancelledSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
|
||||
|
||||
override def createSubscriber(): Subscriber[Int] =
|
||||
Flow[Int].to(Sink.cancelled).runWith(Source.asSubscriber)
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
override def required_spec201_blackbox_mustSignalDemandViaSubscriptionRequest() = {
|
||||
throw new SkipException("Cancelled sink doesn't signal demand")
|
||||
}
|
||||
override def required_spec209_blackbox_mustBePreparedToReceiveAnOnCompleteSignalWithPrecedingRequestCall() = {
|
||||
throw new SkipException("Cancelled sink doesn't signal demand")
|
||||
}
|
||||
override def required_spec210_blackbox_mustBePreparedToReceiveAnOnErrorSignalWithPrecedingRequestCall() = {
|
||||
throw new SkipException("Cancelled sink doesn't signal demand")
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class ConcatTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
Source(iterable(elements / 2)).concat(Source(iterable((elements + 1) / 2))).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,16 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko.stream.impl.EmptyPublisher
|
||||
|
||||
class EmptyPublisherTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = EmptyPublisher[Int]
|
||||
|
||||
override def maxElementsFromPublisher(): Long = 0
|
||||
}
|
||||
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class FanoutPublisherTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val iterable: immutable.Iterable[Int] =
|
||||
if (elements == 0) new immutable.Iterable[Int] { override def iterator = Iterator.from(0) }
|
||||
else
|
||||
0 until elements.toInt
|
||||
|
||||
Source(iterable).runWith(Sink.asPublisher(true))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.nio.file.Files
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
import org.testng.annotations.{ AfterClass, BeforeClass }
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.actor.ActorSystem
|
||||
import pekko.event.Logging
|
||||
import pekko.stream.scaladsl.{ FileIO, Sink }
|
||||
import pekko.stream.testkit.Utils._
|
||||
import pekko.testkit.{ EventFilter, TestEvent }
|
||||
import pekko.testkit.PekkoSpec
|
||||
import pekko.util.ByteString
|
||||
|
||||
class FilePublisherTest extends PekkoPublisherVerification[ByteString] {
|
||||
|
||||
val ChunkSize = 256
|
||||
val Elements = 1000
|
||||
|
||||
@BeforeClass
|
||||
override def createActorSystem(): Unit = {
|
||||
_system = ActorSystem(Logging.simpleName(getClass), UnboundedMailboxConfig.withFallback(PekkoSpec.testConf))
|
||||
_system.eventStream.publish(TestEvent.Mute(EventFilter[RuntimeException]("Test exception")))
|
||||
}
|
||||
|
||||
val file = {
|
||||
val f = Files.createTempFile("file-source-tck", ".tmp")
|
||||
val chunk = "x" * ChunkSize
|
||||
|
||||
val fw = Files.newBufferedWriter(f)
|
||||
List.fill(Elements)(chunk).foreach(fw.append)
|
||||
fw.close()
|
||||
f
|
||||
}
|
||||
|
||||
def createPublisher(elements: Long): Publisher[ByteString] =
|
||||
FileIO.fromPath(file, chunkSize = 512).take(elements).runWith(Sink.asPublisher(false))
|
||||
|
||||
@AfterClass
|
||||
def after() = Files.delete(file)
|
||||
|
||||
override def maxElementsFromPublisher(): Long = Elements
|
||||
}
|
||||
|
|
@ -0,0 +1,30 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.concurrent.{ Await, Promise }
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.reactivestreams.{ Publisher, Subscriber }
|
||||
|
||||
import org.apache.pekko.stream.scaladsl.{ Sink, Source }
|
||||
|
||||
class FlatMapConcatDoubleSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
|
||||
|
||||
def createSubscriber(): Subscriber[Int] = {
|
||||
val subscriber = Promise[Subscriber[Int]]()
|
||||
Source
|
||||
.single(Source.fromPublisher(new Publisher[Int] {
|
||||
def subscribe(s: Subscriber[_ >: Int]): Unit =
|
||||
subscriber.success(s.asInstanceOf[Subscriber[Int]])
|
||||
}))
|
||||
.flatMapConcat(identity)
|
||||
.runWith(Sink.ignore)
|
||||
|
||||
Await.result(subscriber.future, 1.second)
|
||||
}
|
||||
|
||||
def createElement(element: Int): Int = element
|
||||
}
|
||||
|
|
@ -0,0 +1,21 @@
|
|||
/*
|
||||
* Copyright (C) 2019-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko.stream.scaladsl.{ Flow, Keep, Sink, Source }
|
||||
|
||||
class FlatMapPrefixTest extends PekkoPublisherVerification[Int] {
|
||||
override def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val publisher = Source(iterable(elements))
|
||||
.map(_.toInt)
|
||||
.flatMapPrefixMat(1) { seq =>
|
||||
Flow[Int].prepend(Source(seq))
|
||||
}(Keep.left)
|
||||
.runWith(Sink.asPublisher(false))
|
||||
publisher
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
import pekko.util.ConstantFun
|
||||
|
||||
class FlattenTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val s1 = Source(iterable(elements / 2))
|
||||
val s2 = Source(iterable((elements + 1) / 2))
|
||||
Source(List(s1, s2)).flatMapConcat(ConstantFun.scalaIdentityFunction).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,17 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Subscriber
|
||||
|
||||
import org.apache.pekko.stream.scaladsl._
|
||||
|
||||
class FoldSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
|
||||
|
||||
override def createSubscriber(): Subscriber[Int] =
|
||||
Flow[Int].to(Sink.fold(0)(_ + _)).runWith(Source.asSubscriber)
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
}
|
||||
|
|
@ -0,0 +1,20 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Subscriber
|
||||
|
||||
import org.apache.pekko.stream.scaladsl._
|
||||
|
||||
class ForeachSinkSubscriberTest extends PekkoSubscriberBlackboxVerification[Int] {
|
||||
|
||||
override def createSubscriber(): Subscriber[Int] =
|
||||
Flow[Int]
|
||||
.to(Sink.foreach { _ =>
|
||||
})
|
||||
.runWith(Source.asSubscriber)
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
}
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Processor
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream._
|
||||
import pekko.stream.scaladsl.Flow
|
||||
|
||||
class FusableProcessorTest extends PekkoIdentityProcessorVerification[Int] {
|
||||
|
||||
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
|
||||
Flow[Int]
|
||||
.map(identity)
|
||||
.toProcessor
|
||||
.withAttributes(Attributes.inputBuffer(initial = maxBufferSize / 2, max = maxBufferSize))
|
||||
.run()
|
||||
}
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,25 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.concurrent.Promise
|
||||
|
||||
import org.reactivestreams._
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class FuturePublisherTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val p = Promise[Int]()
|
||||
val pub = Source.future(p.future).runWith(Sink.asPublisher(false))
|
||||
p.success(0)
|
||||
pub
|
||||
}
|
||||
|
||||
override def maxElementsFromPublisher(): Long = 1
|
||||
}
|
||||
|
|
@ -0,0 +1,29 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.impl.EmptyPublisher
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class GroupByTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] =
|
||||
if (elements == 0) EmptyPublisher[Int]
|
||||
else {
|
||||
val futureGroupSource =
|
||||
Source(iterable(elements)).groupBy(1, _ => "all").prefixAndTail(0).map(_._2).concatSubstreams.runWith(Sink.head)
|
||||
val groupSource = Await.result(futureGroupSource, 3.seconds)
|
||||
groupSource.runWith(Sink.asPublisher(false))
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.io.InputStream
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.ActorAttributes
|
||||
import pekko.stream.scaladsl.{ Sink, StreamConverters }
|
||||
import pekko.util.ByteString
|
||||
|
||||
class InputStreamSourceTest extends PekkoPublisherVerification[ByteString] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[ByteString] = {
|
||||
StreamConverters
|
||||
.fromInputStream(() =>
|
||||
new InputStream {
|
||||
@volatile var num = 0
|
||||
override def read(): Int = {
|
||||
num += 1
|
||||
num
|
||||
}
|
||||
})
|
||||
.withAttributes(ActorAttributes.dispatcher("pekko.test.stream-dispatcher"))
|
||||
.take(elements)
|
||||
.runWith(Sink.asPublisher(false))
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams._
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class IterablePublisherTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
override def createPublisher(elements: Long): Publisher[Int] = {
|
||||
Source(iterable(elements)).runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Processor
|
||||
|
||||
import org.apache.pekko.stream.scaladsl.Flow
|
||||
|
||||
class MapTest extends PekkoIdentityProcessorVerification[Int] {
|
||||
|
||||
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
|
||||
Flow[Int].map(elem => elem).named("identity").toProcessor.run()
|
||||
}
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,20 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko.stream.scaladsl.{ Keep, Sink, Source }
|
||||
|
||||
class MaybeSourceTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val (p, pub) = Source.maybe[Int].toMat(Sink.asPublisher(false))(Keep.both).run()
|
||||
p.success(Some(1))
|
||||
pub
|
||||
}
|
||||
|
||||
override def maxElementsFromPublisher(): Long = 1
|
||||
}
|
||||
|
|
@ -0,0 +1,55 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.util.concurrent.ExecutorService
|
||||
import java.util.concurrent.Executors
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
import org.reactivestreams.{ Processor, Publisher, Subscriber, Subscription }
|
||||
import org.reactivestreams.tck.IdentityProcessorVerification
|
||||
import org.reactivestreams.tck.TestEnvironment
|
||||
import org.scalatestplus.testng.TestNGSuiteLike
|
||||
import org.testng.annotations.AfterClass
|
||||
|
||||
import org.apache.pekko.stream.testkit.TestPublisher
|
||||
|
||||
abstract class PekkoIdentityProcessorVerification[T](env: TestEnvironment, publisherShutdownTimeout: Long)
|
||||
extends IdentityProcessorVerification[T](env, publisherShutdownTimeout)
|
||||
with TestNGSuiteLike
|
||||
with ActorSystemLifecycle {
|
||||
|
||||
def this(printlnDebug: Boolean) =
|
||||
this(
|
||||
new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug),
|
||||
Timeouts.publisherShutdownTimeoutMillis)
|
||||
|
||||
def this() = this(false)
|
||||
|
||||
override def createFailedPublisher(): Publisher[T] =
|
||||
TestPublisher.error(new Exception("Unable to serve subscribers right now!"))
|
||||
|
||||
def processorFromSubscriberAndPublisher(sub: Subscriber[T], pub: Publisher[T]): Processor[T, T] = {
|
||||
new Processor[T, T] {
|
||||
override def onSubscribe(s: Subscription): Unit = sub.onSubscribe(s)
|
||||
override def onError(t: Throwable): Unit = sub.onError(t)
|
||||
override def onComplete(): Unit = sub.onComplete()
|
||||
override def onNext(t: T): Unit = sub.onNext(t)
|
||||
override def subscribe(s: Subscriber[_ >: T]): Unit = pub.subscribe(s)
|
||||
}
|
||||
}
|
||||
|
||||
/** By default Akka Publishers do not support Fanout! */
|
||||
override def maxSupportedSubscribers: Long = 1L
|
||||
|
||||
override lazy val publisherExecutorService: ExecutorService =
|
||||
Executors.newFixedThreadPool(3)
|
||||
|
||||
@AfterClass
|
||||
def shutdownPublisherExecutorService(): Unit = {
|
||||
publisherExecutorService.shutdown()
|
||||
publisherExecutorService.awaitTermination(3, TimeUnit.SECONDS)
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.reactivestreams.Publisher
|
||||
import org.reactivestreams.tck.PublisherVerification
|
||||
import org.reactivestreams.tck.TestEnvironment
|
||||
import org.scalatestplus.testng.TestNGSuiteLike
|
||||
|
||||
import org.apache.pekko.stream.testkit.TestPublisher
|
||||
|
||||
abstract class PekkoPublisherVerification[T](val env: TestEnvironment, publisherShutdownTimeout: Long)
|
||||
extends PublisherVerification[T](env, publisherShutdownTimeout)
|
||||
with TestNGSuiteLike
|
||||
with ActorSystemLifecycle {
|
||||
|
||||
override def additionalConfig: Config =
|
||||
ConfigFactory.parseString("""
|
||||
pekko.stream.materializer.initial-input-buffer-size = 512
|
||||
pekko.stream.materializer.max-input-buffer-size = 512
|
||||
""")
|
||||
|
||||
def this(printlnDebug: Boolean) =
|
||||
this(
|
||||
new TestEnvironment(Timeouts.defaultTimeoutMillis, Timeouts.defaultNoSignalsTimeoutMillis, printlnDebug),
|
||||
Timeouts.publisherShutdownTimeoutMillis)
|
||||
|
||||
def this() = this(false)
|
||||
|
||||
override def createFailedPublisher(): Publisher[T] =
|
||||
TestPublisher.error(new Exception("Unable to serve subscribers right now!"))
|
||||
|
||||
def iterable(elements: Long): immutable.Iterable[Int] =
|
||||
if (elements > Int.MaxValue)
|
||||
new immutable.Iterable[Int] { override def iterator = Iterator.from(0) }
|
||||
else
|
||||
0 until elements.toInt
|
||||
}
|
||||
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class PrefixAndTailTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] = {
|
||||
val futureTailSource = Source(iterable(elements)).prefixAndTail(0).map { case (_, tail) => tail }.runWith(Sink.head)
|
||||
val tailSource = Await.result(futureTailSource, 3.seconds)
|
||||
tailSource.runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class SingleElementSourceTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] =
|
||||
Source.single(1).runWith(Sink.asPublisher(false))
|
||||
|
||||
override def maxElementsFromPublisher(): Long = 1
|
||||
}
|
||||
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import java.lang.{ Integer => JInt }
|
||||
|
||||
import scala.concurrent.Promise
|
||||
|
||||
import org.reactivestreams.{ Subscriber, Subscription }
|
||||
import org.reactivestreams.tck.{ SubscriberWhiteboxVerification, TestEnvironment }
|
||||
import org.reactivestreams.tck.SubscriberWhiteboxVerification.{ SubscriberPuppet, WhiteboxSubscriberProbe }
|
||||
import org.scalatestplus.testng.TestNGSuiteLike
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.Done
|
||||
import pekko.stream.impl.SinkholeSubscriber
|
||||
|
||||
class SinkholeSubscriberTest extends SubscriberWhiteboxVerification[JInt](new TestEnvironment()) with TestNGSuiteLike {
|
||||
override def createSubscriber(probe: WhiteboxSubscriberProbe[JInt]): Subscriber[JInt] = {
|
||||
new Subscriber[JInt] {
|
||||
val hole = new SinkholeSubscriber[JInt](Promise[Done]())
|
||||
|
||||
override def onError(t: Throwable): Unit = {
|
||||
hole.onError(t)
|
||||
probe.registerOnError(t)
|
||||
}
|
||||
|
||||
override def onSubscribe(s: Subscription): Unit = {
|
||||
probe.registerOnSubscribe(new SubscriberPuppet() {
|
||||
override def triggerRequest(elements: Long): Unit = s.request(elements)
|
||||
override def signalCancel(): Unit = s.cancel()
|
||||
})
|
||||
hole.onSubscribe(s)
|
||||
}
|
||||
|
||||
override def onComplete(): Unit = {
|
||||
hole.onComplete()
|
||||
probe.registerOnComplete()
|
||||
}
|
||||
|
||||
override def onNext(t: JInt): Unit = {
|
||||
hole.onNext(t)
|
||||
probe.registerOnNext(t)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def createElement(element: Int): JInt = element
|
||||
}
|
||||
|
|
@ -0,0 +1,28 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.impl.EmptyPublisher
|
||||
import pekko.stream.scaladsl.Sink
|
||||
import pekko.stream.scaladsl.Source
|
||||
|
||||
class SplitWhenTest extends PekkoPublisherVerification[Int] {
|
||||
|
||||
def createPublisher(elements: Long): Publisher[Int] =
|
||||
if (elements == 0) EmptyPublisher[Int]
|
||||
else {
|
||||
val futureSource =
|
||||
Source(iterable(elements)).splitWhen(_ => false).prefixAndTail(0).map(_._2).concatSubstreams.runWith(Sink.head)
|
||||
val source = Await.result(futureSource, 3.seconds)
|
||||
source.runWith(Sink.asPublisher(false))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,18 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
/**
|
||||
* Specifies timeouts for the TCK
|
||||
*/
|
||||
object Timeouts {
|
||||
|
||||
def publisherShutdownTimeoutMillis: Int = 3000
|
||||
|
||||
def defaultTimeoutMillis: Int = 800
|
||||
|
||||
def defaultNoSignalsTimeoutMillis: Int = 200
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,39 @@
|
|||
/*
|
||||
* Copyright (C) 2014-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Processor
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.Attributes
|
||||
import pekko.stream.impl.fusing.GraphStages.SimpleLinearGraphStage
|
||||
import pekko.stream.scaladsl.Flow
|
||||
import pekko.stream.stage.GraphStageLogic
|
||||
import pekko.stream.stage.InHandler
|
||||
import pekko.stream.stage.OutHandler
|
||||
|
||||
class TransformProcessorTest extends PekkoIdentityProcessorVerification[Int] {
|
||||
|
||||
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
|
||||
val stage =
|
||||
new SimpleLinearGraphStage[Int] {
|
||||
override def createLogic(inheritedAttributes: Attributes) =
|
||||
new GraphStageLogic(shape) with InHandler with OutHandler {
|
||||
override def onPush(): Unit = push(out, grab(in))
|
||||
override def onPull(): Unit = pull(in)
|
||||
setHandlers(in, out, this)
|
||||
}
|
||||
}
|
||||
|
||||
Flow[Int]
|
||||
.via(stage)
|
||||
.toProcessor
|
||||
.withAttributes(Attributes.inputBuffer(initial = maxBufferSize / 2, max = maxBufferSize))
|
||||
.run()
|
||||
}
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,34 @@
|
|||
/*
|
||||
* Copyright (C) 2015-2022 Lightbend Inc. <https://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package org.apache.pekko.stream.tck
|
||||
|
||||
import org.reactivestreams.Processor
|
||||
|
||||
import org.apache.pekko
|
||||
import pekko.stream.impl.VirtualProcessor
|
||||
import pekko.stream.scaladsl.Flow
|
||||
|
||||
class VirtualProcessorTest extends PekkoIdentityProcessorVerification[Int] {
|
||||
|
||||
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] = {
|
||||
val identity = Flow[Int].map(elem => elem).named("identity").toProcessor.run()
|
||||
val left, right = new VirtualProcessor[Int]
|
||||
left.subscribe(identity)
|
||||
identity.subscribe(right)
|
||||
processorFromSubscriberAndPublisher(left, right)
|
||||
}
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
}
|
||||
|
||||
class VirtualProcessorSingleTest extends PekkoIdentityProcessorVerification[Int] {
|
||||
|
||||
override def createIdentityProcessor(maxBufferSize: Int): Processor[Int, Int] =
|
||||
new VirtualProcessor[Int]
|
||||
|
||||
override def createElement(element: Int): Int = element
|
||||
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue