which is used as a materialized value for Sources/Sinks that work with IO.
This commit is contained in:
Martynas Mickevičius 2016-01-21 18:06:42 +02:00
parent 53ad1085ec
commit f2553eb3b6
22 changed files with 278 additions and 92 deletions

View file

@ -46,8 +46,8 @@ class FileSourcesBenchmark {
@Param(Array("2048"))
val bufSize = 0
var fileChannelSource: Source[ByteString, Future[Long]] = _
var fileInputStreamSource: Source[ByteString, Future[Long]] = _
var fileChannelSource: Source[ByteString, Future[IOResult]] = _
var fileInputStreamSource: Source[ByteString, Future[IOResult]] = _
var ioSourceLinesIterator: Source[ByteString, NotUsed] = _
@Setup

View file

@ -11,6 +11,7 @@ import java.io.IOException;
import akka.Done;
import akka.actor.ActorSystem;
import akka.stream.ActorAttributes;
import akka.stream.io.IOResult;
import akka.stream.javadsl.Sink;
import akka.stream.javadsl.FileIO;
import docs.stream.SilenceSystemOut;
@ -58,7 +59,7 @@ public class StreamFileDocTest {
Sink<ByteString, Future<Done>> printlnSink =
Sink.foreach(chunk -> System.out.println(chunk.utf8String()));
Future<Long> bytesWritten =
Future<IOResult> ioResult =
FileIO.fromFile(file)
.to(printlnSink)
.run(mat);
@ -73,7 +74,7 @@ public class StreamFileDocTest {
final File file = File.createTempFile(getClass().getName(), ".tmp");
try {
Sink<ByteString, Future<Long>> byteStringFutureSink =
Sink<ByteString, Future<IOResult>> fileSink =
//#custom-dispatcher-code
FileIO.toFile(file)
.withAttributes(ActorAttributes.dispatcher("custom-blocking-io-dispatcher"));

View file

@ -73,8 +73,8 @@ In Akka 2.4.x this is formulated like so:
.. includecode:: ../code/docs/stream/MigrationsJava.java#expand-state
Changed Sinks
=============
Changed Sources / Sinks
=======================
Sink.asPublisher is now configured using an enum
------------------------------------------------
@ -85,3 +85,16 @@ In order to not use a meaningless boolean parameter we have changed the signatur
.. includecode:: ../code/docs/stream/MigrationsJava.java#asPublisher
IO Sources / Sinks materialize IOResult
---------------------------------------
Materialized values of the following sources and sinks:
* ``FileIO.fromFile``
* ``FileIO.toFile``
* ``StreamConverters.fromInputStream``
* ``StreamConverters.fromOutputStream``
have been changed from ``Long`` to ``akka.stream.io.IOResult``.
This allows to signal more complicated completion scenarios. For example, on failure it is now possible
to return the exception and the number of bytes written until that exception occured.

View file

@ -6,6 +6,7 @@ package docs.stream.io
import java.io.File
import akka.stream._
import akka.stream.io.IOResult
import akka.stream.scaladsl.{ FileIO, Sink, Source }
import akka.stream.testkit.Utils._
import akka.stream.testkit._
@ -45,7 +46,7 @@ class StreamFileDocSpec extends AkkaSpec(UnboundedMailboxConfig) {
//#file-source
val foreach: Future[Long] = FileIO.fromFile(file)
val foreach: Future[IOResult] = FileIO.fromFile(file)
.to(Sink.ignore)
.run()
//#file-source

View file

@ -71,3 +71,19 @@ In Akka 2.4.x this is formulated like so:
.. includecode:: ../code/docs/stream/MigrationsScala.scala#expand-state
Changed Sources / Sinks
=======================
IO Sources / Sinks materialize IOResult
---------------------------------------
Materialized values of the following sources and sinks:
* ``FileIO.fromFile``
* ``FileIO.toFile``
* ``StreamConverters.fromInputStream``
* ``StreamConverters.fromOutputStream``
have been changed from ``Long`` to ``akka.stream.io.IOResult``.
This allows to signal more complicated completion scenarios. For example, on failure it is now possible
to return the exception and the number of bytes written until that exception occured.

View file

@ -4,10 +4,12 @@
package akka.stream;
import org.scalatest.junit.JUnitSuite;
import akka.actor.ActorSystem;
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
public abstract class StreamTest {
public abstract class StreamTest extends JUnitSuite {
final protected ActorSystem system;
final protected ActorMaterializer materializer;

View file

@ -0,0 +1,53 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.io;
import akka.stream.StreamTest;
import akka.stream.javadsl.AkkaJUnitActorSystemResource;
import akka.stream.javadsl.Source;
import akka.stream.javadsl.StreamConverters;
import akka.stream.testkit.Utils;
import akka.util.ByteString;
import org.junit.ClassRule;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.Future;
import scala.concurrent.duration.FiniteDuration;
import java.io.OutputStream;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
public class OutputStreamSinkTest extends StreamTest {
public OutputStreamSinkTest() {
super(actorSystemResource);
}
@ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("OutputStreamSink",
Utils.UnboundedMailboxConfig());
@Test
public void mustSignalFailureViaIoResult() throws Exception {
final FiniteDuration timeout = FiniteDuration.create(300, TimeUnit.MILLISECONDS);
final OutputStream os = new OutputStream() {
volatile int left = 3;
public void write(int data) {
if (left == 0) {
throw new RuntimeException("Can't accept more data.");
}
left -= 1;
}
};
final Future<IOResult> resultFuture = Source.single(ByteString.fromString("123456")).runWith(StreamConverters.fromOutputStream(() -> os), materializer);
final IOResult result = Await.result(resultFuture, timeout);
assertFalse(result.wasSuccessful());
assertTrue(result.getError().getMessage().equals("Can't accept more data."));
}
}

View file

@ -4,14 +4,26 @@
package akka.stream.javadsl;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import java.util.Collections;
import org.junit.ClassRule;
import org.junit.Test;
import akka.stream.Attributes;
import akka.stream.StreamTest;
import akka.stream.testkit.AkkaSpec;
public class AttributesTest {
public class AttributesTest extends StreamTest {
public AttributesTest() {
super(actorSystemResource);
}
@ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("AttributesTest",
AkkaSpec.testConf());
final Attributes attributes =
Attributes.name("a")

View file

@ -106,7 +106,7 @@ public class SourceTest extends StreamTest {
probe.expectMsgEquals("a");
probe.expectMsgEquals("b");
probe.expectMsgEquals("c");
probe.expectMsgEquals("()");
probe.expectMsgEquals("Done");
}
@Ignore("StatefulStage to be converted to GraphStage when Java Api is available (#18817)") @Test
@ -300,7 +300,7 @@ public class SourceTest extends StreamTest {
}
}), materializer);
probe.expectMsgClass(NotUsed.class);
probe.expectMsgClass(Done.class);
}
@Test

View file

@ -46,8 +46,8 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
val completion = Source(TestByteStrings)
.runWith(FileIO.toFile(f))
val size = Await.result(completion, 3.seconds)
size should equal(6006)
val result = Await.result(completion, 3.seconds)
result.count should equal(6006)
checkFileContents(f, TestLines.mkString(""))
}
}
@ -64,9 +64,9 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
val lastWrite = List("x" * 100)
val completion2 = write(lastWrite)
val written2 = Await.result(completion2, 3.seconds)
val result = Await.result(completion2, 3.seconds)
written2 should ===(lastWrite.flatten.length)
result.count should ===(lastWrite.flatten.length)
checkFileContents(f, lastWrite.mkString("") + TestLines.mkString("").drop(100))
}
}
@ -79,13 +79,13 @@ class FileSinkSpec extends AkkaSpec(UnboundedMailboxConfig) {
.runWith(FileIO.toFile(f, append = true))
val completion1 = write()
val written1 = Await.result(completion1, 3.seconds)
val result1 = Await.result(completion1, 3.seconds)
val lastWrite = List("x" * 100)
val completion2 = write(lastWrite)
val written2 = Await.result(completion2, 3.seconds)
val result2 = Await.result(completion2, 3.seconds)
f.length() should ===(written1 + written2)
f.length() should ===(result1.count + result2.count)
checkFileContents(f, TestLines.mkString("") + lastWrite.mkString("") + "\n")
}
}

