Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import org.scalatest.time.Span
import scala.annotation.nowarn
import scala.concurrent.Await
import scala.concurrent.Future
import scala.concurrent.Promise
//#imports
import akka.stream._

Expand Down Expand Up @@ -488,7 +489,63 @@ class SourceSpec extends StreamSpec with DefaultTimeout {
}
}

"Source.future" must {
"optimize already completed future" in {
val future = Future.successful("done")
val source = Source.future(future)
source.getAttributes.nameLifted should ===(Some("singleSource"))
source.runWith(Sink.head).futureValue should ===("done")
}

"optimize already failed future" in {
val future = Future.failed(TE("boom"))
val source = Source.future(future)
source.getAttributes.nameLifted should ===(Some("failedSource"))
source.runWith(Sink.head).failed.futureValue shouldBe a[TE]
}

"handle regular future" in {
val promise = Promise[String]()
val source = Source.future(promise.future)
source.getAttributes.nameLifted should ===(Some("futureSource"))
promise.success("done")
source.runWith(Sink.head).futureValue should ===("done")
}
}

"Source.futureSource" must {
"optimize already completed future" in {
val future = Future.successful(Source.single("done"))
val source = Source.futureSource(future)
source.getAttributes.nameLifted should ===(Some("singleSource"))
source.runWith(Sink.head).futureValue should ===("done")
}

"pass along materialized value for already completed future" in {
val future = Future.successful(Source.single("done").mapMaterializedValue(_ => "materializedValue"))
val source = Source.futureSource(future)
source.toMat(Sink.ignore)(Keep.left).run().futureValue should ===("materializedValue")
}

"handle already failed future" in {
val future = Future.failed[Source[String, NotUsed]](TE("boom"))
val source = Source.futureSource(future)
val (futureMat, streamResult) = source.toMat(Sink.head)(Keep.both).run()

streamResult.failed.futureValue should ===(TE("boom"))
futureMat.failed.futureValue should ===(TE("boom"))
}

"handle later failed future" in {
val promise = Promise[Source[String, NotUsed]]()
val source = Source.futureSource(promise.future)
promise.failure(TE("boom"))

val (futureMat, streamResult) = source.toMat(Sink.head)(Keep.both).run()

streamResult.failed.futureValue should ===(TE("boom"))
futureMat.failed.futureValue should ===(TE("boom"))
}

"not cancel substream twice" in {
val result = Source
Expand Down
22 changes: 16 additions & 6 deletions akka-stream/src/main/scala/akka/stream/scaladsl/Source.scala
Original file line number Diff line number Diff line change
Expand Up @@ -5,17 +5,14 @@
package akka.stream.scaladsl

import java.util.concurrent.CompletionStage

import scala.annotation.nowarn
import scala.annotation.tailrec
import scala.annotation.unchecked.uncheckedVariance
import scala.collection.immutable
import scala.jdk.FutureConverters._
import scala.concurrent.{ Future, Promise }
import scala.concurrent.duration.FiniteDuration

import org.reactivestreams.{ Publisher, Subscriber }

import akka.{ Done, NotUsed }
import akka.actor.{ ActorRef, Cancellable }
import akka.annotation.InternalApi
Expand All @@ -27,6 +24,9 @@ import akka.stream.impl.fusing.GraphStages._
import akka.stream.stage.GraphStageWithMaterializedValue
import akka.util.ConstantFun

import scala.util.Failure
import scala.util.Success

/**
* A `Source` is a set of stream processing steps that has one open output. It can comprise
* any number of internal sources and transformations that are wired together, or it can be
Expand Down Expand Up @@ -505,7 +505,12 @@ object Source {
* The stream fails if the `Future` is completed with a failure.
*/
def future[T](futureElement: Future[T]): Source[T, NotUsed] =
fromGraph(new FutureSource[T](futureElement))
futureElement.value match {
case Some(Success(null)) => empty
case Some(Success(value)) => single(value)
case Some(Failure(cause)) => failed(cause)
case None => fromGraph(new FutureSource[T](futureElement))
}

/**
* Never emits any elements, never completes and never fails.
Expand All @@ -527,8 +532,13 @@ object Source {
* Turn a `Future[Source]` into a source that will emit the values of the source when the future completes successfully.
* If the `Future` is completed with a failure the stream is failed.
*/
def futureSource[T, M](futureSource: Future[Source[T, M]]): Source[T, Future[M]] =
fromGraph(new FutureFlattenSource(futureSource))
def futureSource[T, M](futureSource: Future[Source[T, M]]): Source[T, Future[M]] = {
futureSource.value match {
case Some(Success(source)) => source.mapMaterializedValue(Future.successful)
case Some(Failure(exc)) => failed(exc).mapMaterializedValue(_ => Future.failed(exc))
case _ => fromGraph(new FutureFlattenSource(futureSource))
}
}

/**
* Defers invoking the `create` function to create a single element until there is downstream demand.
Expand Down