use blocking-io-dispatcher correctly in OutputStreamSource, #20666
* it used the materializer dispatcher for the blocking take * it was leaking threads (blocking threads) when when materializer was shutdown abruptly
This commit is contained in:
parent
be448e9fbb
commit
b97a72c773
4 changed files with 74 additions and 20 deletions
|
|
@ -29,7 +29,7 @@ public class OutputStreamSinkTest extends StreamTest {
|
|||
}
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("OutputStreamSink",
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("OutputStreamSinkTest",
|
||||
Utils.UnboundedMailboxConfig());
|
||||
@Test
|
||||
public void mustSignalFailureViaIoResult() throws Exception {
|
||||
|
|
@ -44,7 +44,7 @@ public class OutputStreamSinkTest extends StreamTest {
|
|||
}
|
||||
};
|
||||
final CompletionStage<IOResult> resultFuture = Source.single(ByteString.fromString("123456")).runWith(StreamConverters.fromOutputStream(() -> os), materializer);
|
||||
final IOResult result = resultFuture.toCompletableFuture().get(300, TimeUnit.MILLISECONDS);
|
||||
final IOResult result = resultFuture.toCompletableFuture().get(3000, TimeUnit.MILLISECONDS);
|
||||
|
||||
assertFalse(result.wasSuccessful());
|
||||
assertTrue(result.getError().getMessage().equals("Can't accept more data."));
|
||||
|
|
|
|||
|
|
@ -29,11 +29,11 @@ public class OutputStreamSourceTest extends StreamTest {
|
|||
}
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("OutputStreamSource",
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("OutputStreamSourceTest2",
|
||||
Utils.UnboundedMailboxConfig());
|
||||
@Test
|
||||
public void mustSendEventsViaOutputStream() throws Exception {
|
||||
final FiniteDuration timeout = FiniteDuration.create(300, TimeUnit.MILLISECONDS);
|
||||
final FiniteDuration timeout = FiniteDuration.create(3000, TimeUnit.MILLISECONDS);
|
||||
final JavaTestKit probe = new JavaTestKit(system);
|
||||
|
||||
final Source<ByteString, OutputStream> source = StreamConverters.asOutputStream(timeout);
|
||||
|
|
@ -45,6 +45,8 @@ public class OutputStreamSourceTest extends StreamTest {
|
|||
})).run(materializer);
|
||||
|
||||
s.write("a".getBytes());
|
||||
|
||||
|
||||
assertEquals(ByteString.fromString("a"), probe.receiveOne(timeout));
|
||||
s.close();
|
||||
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
val settings = ActorMaterializerSettings(system).withDispatcher("akka.actor.default-dispatcher")
|
||||
implicit val materializer = ActorMaterializer(settings)
|
||||
|
||||
val timeout = 300.milliseconds
|
||||
val timeout = 3.seconds
|
||||
val bytesArray = Array.fill[Byte](3)(Random.nextInt(1024).asInstanceOf[Byte])
|
||||
val byteString = ByteString(bytesArray)
|
||||
|
||||
|
|
@ -41,6 +41,16 @@ class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
def expectSuccess[T](f: Future[T], value: T) =
|
||||
Await.result(f, remainingOrDefault) should be(value)
|
||||
|
||||
def assertNoBlockedThreads(): Unit = {
|
||||
def threadsBlocked =
|
||||
ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).toSeq
|
||||
.filter(t ⇒ t.getThreadName.startsWith("OutputStreamSourceSpec") &&
|
||||
t.getLockName != null &&
|
||||
t.getLockName.startsWith("java.util.concurrent.locks.AbstractQueuedSynchronizer"))
|
||||
|
||||
awaitAssert(threadsBlocked should ===(Seq()), 3.seconds)
|
||||
}
|
||||
|
||||
"OutputStreamSource" must {
|
||||
"read bytes from OutputStream" in assertAllStagesStopped {
|
||||
val (outputStream, probe) = StreamConverters.asOutputStream().toMat(TestSink.probe[ByteString])(Keep.both).run
|
||||
|
|
@ -156,11 +166,11 @@ class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
.withAttributes(inputBuffer(0, 0))
|
||||
.runWith(Sink.head)
|
||||
/*
|
||||
With Sink.head we test the code path in which the source
|
||||
itself throws an exception when being materialized. If
|
||||
Sink.ignore is used, the same exception is thrown by
|
||||
Materializer.
|
||||
*/
|
||||
With Sink.head we test the code path in which the source
|
||||
itself throws an exception when being materialized. If
|
||||
Sink.ignore is used, the same exception is thrown by
|
||||
Materializer.
|
||||
*/
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -175,13 +185,22 @@ class OutputStreamSourceSpec extends AkkaSpec(UnboundedMailboxConfig) {
|
|||
sub.request(1)
|
||||
sub.cancel()
|
||||
|
||||
def threadsBlocked =
|
||||
ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).toSeq
|
||||
.filter(t ⇒ t.getThreadName.startsWith("OutputStreamSourceSpec") &&
|
||||
t.getLockName != null &&
|
||||
t.getLockName.startsWith("java.util.concurrent.locks.AbstractQueuedSynchronizer"))
|
||||
assertNoBlockedThreads()
|
||||
}
|
||||
|
||||
awaitAssert(threadsBlocked should ===(Seq()), 3.seconds)
|
||||
"not leave blocked threads when materializer shutdown" in {
|
||||
val materializer2 = ActorMaterializer(settings)
|
||||
val (outputStream, probe) = StreamConverters.asOutputStream(timeout)
|
||||
.toMat(TestSink.probe[ByteString])(Keep.both).run()(materializer2)
|
||||
|
||||
val sub = probe.expectSubscription()
|
||||
|
||||
// triggers a blocking read on the queue
|
||||
// and then shutdown the materializer before we got anything
|
||||
sub.request(1)
|
||||
materializer2.shutdown()
|
||||
|
||||
assertNoBlockedThreads()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,18 +6,21 @@ package akka.stream.impl.io
|
|||
import java.io.{ IOException, OutputStream }
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import java.util.concurrent.{ BlockingQueue, LinkedBlockingQueue }
|
||||
|
||||
import akka.stream.{ Outlet, SourceShape, Attributes }
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
import akka.stream.impl.io.OutputStreamSourceStage._
|
||||
import akka.stream.stage._
|
||||
import akka.util.ByteString
|
||||
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.{ Await, Future, Promise }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import akka.stream.ActorAttributes
|
||||
import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import scala.concurrent.ExecutionContext
|
||||
import akka.stream.ActorMaterializer
|
||||
|
||||
private[stream] object OutputStreamSourceStage {
|
||||
sealed trait AdapterToStageMessage
|
||||
|
|
@ -40,6 +43,9 @@ final private[stream] class OutputStreamSourceStage(writeTimeout: FiniteDuration
|
|||
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, OutputStream) = {
|
||||
val maxBuffer = inheritedAttributes.getAttribute(classOf[InputBuffer], InputBuffer(16, 16)).max
|
||||
|
||||
val dispatcherId = inheritedAttributes.get[Dispatcher](IODispatcher).dispatcher
|
||||
|
||||
require(maxBuffer > 0, "Buffer size must be greater than 0")
|
||||
|
||||
val dataQueue = new LinkedBlockingQueue[ByteString](maxBuffer)
|
||||
|
|
@ -49,6 +55,9 @@ final private[stream] class OutputStreamSourceStage(writeTimeout: FiniteDuration
|
|||
var flush: Option[Promise[Unit]] = None
|
||||
var close: Option[Promise[Unit]] = None
|
||||
|
||||
private var dispatcher: ExecutionContext = null // set in preStart
|
||||
private var blockingThread: Thread = null // for postStop interrupt
|
||||
|
||||
private val downstreamCallback: AsyncCallback[Try[ByteString]] =
|
||||
getAsyncCallback {
|
||||
case Success(elem) ⇒ onPush(elem)
|
||||
|
|
@ -102,6 +111,11 @@ final private[stream] class OutputStreamSourceStage(writeTimeout: FiniteDuration
|
|||
sendResponseIfNeed()
|
||||
}
|
||||
|
||||
override def preStart(): Unit = {
|
||||
dispatcher = ActorMaterializer.downcast(materializer).system.dispatchers.lookup(dispatcherId)
|
||||
super.preStart()
|
||||
}
|
||||
|
||||
setHandler(out, new OutHandler {
|
||||
override def onDownstreamFinish(): Unit = {
|
||||
//assuming there can be no further in messages
|
||||
|
|
@ -112,10 +126,29 @@ final private[stream] class OutputStreamSourceStage(writeTimeout: FiniteDuration
|
|||
completeStage()
|
||||
}
|
||||
override def onPull(): Unit = {
|
||||
implicit val ex = interpreter.materializer.executionContext
|
||||
Future(dataQueue.take()).onComplete(downstreamCallback.invoke)
|
||||
implicit val ec = dispatcher
|
||||
Future {
|
||||
// keep track of the thread for postStop interrupt
|
||||
blockingThread = Thread.currentThread()
|
||||
try {
|
||||
dataQueue.take()
|
||||
} catch {
|
||||
case _: InterruptedException ⇒
|
||||
Thread.interrupted()
|
||||
ByteString()
|
||||
} finally {
|
||||
blockingThread = null
|
||||
}
|
||||
}.onComplete(downstreamCallback.invoke)
|
||||
}
|
||||
})
|
||||
|
||||
override def postStop(): Unit = {
|
||||
// interrupt any pending blocking take
|
||||
if (blockingThread != null)
|
||||
blockingThread.interrupt()
|
||||
super.postStop()
|
||||
}
|
||||
}
|
||||
(logic, new OutputStreamAdapter(dataQueue, downstreamStatus, logic.wakeUp, writeTimeout))
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue