=str 21732 rewrite file souce with NIO and GraphStagу
add MiMa excludes and additional test add benchmark and move eclusions to 2.5.4 add future completion on stream termination
This commit is contained in:
parent
e57cab6c85
commit
000ed93576
7 changed files with 259 additions and 164 deletions
|
|
@ -4,48 +4,140 @@
|
|||
package akka.stream.impl.io
|
||||
|
||||
import java.io.InputStream
|
||||
import java.nio.file.Path
|
||||
import java.nio.ByteBuffer
|
||||
import java.nio.channels.{ CompletionHandler, FileChannel }
|
||||
import java.nio.file.{ Files, Path, StandardOpenOption }
|
||||
|
||||
import akka.Done
|
||||
import akka.annotation.InternalApi
|
||||
import akka.stream._
|
||||
import akka.stream.ActorAttributes.Dispatcher
|
||||
import akka.stream.IOResult
|
||||
import akka.stream.impl.Stages.DefaultAttributes.IODispatcher
|
||||
import akka.stream.Attributes.InputBuffer
|
||||
import akka.stream.impl.Stages.DefaultAttributes
|
||||
import akka.stream.impl.{ ErrorPublisher, SourceModule }
|
||||
import akka.stream.stage._
|
||||
import akka.stream.{ IOResult, _ }
|
||||
import akka.util.ByteString
|
||||
import org.reactivestreams._
|
||||
import org.reactivestreams.Publisher
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.concurrent.{ Future, Promise }
|
||||
import scala.util.control.NonFatal
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Creates simple synchronous Source backed by the given file.
|
||||
*/
|
||||
@InternalApi private[akka] final class FileSource(f: Path, chunkSize: Int, startPosition: Long, val attributes: Attributes, shape: SourceShape[ByteString])
|
||||
extends SourceModule[ByteString, Future[IOResult]](shape) {
|
||||
|
||||
private[akka] object FileSource {
|
||||
|
||||
val completionHandler = new CompletionHandler[Integer, Try[Int] ⇒ Unit] {
|
||||
|
||||
override def completed(result: Integer, attachment: Try[Int] ⇒ Unit): Unit = {
|
||||
attachment(Success(result))
|
||||
}
|
||||
|
||||
override def failed(ex: Throwable, attachment: Try[Int] ⇒ Unit): Unit = {
|
||||
attachment(Failure(ex))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Creates simple asynchronous Source backed by the given file.
|
||||
*/
|
||||
private[akka] final class FileSource(path: Path, chunkSize: Int, startPosition: Long)
|
||||
extends GraphStageWithMaterializedValue[SourceShape[ByteString], Future[IOResult]] {
|
||||
require(chunkSize > 0, "chunkSize must be greater than 0")
|
||||
require(startPosition >= 0, "startPosition must be equal or greater than 0")
|
||||
override def create(context: MaterializationContext) = {
|
||||
// FIXME rewrite to be based on GraphStage rather than dangerous downcasts
|
||||
val materializer = ActorMaterializerHelper.downcast(context.materializer)
|
||||
val settings = materializer.effectiveSettings(context.effectiveAttributes)
|
||||
val out = Outlet[ByteString]("FileSource.out")
|
||||
|
||||
override val shape = SourceShape(out)
|
||||
|
||||
override def createLogicAndMaterializedValue(inheritedAttributes: Attributes): (GraphStageLogic, Future[IOResult]) = {
|
||||
val ioResultPromise = Promise[IOResult]()
|
||||
val props = FilePublisher.props(f, ioResultPromise, chunkSize, startPosition, settings.initialInputBufferSize, settings.maxInputBufferSize)
|
||||
val dispatcher = context.effectiveAttributes.get[Dispatcher](IODispatcher).dispatcher
|
||||
|
||||
val ref = materializer.actorOf(context, props.withDispatcher(dispatcher))
|
||||
val logic = new GraphStageLogic(shape) with OutHandler {
|
||||
handler ⇒
|
||||
val buffer = ByteBuffer.allocate(chunkSize)
|
||||
val maxReadAhead = inheritedAttributes.getAttribute(classOf[InputBuffer], InputBuffer(16, 16)).max
|
||||
var channel: FileChannel = _
|
||||
var position = startPosition
|
||||
var chunkCallback: Try[Int] ⇒ Unit = _
|
||||
var eofEncountered = false
|
||||
var availableChunks: Vector[ByteString] = Vector.empty[ByteString]
|
||||
|
||||
(akka.stream.actor.ActorPublisher[ByteString](ref), ioResultPromise.future)
|
||||
setHandler(out, this)
|
||||
|
||||
override def preStart(): Unit = {
|
||||
try {
|
||||
// this is a bit weird but required to keep existing semantics
|
||||
require(Files.exists(path), s"Path '$path' does not exist")
|
||||
require(Files.isRegularFile(path), s"Path '$path' is not a regular file")
|
||||
require(Files.isReadable(path), s"Missing read permission for '$path'")
|
||||
|
||||
channel = FileChannel.open(path, StandardOpenOption.READ)
|
||||
channel.position(position)
|
||||
} catch {
|
||||
case ex: Exception ⇒
|
||||
ioResultPromise.trySuccess(IOResult(position, Failure(ex)))
|
||||
throw ex
|
||||
}
|
||||
}
|
||||
|
||||
override def onPull(): Unit = {
|
||||
if (availableChunks.size < maxReadAhead && !eofEncountered)
|
||||
availableChunks = readAhead(maxReadAhead, availableChunks)
|
||||
//if already read something and try
|
||||
if (availableChunks.nonEmpty) {
|
||||
emitMultiple(out, availableChunks.iterator,
|
||||
() ⇒ if (eofEncountered) success() else setHandler(out, handler)
|
||||
)
|
||||
availableChunks = Vector.empty[ByteString]
|
||||
} else if (eofEncountered) success()
|
||||
}
|
||||
|
||||
private def success(): Unit = {
|
||||
completeStage()
|
||||
ioResultPromise.trySuccess(IOResult(position, Success(Done)))
|
||||
}
|
||||
|
||||
/** BLOCKING I/O READ */
|
||||
@tailrec def readAhead(maxChunks: Int, chunks: Vector[ByteString]): Vector[ByteString] =
|
||||
if (chunks.size < maxChunks && !eofEncountered) {
|
||||
val readBytes = try channel.read(buffer, position) catch {
|
||||
case NonFatal(ex) ⇒
|
||||
failStage(ex)
|
||||
ioResultPromise.trySuccess(IOResult(position, Failure(ex)))
|
||||
throw ex
|
||||
}
|
||||
|
||||
if (readBytes > 0) {
|
||||
buffer.flip()
|
||||
position += readBytes
|
||||
val newChunks = chunks :+ ByteString.fromByteBuffer(buffer)
|
||||
buffer.clear()
|
||||
|
||||
if (readBytes < chunkSize) {
|
||||
eofEncountered = true
|
||||
newChunks
|
||||
} else readAhead(maxChunks, newChunks)
|
||||
} else {
|
||||
eofEncountered = true
|
||||
chunks
|
||||
}
|
||||
} else chunks
|
||||
|
||||
override def onDownstreamFinish(): Unit = success()
|
||||
|
||||
override def postStop(): Unit = {
|
||||
ioResultPromise.trySuccess(IOResult(position, Success(Done)))
|
||||
if ((channel ne null) && channel.isOpen) channel.close()
|
||||
}
|
||||
}
|
||||
|
||||
(logic, ioResultPromise.future)
|
||||
}
|
||||
|
||||
override protected def newInstance(shape: SourceShape[ByteString]): SourceModule[ByteString, Future[IOResult]] =
|
||||
new FileSource(f, chunkSize, startPosition, attributes, shape)
|
||||
|
||||
override def withAttributes(attr: Attributes): SourceModule[ByteString, Future[IOResult]] =
|
||||
new FileSource(f, chunkSize, startPosition, attr, amendShape(attr))
|
||||
|
||||
override protected def label: String = s"FileSource($f, $chunkSize)"
|
||||
override def toString = s"FileSource($path, $chunkSize)"
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -79,4 +171,4 @@ import scala.concurrent.{ Future, Promise }
|
|||
|
||||
override def withAttributes(attr: Attributes): SourceModule[ByteString, Future[IOResult]] =
|
||||
new InputStreamSource(createInputStream, chunkSize, attr, amendShape(attr))
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue