2015-07-12 23:04:26 -04:00
|
|
|
/**
|
2016-02-23 12:58:39 +01:00
|
|
|
* Copyright (C) 2015-2016 Lightbend Inc. <http://www.lightbend.com>
|
2015-07-12 23:04:26 -04:00
|
|
|
*/
|
|
|
|
|
package akka.stream.io
|
|
|
|
|
|
|
|
|
|
import java.io.{ IOException, InputStream }
|
|
|
|
|
import java.util.concurrent.TimeoutException
|
2016-01-26 11:47:30 -05:00
|
|
|
import akka.actor.ActorSystem
|
2015-07-12 23:04:26 -04:00
|
|
|
import akka.stream._
|
2016-02-15 13:38:37 +01:00
|
|
|
import akka.stream.Attributes.inputBuffer
|
2015-07-12 23:04:26 -04:00
|
|
|
import akka.stream.impl.StreamSupervisor.Children
|
|
|
|
|
import akka.stream.impl.io.InputStreamSinkStage
|
|
|
|
|
import akka.stream.impl.{ ActorMaterializerImpl, StreamSupervisor }
|
2016-01-26 11:47:30 -05:00
|
|
|
import akka.stream.scaladsl.{ Keep, Source, StreamConverters }
|
2015-07-12 23:04:26 -04:00
|
|
|
import akka.stream.testkit.Utils._
|
|
|
|
|
import akka.stream.testkit.scaladsl.TestSource
|
2016-02-25 14:27:45 +01:00
|
|
|
import akka.stream.testkit.{ GraphStageMessages, TestSinkStage }
|
2015-07-12 23:04:26 -04:00
|
|
|
import akka.testkit.TestProbe
|
|
|
|
|
import akka.util.ByteString
|
|
|
|
|
import scala.concurrent.duration._
|
2016-01-02 21:01:04 +01:00
|
|
|
import scala.concurrent.forkjoin.ThreadLocalRandom
|
2015-07-12 23:04:26 -04:00
|
|
|
import scala.concurrent.{ Await, Future }
|
|
|
|
|
import scala.util.control.NoStackTrace
|
2016-02-25 14:27:45 +01:00
|
|
|
import akka.testkit.AkkaSpec
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
class InputStreamSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|
|
|
|
import system.dispatcher
|
|
|
|
|
|
|
|
|
|
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
|
|
|
|
|
implicit val materializer = ActorMaterializer(settings)
|
|
|
|
|
|
|
|
|
|
val timeout = 300.milliseconds
|
2016-01-02 21:01:04 +01:00
|
|
|
def randomByteString(size: Int): ByteString = {
|
2015-07-12 23:04:26 -04:00
|
|
|
val a = new Array[Byte](size)
|
2016-01-02 21:01:04 +01:00
|
|
|
ThreadLocalRandom.current().nextBytes(a)
|
|
|
|
|
ByteString(a)
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|
|
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
val byteString = randomByteString(3)
|
|
|
|
|
val byteArray = byteString.toArray
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-26 11:47:30 -05:00
|
|
|
def readN(is: InputStream, n: Int): (Int, ByteString) = {
|
2016-01-02 21:01:04 +01:00
|
|
|
val buf = new Array[Byte](n)
|
|
|
|
|
val r = is.read(buf)
|
|
|
|
|
(r, ByteString.fromArray(buf, 0, r))
|
2015-12-18 00:45:52 -05:00
|
|
|
}
|
2016-01-26 11:47:30 -05:00
|
|
|
def testSink(probe: TestProbe) = TestSinkStage(new InputStreamSinkStage(timeout), probe)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
"InputStreamSink" must {
|
|
|
|
|
"read bytes from InputStream" in assertAllStagesStopped {
|
2015-12-18 00:45:52 -05:00
|
|
|
val inputStream = Source.single(byteString).runWith(StreamConverters.asInputStream())
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, byteString.size) should ===((byteString.size, byteString))
|
2015-07-12 23:04:26 -04:00
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"read bytes correctly if requested by InputStream not in chunk size" in assertAllStagesStopped {
|
|
|
|
|
val sinkProbe = TestProbe()
|
2016-01-02 21:01:04 +01:00
|
|
|
val byteString2 = randomByteString(3)
|
2016-01-26 11:47:30 -05:00
|
|
|
val inputStream = Source(byteString :: byteString2 :: Nil)
|
|
|
|
|
.runWith(testSink(sinkProbe))
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsgAllOf(GraphStageMessages.Push, GraphStageMessages.Push)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, 2) should ===((2, byteString.take(2)))
|
|
|
|
|
readN(inputStream, 2) should ===((2, byteString.drop(2) ++ byteString2.take(1)))
|
|
|
|
|
readN(inputStream, 2) should ===((2, byteString2.drop(1)))
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"returns less than was expected when the data source has provided some but not enough data" in assertAllStagesStopped {
|
2016-01-02 21:01:04 +01:00
|
|
|
val inputStream = Source.single(byteString).runWith(StreamConverters.asInputStream())
|
2015-12-18 00:45:52 -05:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
val arr = new Array[Byte](byteString.size + 1)
|
|
|
|
|
inputStream.read(arr) should ===(arr.size - 1)
|
|
|
|
|
ByteString(arr) should ===(byteString :+ 0)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"block read until get requested number of bytes from upstream" in assertAllStagesStopped {
|
2015-12-08 18:47:58 +01:00
|
|
|
val (probe, inputStream) = TestSource.probe[ByteString].toMat(StreamConverters.asInputStream())(Keep.both).run()
|
2016-01-02 21:01:04 +01:00
|
|
|
val f = Future(inputStream.read(new Array[Byte](byteString.size)))
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
the[Exception] thrownBy Await.result(f, timeout) shouldBe a[TimeoutException]
|
|
|
|
|
probe.sendNext(byteString)
|
2016-01-02 21:01:04 +01:00
|
|
|
Await.result(f, timeout) should ===(byteString.size)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
probe.sendComplete()
|
2016-01-02 21:01:04 +01:00
|
|
|
inputStream.read() should ===(-1)
|
2015-07-12 23:04:26 -04:00
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"fill up buffer by default" in assertAllStagesStopped {
|
2016-01-02 21:01:04 +01:00
|
|
|
val byteString2 = randomByteString(3)
|
|
|
|
|
val inputStream = Source(byteString :: byteString2 :: Nil).runWith(StreamConverters.asInputStream())
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, 3) should ===((3, byteString))
|
|
|
|
|
readN(inputStream, 3) should ===((3, byteString2))
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"throw error when reactive stream is closed" in assertAllStagesStopped {
|
2015-12-08 18:47:58 +01:00
|
|
|
val (probe, inputStream) = TestSource.probe[ByteString].toMat(StreamConverters.asInputStream())(Keep.both).run()
|
2015-07-12 23:04:26 -04:00
|
|
|
probe.sendNext(byteString)
|
|
|
|
|
inputStream.close()
|
|
|
|
|
probe.expectCancellation()
|
2016-01-02 21:01:04 +01:00
|
|
|
the[Exception] thrownBy inputStream.read() shouldBe a[IOException]
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"return all data when upstream is completed" in assertAllStagesStopped {
|
|
|
|
|
val sinkProbe = TestProbe()
|
2016-01-26 11:47:30 -05:00
|
|
|
val (probe, inputStream) = TestSource.probe[ByteString]
|
|
|
|
|
.toMat(testSink(sinkProbe))(Keep.both).run()
|
2016-01-02 21:01:04 +01:00
|
|
|
val bytes = randomByteString(1)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
probe.sendNext(bytes)
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsg(GraphStageMessages.Push)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
probe.sendComplete()
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsg(GraphStageMessages.UpstreamFinish)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, 3) should ===((1, bytes))
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|
|
|
|
|
|
2015-12-18 00:45:52 -05:00
|
|
|
"work when read chunks smaller than stream chunks" in assertAllStagesStopped {
|
2016-01-02 21:01:04 +01:00
|
|
|
val bytes = randomByteString(10)
|
2015-12-18 00:45:52 -05:00
|
|
|
val inputStream = Source.single(bytes).runWith(StreamConverters.asInputStream())
|
|
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
for (expect ← bytes.sliding(3, 3))
|
|
|
|
|
readN(inputStream, 3) should ===((expect.size, expect))
|
|
|
|
|
|
2015-12-18 00:45:52 -05:00
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"throw exception when call read with wrong parameters" in assertAllStagesStopped {
|
|
|
|
|
val inputStream = Source.single(byteString).runWith(StreamConverters.asInputStream())
|
2016-01-02 21:01:04 +01:00
|
|
|
val buf = new Array[Byte](3)
|
|
|
|
|
an[IllegalArgumentException] shouldBe thrownBy(inputStream.read(buf, -1, 2))
|
|
|
|
|
an[IllegalArgumentException] shouldBe thrownBy(inputStream.read(buf, 0, 5))
|
2015-12-18 00:45:52 -05:00
|
|
|
an[IllegalArgumentException] shouldBe thrownBy(inputStream.read(new Array[Byte](0), 0, 1))
|
2016-01-02 21:01:04 +01:00
|
|
|
an[IllegalArgumentException] shouldBe thrownBy(inputStream.read(buf, 0, 0))
|
2015-12-18 00:45:52 -05:00
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"successfully read several chunks at once" in assertAllStagesStopped {
|
2016-01-02 21:01:04 +01:00
|
|
|
val bytes = List.fill(4)(randomByteString(4))
|
2015-12-18 00:45:52 -05:00
|
|
|
val sinkProbe = TestProbe()
|
2016-01-26 11:47:30 -05:00
|
|
|
val inputStream = Source[ByteString](bytes)
|
|
|
|
|
.runWith(testSink(sinkProbe))
|
2015-12-18 00:45:52 -05:00
|
|
|
|
|
|
|
|
//need to wait while all elements arrive to sink
|
2016-01-26 11:47:30 -05:00
|
|
|
bytes foreach { _ ⇒ sinkProbe.expectMsg(GraphStageMessages.Push) }
|
2015-12-18 00:45:52 -05:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
for (i ← 0 to 1)
|
|
|
|
|
readN(inputStream, 8) should ===((8, bytes(i * 2) ++ bytes(i * 2 + 1)))
|
2015-12-18 00:45:52 -05:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"work when read chunks bigger than stream chunks" in assertAllStagesStopped {
|
2016-01-02 21:01:04 +01:00
|
|
|
val bytes1 = randomByteString(10)
|
|
|
|
|
val bytes2 = randomByteString(10)
|
2015-12-18 00:45:52 -05:00
|
|
|
val sinkProbe = TestProbe()
|
|
|
|
|
val inputStream = Source(bytes1 :: bytes2 :: Nil).runWith(testSink(sinkProbe))
|
|
|
|
|
|
|
|
|
|
//need to wait while both elements arrive to sink
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsgAllOf(GraphStageMessages.Push, GraphStageMessages.Push)
|
2015-12-18 00:45:52 -05:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, 15) should ===((15, bytes1 ++ bytes2.take(5)))
|
|
|
|
|
readN(inputStream, 15) should ===((5, bytes2.drop(5)))
|
2015-12-18 00:45:52 -05:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
2015-07-12 23:04:26 -04:00
|
|
|
"return -1 when read after stream is completed" in assertAllStagesStopped {
|
2015-12-18 00:45:52 -05:00
|
|
|
val inputStream = Source.single(byteString).runWith(StreamConverters.asInputStream())
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, byteString.size) should ===((byteString.size, byteString))
|
|
|
|
|
inputStream.read() should ===(-1)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"return IOException when stream is failed" in assertAllStagesStopped {
|
|
|
|
|
val sinkProbe = TestProbe()
|
|
|
|
|
val (probe, inputStream) = TestSource.probe[ByteString].toMat(testSink(sinkProbe))(Keep.both).run()
|
|
|
|
|
val ex = new RuntimeException("Stream failed.") with NoStackTrace
|
|
|
|
|
|
|
|
|
|
probe.sendNext(byteString)
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsg(GraphStageMessages.Push)
|
2015-07-12 23:04:26 -04:00
|
|
|
|
2016-01-02 21:01:04 +01:00
|
|
|
readN(inputStream, byteString.size) should ===((byteString.size, byteString))
|
2015-07-12 23:04:26 -04:00
|
|
|
|
|
|
|
|
probe.sendError(ex)
|
2016-01-26 11:47:30 -05:00
|
|
|
sinkProbe.expectMsg(GraphStageMessages.Failure(ex))
|
2016-01-02 21:01:04 +01:00
|
|
|
val e = intercept[IOException] { Await.result(Future(inputStream.read()), timeout) }
|
|
|
|
|
e.getCause should ===(ex)
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"use dedicated default-blocking-io-dispatcher by default" in assertAllStagesStopped {
|
|
|
|
|
val sys = ActorSystem("dispatcher-testing", UnboundedMailboxConfig)
|
2015-12-11 14:45:24 +01:00
|
|
|
val materializer = ActorMaterializer()(sys)
|
2015-07-12 23:04:26 -04:00
|
|
|
try {
|
2015-12-08 18:47:58 +01:00
|
|
|
TestSource.probe[ByteString].runWith(StreamConverters.asInputStream())(materializer)
|
2015-12-11 14:45:24 +01:00
|
|
|
materializer.asInstanceOf[ActorMaterializerImpl].supervisor.tell(StreamSupervisor.GetChildren, testActor)
|
2015-11-17 13:17:30 +01:00
|
|
|
val ref = expectMsgType[Children].children.find(_.path.toString contains "inputStreamSink").get
|
2015-07-12 23:04:26 -04:00
|
|
|
assertDispatcher(ref, "akka.stream.default-blocking-io-dispatcher")
|
|
|
|
|
} finally shutdown(sys)
|
|
|
|
|
}
|
2016-02-01 14:08:46 +01:00
|
|
|
|
|
|
|
|
"work when more bytes pulled from InputStream than available" in assertAllStagesStopped {
|
|
|
|
|
val inputStream = Source.single(byteString).runWith(StreamConverters.asInputStream())
|
|
|
|
|
|
|
|
|
|
readN(inputStream, byteString.size * 2) should ===((byteString.size, byteString))
|
|
|
|
|
inputStream.read() should ===(-1)
|
|
|
|
|
|
|
|
|
|
inputStream.close()
|
|
|
|
|
}
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|
2016-02-15 13:38:37 +01:00
|
|
|
|
|
|
|
|
"fail to materialize with zero sized input buffer" in {
|
|
|
|
|
an[IllegalArgumentException] shouldBe thrownBy {
|
|
|
|
|
Source.single(byteString)
|
|
|
|
|
.runWith(StreamConverters.asInputStream(timeout).withAttributes(inputBuffer(0, 0)))
|
|
|
|
|
/*
|
|
|
|
|
With Source.single we test the code path in which the sink
|
|
|
|
|
itself throws an exception when being materialized. If
|
|
|
|
|
Source.empty is used, the same exception is thrown by
|
|
|
|
|
Materializer.
|
|
|
|
|
*/
|
|
|
|
|
}
|
|
|
|
|
}
|
2015-07-12 23:04:26 -04:00
|
|
|
}
|