View file

@ -7,17 +7,20 @@ import java.io.File
import java.nio.ByteBuffer
import java.nio.channels.FileChannel
import akka.Done
import akka.actor.{ Deploy, ActorLogging, DeadLetterSuppression, Props }
import akka.stream.actor.ActorPublisherMessage
import akka.stream.io.IOResult
import akka.util.ByteString
import scala.annotation.tailrec
import scala.concurrent.Promise
import scala.util.{ Failure, Success }
import scala.util.control.NonFatal
/** INTERNAL API */
private[akka] object FilePublisher {
def props(f: File, completionPromise: Promise[Long], chunkSize: Int, initialBuffer: Int, maxBuffer: Int) = {
def props(f: File, completionPromise: Promise[IOResult], chunkSize: Int, initialBuffer: Int, maxBuffer: Int) = {
require(chunkSize > 0, s"chunkSize must be > 0 (was $chunkSize)")
require(initialBuffer > 0, s"initialBuffer must be > 0 (was $initialBuffer)")
require(maxBuffer >= initialBuffer, s"maxBuffer must be >= initialBuffer (was $maxBuffer)")
@ -32,7 +35,7 @@ private[akka] object FilePublisher {
}
/** INTERNAL API */
private[akka] final class FilePublisher(f: File, bytesReadPromise: Promise[Long], chunkSize: Int, initialBuffer: Int, maxBuffer: Int)
private[akka] final class FilePublisher(f: File, completionPromise: Promise[IOResult], chunkSize: Int, initialBuffer: Int, maxBuffer: Int)
extends akka.stream.actor.ActorPublisher[ByteString] with ActorLogging {
import FilePublisher._
@ -101,8 +104,14 @@ private[akka] final class FilePublisher(f: File, bytesReadPromise: Promise[Long]
override def postStop(): Unit = {
super.postStop()
bytesReadPromise.trySuccess(readBytesTotal)
try {
if (chan ne null) chan.close()
} catch {
case ex: Exception
completionPromise.success(IOResult(readBytesTotal, Failure(ex)))
}
completionPromise.trySuccess(IOResult(readBytesTotal, Success(Done)))
}
}

View file

@ -7,15 +7,18 @@ import java.io.File
import java.nio.channels.FileChannel
import java.util.Collections
import akka.Done
import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.io.IOResult
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.util.ByteString
import scala.concurrent.Promise
import scala.util.{ Failure, Success }
/** INTERNAL API */
private[akka] object FileSubscriber {
def props(f: File, completionPromise: Promise[Long], bufSize: Int, append: Boolean) = {
def props(f: File, completionPromise: Promise[IOResult], bufSize: Int, append: Boolean) = {
require(bufSize > 0, "buffer size must be > 0")
Props(classOf[FileSubscriber], f, completionPromise, bufSize, append).withDeploy(Deploy.local)
}
@ -26,7 +29,7 @@ private[akka] object FileSubscriber {
}
/** INTERNAL API */
private[akka] class FileSubscriber(f: File, bytesWrittenPromise: Promise[Long], bufSize: Int, append: Boolean)
private[akka] class FileSubscriber(f: File, completionPromise: Promise[IOResult], bufSize: Int, append: Boolean)
extends akka.stream.actor.ActorSubscriber
with ActorLogging {
@ -43,7 +46,7 @@ private[akka] class FileSubscriber(f: File, bytesWrittenPromise: Promise[Long],
super.preStart()
} catch {
case ex: Exception
bytesWrittenPromise.failure(ex)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
cancel()
}
@ -53,12 +56,13 @@ private[akka] class FileSubscriber(f: File, bytesWrittenPromise: Promise[Long],
bytesWritten += chan.write(bytes.asByteBuffer)
} catch {
case ex: Exception
bytesWrittenPromise.failure(ex)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
cancel()
}
case ActorSubscriberMessage.OnError(cause)
log.error(cause, "Tearing down FileSink({}) due to upstream error", f.getAbsolutePath)
case ActorSubscriberMessage.OnError(ex)
log.error(ex, "Tearing down FileSink({}) due to upstream error", f.getAbsolutePath)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
context.stop(self)
case ActorSubscriberMessage.OnComplete
@ -66,15 +70,20 @@ private[akka] class FileSubscriber(f: File, bytesWrittenPromise: Promise[Long],
chan.force(true)
} catch {
case ex: Exception
bytesWrittenPromise.failure(ex)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
}
context.stop(self)
}
override def postStop(): Unit = {
bytesWrittenPromise.trySuccess(bytesWritten)
try {
if (chan ne null) chan.close()
} catch {
case ex: Exception
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
}
completionPromise.trySuccess(IOResult(bytesWritten, Success(Done)))
super.postStop()
}
}

View file

@ -4,6 +4,7 @@
package akka.stream.impl.io
import java.io.{ File, OutputStream }
import akka.stream.io.IOResult
import akka.stream.impl.SinkModule
import akka.stream.impl.StreamLayout.Module
import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
@ -18,21 +19,21 @@ import scala.concurrent.{ Future, Promise }
* (creating it before hand if necessary).
*/
private[akka] final class FileSink(f: File, append: Boolean, val attributes: Attributes, shape: SinkShape[ByteString])
extends SinkModule[ByteString, Future[Long]](shape) {
extends SinkModule[ByteString, Future[IOResult]](shape) {
override def create(context: MaterializationContext) = {
val materializer = ActorMaterializer.downcast(context.materializer)
val settings = materializer.effectiveSettings(context.effectiveAttributes)
val bytesWrittenPromise = Promise[Long]()
val props = FileSubscriber.props(f, bytesWrittenPromise, settings.maxInputBufferSize, append)
val ioResultPromise = Promise[IOResult]()
val props = FileSubscriber.props(f, ioResultPromise, settings.maxInputBufferSize, append)
val dispatcher = context.effectiveAttributes.get[Dispatcher](IODispatcher).dispatcher
val ref = materializer.actorOf(context, props.withDispatcher(dispatcher))
(akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future)
(akka.stream.actor.ActorSubscriber[ByteString](ref), ioResultPromise.future)
}
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[Long]] =
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[IOResult]] =
new FileSink(f, append, attributes, shape)
override def withAttributes(attr: Attributes): Module =
@ -45,22 +46,22 @@ private[akka] final class FileSink(f: File, append: Boolean, val attributes: Att
* (creating it before hand if necessary).
*/
private[akka] final class OutputStreamSink(createOutput: () OutputStream, val attributes: Attributes, shape: SinkShape[ByteString])
extends SinkModule[ByteString, Future[Long]](shape) {
extends SinkModule[ByteString, Future[IOResult]](shape) {
override def create(context: MaterializationContext) = {
val materializer = ActorMaterializer.downcast(context.materializer)
val settings = materializer.effectiveSettings(context.effectiveAttributes)
val bytesWrittenPromise = Promise[Long]()
val ioResultPromise = Promise[IOResult]()
val os = createOutput() // if it fails, we fail the materialization
val props = OutputStreamSubscriber.props(os, bytesWrittenPromise, settings.maxInputBufferSize)
val props = OutputStreamSubscriber.props(os, ioResultPromise, settings.maxInputBufferSize)
val ref = materializer.actorOf(context, props)
(akka.stream.actor.ActorSubscriber[ByteString](ref), bytesWrittenPromise.future)
(akka.stream.actor.ActorSubscriber[ByteString](ref), ioResultPromise.future)
}
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[Long]] =
override protected def newInstance(shape: SinkShape[ByteString]): SinkModule[ByteString, Future[IOResult]] =
new OutputStreamSink(createOutput, attributes, shape)
override def withAttributes(attr: Attributes): Module =

View file

@ -7,6 +7,7 @@ import java.io.{ File, InputStream }
import akka.stream._
import akka.stream.ActorAttributes.Dispatcher
import akka.stream.io.IOResult
import akka.stream.impl.StreamLayout.Module
import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
import akka.stream.impl.{ ErrorPublisher, SourceModule }
@ -19,23 +20,23 @@ import scala.concurrent.{ Future, Promise }
* Creates simple synchronous (Java 6 compatible) Source backed by the given file.
*/
private[akka] final class FileSource(f: File, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
extends SourceModule[ByteString, Future[Long]](shape) {
extends SourceModule[ByteString, Future[IOResult]](shape) {
require(chunkSize > 0, "chunkSize must be greater than 0")
override def create(context: MaterializationContext) = {
// FIXME rewrite to be based on GraphStage rather than dangerous downcasts
val materializer = ActorMaterializer.downcast(context.materializer)
val settings = materializer.effectiveSettings(context.effectiveAttributes)
val bytesReadPromise = Promise[Long]()
val props = FilePublisher.props(f, bytesReadPromise, chunkSize, settings.initialInputBufferSize, settings.maxInputBufferSize)
val ioResultPromise = Promise[IOResult]()
val props = FilePublisher.props(f, ioResultPromise, chunkSize, settings.initialInputBufferSize, settings.maxInputBufferSize)
val dispatcher = context.effectiveAttributes.get[Dispatcher](IODispatcher).dispatcher
val ref = materializer.actorOf(context, props.withDispatcher(dispatcher))
(akka.stream.actor.ActorPublisher[ByteString](ref), bytesReadPromise.future)
(akka.stream.actor.ActorPublisher[ByteString](ref), ioResultPromise.future)
}
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[Long]] =
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[IOResult]] =
new FileSource(f, chunkSize, attributes, shape)
override def withAttributes(attr: Attributes): Module =
@ -47,28 +48,28 @@ private[akka] final class FileSource(f: File, chunkSize: Int, val attributes: At
* Source backed by the given input stream.
*/
private[akka] final class InputStreamSource(createInputStream: () InputStream, chunkSize: Int, val attributes: Attributes, shape: SourceShape[ByteString])
extends SourceModule[ByteString, Future[Long]](shape) {
extends SourceModule[ByteString, Future[IOResult]](shape) {
override def create(context: MaterializationContext) = {
val materializer = ActorMaterializer.downcast(context.materializer)
val bytesReadPromise = Promise[Long]()
val ioResultPromise = Promise[IOResult]()
val pub = try {
val is = createInputStream() // can throw, i.e. FileNotFound
val props = InputStreamPublisher.props(is, bytesReadPromise, chunkSize)
val props = InputStreamPublisher.props(is, ioResultPromise, chunkSize)
val ref = materializer.actorOf(context, props)
akka.stream.actor.ActorPublisher[ByteString](ref)
} catch {
case ex: Exception
bytesReadPromise.failure(ex)
ioResultPromise.failure(ex)
ErrorPublisher(ex, attributes.nameOrDefault("inputStreamSource")).asInstanceOf[Publisher[ByteString]]
}
(pub, bytesReadPromise.future)
(pub, ioResultPromise.future)
}
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[Long]] =
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[IOResult]] =
new InputStreamSource(createInputStream, chunkSize, attributes, shape)
override def withAttributes(attr: Attributes): Module =

View file

@ -5,19 +5,22 @@ package akka.stream.impl.io
import java.io.InputStream
import akka.Done
import akka.actor.{ Deploy, ActorLogging, DeadLetterSuppression, Props }
import akka.io.DirectByteBufferPool
import akka.stream.actor.ActorPublisherMessage
import akka.stream.io.IOResult
import akka.util.ByteString
import akka.util.ByteString.ByteString1C
import scala.annotation.tailrec
import scala.concurrent.Promise
import scala.util.{ Failure, Success }
/** INTERNAL API */
private[akka] object InputStreamPublisher {
def props(is: InputStream, completionPromise: Promise[Long], chunkSize: Int): Props = {
def props(is: InputStream, completionPromise: Promise[IOResult], chunkSize: Int): Props = {
require(chunkSize > 0, s"chunkSize must be > 0 (was $chunkSize)")
Props(classOf[InputStreamPublisher], is, completionPromise, chunkSize).withDeploy(Deploy.local)
@ -27,7 +30,7 @@ private[akka] object InputStreamPublisher {
}
/** INTERNAL API */
private[akka] class InputStreamPublisher(is: InputStream, bytesReadPromise: Promise[Long], chunkSize: Int)
private[akka] class InputStreamPublisher(is: InputStream, completionPromise: Promise[IOResult], chunkSize: Int)
extends akka.stream.actor.ActorPublisher[ByteString]
with ActorLogging {
@ -73,8 +76,14 @@ private[akka] class InputStreamPublisher(is: InputStream, bytesReadPromise: Prom
override def postStop(): Unit = {
super.postStop()
bytesReadPromise.trySuccess(readBytesTotal)
try {
if (is ne null) is.close()
} catch {
case ex: Exception
completionPromise.success(IOResult(readBytesTotal, Failure(ex)))
}
completionPromise.trySuccess(IOResult(readBytesTotal, Success(Done)))
}
}

View file

@ -5,15 +5,18 @@ package akka.stream.impl.io
import java.io.OutputStream
import akka.Done
import akka.actor.{ Deploy, ActorLogging, Props }
import akka.stream.actor.{ ActorSubscriberMessage, WatermarkRequestStrategy }
import akka.stream.io.IOResult
import akka.util.ByteString
import scala.concurrent.Promise
import scala.util.{ Failure, Success }
/** INTERNAL API */
private[akka] object OutputStreamSubscriber {
def props(os: OutputStream, completionPromise: Promise[Long], bufSize: Int) = {
def props(os: OutputStream, completionPromise: Promise[IOResult], bufSize: Int) = {
require(bufSize > 0, "buffer size must be > 0")
Props(classOf[OutputStreamSubscriber], os, completionPromise, bufSize).withDeploy(Deploy.local)
}
@ -21,7 +24,7 @@ private[akka] object OutputStreamSubscriber {
}
/** INTERNAL API */
private[akka] class OutputStreamSubscriber(os: OutputStream, bytesWrittenPromise: Promise[Long], bufSize: Int)
private[akka] class OutputStreamSubscriber(os: OutputStream, completionPromise: Promise[IOResult], bufSize: Int)
extends akka.stream.actor.ActorSubscriber
with ActorLogging {
@ -37,12 +40,13 @@ private[akka] class OutputStreamSubscriber(os: OutputStream, bytesWrittenPromise
bytesWritten += bytes.length
} catch {
case ex: Exception
bytesWrittenPromise.failure(ex)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
cancel()
}
case ActorSubscriberMessage.OnError(cause)
log.error(cause, "Tearing down OutputStreamSink due to upstream error, wrote bytes: {}", bytesWritten)
case ActorSubscriberMessage.OnError(ex)
log.error(ex, "Tearing down OutputStreamSink due to upstream error, wrote bytes: {}", bytesWritten)
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
context.stop(self)
case ActorSubscriberMessage.OnComplete
@ -51,9 +55,14 @@ private[akka] class OutputStreamSubscriber(os: OutputStream, bytesWrittenPromise
}
override def postStop(): Unit = {
bytesWrittenPromise.trySuccess(bytesWritten)
try {
if (os ne null) os.close()
} catch {
case ex: Exception
completionPromise.success(IOResult(bytesWritten, Failure(ex)))
}
completionPromise.trySuccess(IOResult(bytesWritten, Success(Done)))
super.postStop()
}
}

View file

@ -0,0 +1,36 @@
/**
* Copyright (C) 2016 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.stream.io
import akka.Done
import scala.util.{ Failure, Success, Try }
/**
* Holds a result of an IO operation.
*
* @param count Numeric value depending on context, for example IO operations performed or bytes processed.
* @param status Status of the result. Can be either [[akka.Done]] or an exception.
*/
final case class IOResult private[stream] (count: Long, status: Try[Done]) {
/**
* Java API: Numeric value depending on context, for example IO operations performed or bytes processed.
*/
def getCount: Long = count
/**
* Java API: Indicates whether IO operation completed successfully or not.
*/
def wasSuccessful: Boolean = status.isSuccess
/**
* Java API: If the IO operation resulted in an error, returns the corresponding [[Throwable]]
* or throws [[UnsupportedOperationException]] otherwise.
*/
def getError: Throwable = status match {
case Failure(t) t
case Success(_) throw new UnsupportedOperationException("IO operation was successfull.")
}
}

View file

@ -7,6 +7,7 @@ import java.io.{ InputStream, OutputStream, File }
import akka.japi.function
import akka.stream.{ scaladsl, javadsl, ActorAttributes }
import akka.stream.io.IOResult
import akka.util.ByteString
import scala.concurrent.Future
@ -21,20 +22,22 @@ object FileIO {
* Overwrites existing files, if you want to append to an existing file use [[#file(File, Boolean)]] and
* pass in `true` as the Boolean argument.
*
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
* Materializes a [[Future]] of [[IOResult]] that will be completed with the size of the file (in bytes) at the streams completion,
* and a possible exception if IO operation was not completed successfully.
*
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* @param f The file to write to
*/
def toFile(f: File): javadsl.Sink[ByteString, Future[java.lang.Long]] = toFile(f, append = false)
def toFile(f: File): javadsl.Sink[ByteString, Future[IOResult]] = toFile(f, append = false)
/**
* Creates a Sink that writes incoming [[ByteString]] elements to the given file and either overwrites
* or appends to it.
*
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
* Materializes a [[Future]] of [[IOResult]] that will be completed with the size of the file (in bytes) at the streams completion,
* and a possible exception if IO operation was not completed successfully.
*
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
@ -42,8 +45,8 @@ object FileIO {
* @param f The file to write to
* @param append Whether or not the file should be overwritten or appended to
*/
def toFile(f: File, append: Boolean): javadsl.Sink[ByteString, Future[java.lang.Long]] =
new Sink(scaladsl.FileIO.toFile(f, append)).asInstanceOf[javadsl.Sink[ByteString, Future[java.lang.Long]]]
def toFile(f: File, append: Boolean): javadsl.Sink[ByteString, Future[IOResult]] =
new Sink(scaladsl.FileIO.toFile(f, append)).asInstanceOf[javadsl.Sink[ByteString, Future[IOResult]]]
/**
* Creates a Source from a Files contents.
@ -53,9 +56,10 @@ object FileIO {
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
* It materializes a [[Future]] of [[IOResult]] containing the number of bytes read from the source file upon completion,
* and a possible exception if IO operation was not completed successfully.
*/
def fromFile(f: File): javadsl.Source[ByteString, Future[java.lang.Long]] = fromFile(f, 8192)
def fromFile(f: File): javadsl.Source[ByteString, Future[IOResult]] = fromFile(f, 8192)
/**
* Creates a synchronous (Java 6 compatible) Source from a Files contents.
@ -65,9 +69,10 @@ object FileIO {
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
* It materializes a [[Future]] of [[IOResult]] containing the number of bytes read from the source file upon completion,
* and a possible exception if IO operation was not completed successfully.
*/
def fromFile(f: File, chunkSize: Int): javadsl.Source[ByteString, Future[java.lang.Long]] =
new Source(scaladsl.FileIO.fromFile(f, chunkSize)).asInstanceOf[Source[ByteString, Future[java.lang.Long]]]
def fromFile(f: File, chunkSize: Int): javadsl.Source[ByteString, Future[IOResult]] =
new Source(scaladsl.FileIO.fromFile(f, chunkSize)).asInstanceOf[Source[ByteString, Future[IOResult]]]
}

View file

@ -7,6 +7,7 @@ import java.io.{ InputStream, OutputStream }
import akka.japi.function
import akka.stream.{ scaladsl, javadsl, ActorAttributes }
import akka.stream.io.IOResult
import akka.util.ByteString
import scala.concurrent.Future
@ -19,15 +20,16 @@ object StreamConverters {
/**
* Sink which writes incoming [[ByteString]]s to an [[OutputStream]] created by the given function.
*
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
* Materializes a [[Future]] of [[IOResult]] that will be completed with the size of the file (in bytes) at the streams completion,
* and a possible exception if IO operation was not completed successfully.
*
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* @param f A Creator which creates an OutputStream to write to
*/
def fromOutputStream(f: function.Creator[OutputStream]): javadsl.Sink[ByteString, Future[java.lang.Long]] =
new Sink(scaladsl.StreamConverters.fromOutputStream(() f.create())).asInstanceOf[javadsl.Sink[ByteString, Future[java.lang.Long]]]
def fromOutputStream(f: function.Creator[OutputStream]): javadsl.Sink[ByteString, Future[IOResult]] =
new Sink(scaladsl.StreamConverters.fromOutputStream(() f.create())).asInstanceOf[javadsl.Sink[ByteString, Future[IOResult]]]
/**
* Creates a Sink which when materialized will return an [[java.io.InputStream]] which it is possible
@ -67,8 +69,8 @@ object StreamConverters {
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
*/
def fromInputStream(in: function.Creator[InputStream], chunkSize: Int): javadsl.Source[ByteString, Future[java.lang.Long]] =
new Source(scaladsl.StreamConverters.fromInputStream(() in.create(), chunkSize)).asInstanceOf[Source[ByteString, Future[java.lang.Long]]]
def fromInputStream(in: function.Creator[InputStream], chunkSize: Int): javadsl.Source[ByteString, Future[IOResult]] =
new Source(scaladsl.StreamConverters.fromInputStream(() in.create(), chunkSize)).asInstanceOf[Source[ByteString, Future[IOResult]]]
/**
* Creates a Source from an [[java.io.InputStream]] created by the given function.
@ -78,9 +80,10 @@ object StreamConverters {
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
* It materializes a [[Future]] of [[IOResult]] containing the number of bytes read from the source file upon completion,
* and a possible exception if IO operation was not completed successfully.
*/
def fromInputStream(in: function.Creator[InputStream]): javadsl.Source[ByteString, Future[java.lang.Long]] = fromInputStream(in, 8192)
def fromInputStream(in: function.Creator[InputStream]): javadsl.Source[ByteString, Future[IOResult]] = fromInputStream(in, 8192)
/**
* Creates a Source which when materialized will return an [[java.io.OutputStream]] which it is possible

View file

@ -6,6 +6,7 @@ package akka.stream.scaladsl
import java.io.{ OutputStream, InputStream, File }
import akka.stream.ActorAttributes
import akka.stream.io.IOResult
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io._
import akka.util.ByteString
@ -29,24 +30,26 @@ object FileIO {
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
* It materializes a [[Future]] of [[IOResult]] containing the number of bytes read from the source file upon completion,
* and a possible exception if IO operation was not completed successfully.
*
* @param f the File to read from
* @param chunkSize the size of each read operation, defaults to 8192
*/
def fromFile(f: File, chunkSize: Int = 8192): Source[ByteString, Future[Long]] =
def fromFile(f: File, chunkSize: Int = 8192): Source[ByteString, Future[IOResult]] =
new Source(new FileSource(f, chunkSize, DefaultAttributes.fileSource, sourceShape("FileSource")))
/**
* Creates a Sink which writes incoming [[ByteString]] elements to the given file and either overwrites
* or appends to it.
*
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
* Materializes a [[Future]] of [[IOResult]] that will be completed with the size of the file (in bytes) at the streams completion,
* and a possible exception if IO operation was not completed successfully.
*
* This source is backed by an Actor which will use the dedicated `akka.stream.blocking-io-dispatcher`,
* unless configured otherwise by using [[ActorAttributes]].
*/
def toFile(f: File, append: Boolean = false): Sink[ByteString, Future[Long]] =
def toFile(f: File, append: Boolean = false): Sink[ByteString, Future[IOResult]] =
new Sink(new FileSink(f, append, DefaultAttributes.fileSink, sinkShape("FileSink")))
}

View file

@ -6,6 +6,7 @@ package akka.stream.scaladsl
import java.io.{ OutputStream, InputStream }
import akka.stream.ActorAttributes
import akka.stream.io.IOResult
import akka.stream.impl.Stages.DefaultAttributes
import akka.stream.impl.io.{ InputStreamSinkStage, OutputStreamSink, OutputStreamSourceStage, InputStreamSource }
import akka.util.ByteString
@ -29,12 +30,13 @@ object StreamConverters {
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*
* It materializes a [[Future]] containing the number of bytes read from the source file upon completion.
* It materializes a [[Future]] of [[IOResult]] containing the number of bytes read from the source file upon completion,
* and a possible exception if IO operation was not completed successfully.
*
* @param in a function which creates the InputStream to read from
* @param chunkSize the size of each read operation, defaults to 8192
*/
def fromInputStream(in: () InputStream, chunkSize: Int = 8192): Source[ByteString, Future[Long]] =
def fromInputStream(in: () InputStream, chunkSize: Int = 8192): Source[ByteString, Future[IOResult]] =
new Source(new InputStreamSource(in, chunkSize, DefaultAttributes.inputStreamSource, sourceShape("InputStreamSource")))
/**
@ -54,12 +56,13 @@ object StreamConverters {
/**
* Creates a Sink which writes incoming [[ByteString]]s to an [[OutputStream]] created by the given function.
*
* Materializes a [[Future]] that will be completed with the size of the file (in bytes) at the streams completion.
* Materializes a [[Future]] of [[IOResult]] that will be completed with the size of the file (in bytes) at the streams completion,
* and a possible exception if IO operation was not completed successfully.
*
* You can configure the default dispatcher for this Source by changing the `akka.stream.blocking-io-dispatcher` or
* set it for a given Source by using [[ActorAttributes]].
*/
def fromOutputStream(out: () OutputStream): Sink[ByteString, Future[Long]] =
def fromOutputStream(out: () OutputStream): Sink[ByteString, Future[IOResult]] =
new Sink(new OutputStreamSink(out, DefaultAttributes.outputStreamSink, sinkShape("OutputStreamSink")))
/**

View file

@ -192,7 +192,7 @@ object Dependencies {
lazy val streamTestkit = l ++= Seq(Test.scalatest.value, Test.scalacheck.value, Test.junit)
lazy val streamTests = l ++= Seq(Test.scalatest.value, Test.scalacheck.value, Test.junit, Test.junitIntf, Test.commonsIo)
lazy val streamTests = l ++= Seq(Test.scalatest.value, Test.scalacheck.value, Test.junit, Test.commonsIo)
lazy val streamTestsTck = l ++= Seq(Test.scalatest.value, Test.scalacheck.value, Test.junit, Test.reactiveStreamsTck)