+htp #15922 import FileAndResourceDirectives from spray

This commit is contained in:
Johannes Rudolph 2014-11-07 15:01:53 +01:00
parent 4631e052f2
commit 188cbabefe
14 changed files with 716 additions and 9 deletions

View file

@ -5,18 +5,21 @@
package akka.http.util package akka.http.util
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import org.reactivestreams.Subscriber import java.io.InputStream
import akka.http.model.RequestEntity import org.reactivestreams.{ Subscriber, Publisher }
import akka.stream.impl.ErrorPublisher
import akka.stream.{ impl, Transformer, FlowMaterializer }
import akka.stream.scaladsl._
import akka.util.ByteString
import org.reactivestreams.Publisher
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.{ ExecutionContext, Future } import scala.concurrent.{ ExecutionContext, Future }
import akka.actor.Props
import akka.util.ByteString
import akka.stream.{ impl, Transformer, FlowMaterializer }
import akka.stream.scaladsl._
import akka.http.model.RequestEntity
/** /**
* INTERNAL API * INTERNAL API
*/ */
@ -51,7 +54,7 @@ private[http] object StreamUtils {
} }
def failedPublisher[T](ex: Throwable): Publisher[T] = def failedPublisher[T](ex: Throwable): Publisher[T] =
ErrorPublisher(ex).asInstanceOf[Publisher[T]] impl.ErrorPublisher(ex).asInstanceOf[Publisher[T]]
def mapErrorTransformer[T](f: Throwable Throwable): Transformer[T, T] = def mapErrorTransformer[T](f: Throwable Throwable): Transformer[T, T] =
new Transformer[T, T] { new Transformer[T, T] {
@ -125,6 +128,49 @@ private[http] object StreamUtils {
def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity = def mapEntityError(f: Throwable Throwable): RequestEntity RequestEntity =
_.transformDataBytes(() mapErrorTransformer(f)) _.transformDataBytes(() mapErrorTransformer(f))
/**
* Simple blocking Source backed by an InputStream.
*
* FIXME: should be provided by akka-stream, see #15588
*/
def fromInputStreamSource(inputStream: InputStream, defaultChunkSize: Int = 65536): Source[ByteString] = {
import akka.stream.impl._
def props(materializer: ActorBasedFlowMaterializer): Props = {
val iterator = new Iterator[ByteString] {
var finished = false
def hasNext: Boolean = !finished
def next(): ByteString =
if (!finished) {
val buffer = new Array[Byte](defaultChunkSize)
val read = inputStream.read(buffer)
if (read < 0) {
finished = true
inputStream.close()
ByteString.empty
} else ByteString.fromArray(buffer, 0, read)
} else ByteString.empty
}
Props(new IteratorPublisherImpl(iterator, materializer.settings)).withDispatcher(materializer.settings.fileIODispatcher)
}
new AtomicBoolean(false) with SimpleActorFlowSource[ByteString] {
override def attach(flowSubscriber: Subscriber[ByteString], materializer: ActorBasedFlowMaterializer, flowName: String): Unit =
create(materializer, flowName)._1.subscribe(flowSubscriber)
override def isActive: Boolean = true
override def create(materializer: ActorBasedFlowMaterializer, flowName: String): (Publisher[ByteString], Unit) =
if (!getAndSet(true)) {
val ref = materializer.actorOf(props(materializer), name = s"$flowName-0-InputStream-source")
val publisher = ActorPublisher[ByteString](ref)
ref ! ExposedPublisher(publisher.asInstanceOf[impl.ActorPublisher[Any]])
(publisher, ())
} else (ErrorPublisher(new IllegalStateException("One time source can only be instantiated once")).asInstanceOf[Publisher[ByteString]], ())
}
}
/** /**
* Returns a source that can only be used once for testing purposes. * Returns a source that can only be used once for testing purposes.
*/ */

View file

@ -99,5 +99,14 @@ package object util {
private[this] val _identityFunc: Any Any = x x private[this] val _identityFunc: Any Any = x x
/** Returns a constant identity function to avoid allocating the closure */ /** Returns a constant identity function to avoid allocating the closure */
def identityFunc[T]: T T = _identityFunc.asInstanceOf[T T] def identityFunc[T]: T T = _identityFunc.asInstanceOf[T T]
def humanReadableByteCount(bytes: Long, si: Boolean): String = {
val unit = if (si) 1000 else 1024
if (bytes >= unit) {
val exp = (math.log(bytes) / math.log(unit)).toInt
val pre = if (si) "kMGTPE".charAt(exp - 1).toString else "KMGTPE".charAt(exp - 1).toString + 'i'
"%.1f %sB" format (bytes / math.pow(unit, exp), pre)
} else bytes.toString + " B"
}
} }

View file

@ -0,0 +1 @@
<p>Lorem ipsum!</p>

View file

@ -0,0 +1 @@
XyZ

View file

@ -0,0 +1 @@
123

View file

@ -0,0 +1 @@
123

View file

@ -0,0 +1,327 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.server
package directives
import java.io.{ File, FileOutputStream }
import akka.http.model.MediaTypes._
import akka.http.model._
import akka.http.model.headers._
import akka.http.util._
import org.scalatest.matchers.Matcher
import org.scalatest.{ Inside, Inspectors }
import scala.concurrent.duration._
import scala.concurrent.{ Await, ExecutionContext, Future }
import scala.util.Properties
class FileAndResourceDirectivesSpec extends RoutingSpec with Inspectors with Inside {
override def testConfigSource =
"""akka.http.routing {
| file-chunking-threshold-size = 16
| file-chunking-chunk-size = 8
| range-coalescing-threshold = 1
|}""".stripMargin
"getFromFile" should {
"reject non-GET requests" in {
Put() ~> getFromFile("some") ~> check { handled shouldEqual (false) }
}
"reject requests to non-existing files" in {
Get() ~> getFromFile("nonExistentFile") ~> check { handled shouldEqual (false) }
}
"reject requests to directories" in {
Get() ~> getFromFile(Properties.javaHome) ~> check { handled shouldEqual (false) }
}
"return the file content with the MediaType matching the file extension" in {
val file = File.createTempFile("akkaHttpTest", ".PDF")
try {
writeAllText("This is PDF", file)
Get() ~> getFromFile(file.getPath) ~> check {
mediaType shouldEqual `application/pdf`
definedCharset shouldEqual None
responseAs[String] shouldEqual "This is PDF"
headers should contain(`Last-Modified`(DateTime(file.lastModified)))
}
} finally file.delete
}
"return the file content with MediaType 'application/octet-stream' on unknown file extensions" in {
val file = File.createTempFile("akkaHttpTest", null)
try {
writeAllText("Some content", file)
Get() ~> getFromFile(file) ~> check {
mediaType shouldEqual `application/octet-stream`
responseAs[String] shouldEqual "Some content"
}
} finally file.delete
}
"return a single range from a file" in {
val file = File.createTempFile("partialTest", null)
try {
writeAllText("ABCDEFGHIJKLMNOPQRSTUVWXYZ", file)
Get() ~> addHeader(Range(ByteRange(0, 10))) ~> getFromFile(file) ~> check {
status shouldEqual StatusCodes.PartialContent
headers should contain(`Content-Range`(ContentRange(0, 10, 26)))
responseAs[String] shouldEqual "ABCDEFGHIJK"
}
} finally file.delete
}
"return multiple ranges from a file at once" in {
val file = File.createTempFile("partialTest", null)
try {
writeAllText("ABCDEFGHIJKLMNOPQRSTUVWXYZ", file)
val rangeHeader = Range(ByteRange(1, 10), ByteRange.suffix(10))
Get() ~> addHeader(rangeHeader) ~> getFromFile(file, ContentTypes.`text/plain`) ~> check {
status shouldEqual StatusCodes.PartialContent
header[`Content-Range`] shouldEqual None
mediaType.withParams(Map.empty) shouldEqual `multipart/byteranges`
val parts = responseAs[Multipart.ByteRanges].toStrict(100.millis).awaitResult(100.millis).strictParts
parts.size shouldEqual 2
parts(0).entity.data.utf8String shouldEqual "BCDEFGHIJK"
parts(1).entity.data.utf8String shouldEqual "QRSTUVWXYZ"
}
} finally file.delete
}
}
"getFromResource" should {
"reject non-GET requests" in {
Put() ~> getFromResource("some") ~> check { handled shouldEqual (false) }
}
"reject requests to non-existing resources" in {
Get() ~> getFromResource("nonExistingResource") ~> check { handled shouldEqual (false) }
}
"return the resource content with the MediaType matching the file extension" in {
val route = getFromResource("sample.html")
def runCheck() =
Get() ~> route ~> check {
mediaType shouldEqual `text/html`
forAtLeast(1, headers) { h
inside(h) {
case `Last-Modified`(dt)
DateTime(2011, 7, 1) should be < dt
dt.clicks should be < System.currentTimeMillis()
}
}
responseAs[String] shouldEqual "<p>Lorem ipsum!</p>"
}
runCheck()
runCheck() // additional test to check that no internal state is kept
}
"return the file content with MediaType 'application/octet-stream' on unknown file extensions" in {
Get() ~> getFromResource("sample.xyz") ~> check {
mediaType shouldEqual `application/octet-stream`
responseAs[String] shouldEqual "XyZ"
}
}
}
"getFromResourceDirectory" should {
"reject requests to non-existing resources" in {
Get("not/found") ~> getFromResourceDirectory("subDirectory") ~> check { handled shouldEqual (false) }
}
val verify = check {
mediaType shouldEqual `application/pdf`
responseAs[String] shouldEqual "123"
}
"return the resource content with the MediaType matching the file extension - example 1" in { Get("empty.pdf") ~> getFromResourceDirectory("subDirectory") ~> verify }
"return the resource content with the MediaType matching the file extension - example 2" in { Get("empty.pdf") ~> getFromResourceDirectory("subDirectory/") ~> verify }
"return the resource content with the MediaType matching the file extension - example 3" in { Get("subDirectory/empty.pdf") ~> getFromResourceDirectory("") ~> verify }
"reject requests to directory resources" in {
Get() ~> getFromResourceDirectory("subDirectory") ~> check { handled shouldEqual (false) }
}
}
"listDirectoryContents" should {
val base = new File(getClass.getClassLoader.getResource("").toURI).getPath
new File(base, "subDirectory/emptySub").mkdir()
def eraseDateTime(s: String) = s.replaceAll("""\d\d\d\d-\d\d-\d\d \d\d:\d\d:\d\d""", "xxxx-xx-xx xx:xx:xx")
implicit val settings = RoutingSettings.default.copy(renderVanityFooter = false)
"properly render a simple directory" in {
Get() ~> listDirectoryContents(base + "/someDir") ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /</title></head>
|<body>
|<h1>Index of /</h1>
|<hr>
|<pre>
|<a href="/sub/">sub/</a> xxxx-xx-xx xx:xx:xx
|<a href="/fileA.txt">fileA.txt</a> xxxx-xx-xx xx:xx:xx 3 B
|<a href="/fileB.xml">fileB.xml</a> xxxx-xx-xx xx:xx:xx 0 B
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render a sub directory" in {
Get("/sub/") ~> listDirectoryContents(base + "/someDir") ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /sub/</title></head>
|<body>
|<h1>Index of /sub/</h1>
|<hr>
|<pre>
|<a href="/">../</a>
|<a href="/sub/file.html">file.html</a> xxxx-xx-xx xx:xx:xx 0 B
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render the union of several directories" in {
Get() ~> listDirectoryContents(base + "/someDir", base + "/subDirectory") ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /</title></head>
|<body>
|<h1>Index of /</h1>
|<hr>
|<pre>
|<a href="/emptySub/">emptySub/</a> xxxx-xx-xx xx:xx:xx
|<a href="/sub/">sub/</a> xxxx-xx-xx xx:xx:xx
|<a href="/empty.pdf">empty.pdf</a> xxxx-xx-xx xx:xx:xx 3 B
|<a href="/fileA.txt">fileA.txt</a> xxxx-xx-xx xx:xx:xx 3 B
|<a href="/fileB.xml">fileB.xml</a> xxxx-xx-xx xx:xx:xx 0 B
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render an empty sub directory with vanity footer" in {
val settings = 0 // shadow implicit
Get("/emptySub/") ~> listDirectoryContents(base + "/subDirectory") ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /emptySub/</title></head>
|<body>
|<h1>Index of /emptySub/</h1>
|<hr>
|<pre>
|<a href="/">../</a>
|</pre>
|<hr>
|<div style="width:100%;text-align:right;color:gray">
|<small>rendered by <a href="http://akka.io">Akka Http</a> on xxxx-xx-xx xx:xx:xx</small>
|</div>
|</body>
|</html>
|"""
}
}
}
"properly render an empty top-level directory" in {
Get() ~> listDirectoryContents(base + "/subDirectory/emptySub") ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /</title></head>
|<body>
|<h1>Index of /</h1>
|<hr>
|<pre>
|(no files)
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render a simple directory with a path prefix" in {
Get("/files/") ~> pathPrefix("files")(listDirectoryContents(base + "/someDir")) ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /files/</title></head>
|<body>
|<h1>Index of /files/</h1>
|<hr>
|<pre>
|<a href="/files/sub/">sub/</a> xxxx-xx-xx xx:xx:xx
|<a href="/files/fileA.txt">fileA.txt</a> xxxx-xx-xx xx:xx:xx 3 B
|<a href="/files/fileB.xml">fileB.xml</a> xxxx-xx-xx xx:xx:xx 0 B
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render a sub directory with a path prefix" in {
Get("/files/sub/") ~> pathPrefix("files")(listDirectoryContents(base + "/someDir")) ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /files/sub/</title></head>
|<body>
|<h1>Index of /files/sub/</h1>
|<hr>
|<pre>
|<a href="/files/">../</a>
|<a href="/files/sub/file.html">file.html</a> xxxx-xx-xx xx:xx:xx 0 B
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"properly render an empty top-level directory with a path prefix" in {
Get("/files/") ~> pathPrefix("files")(listDirectoryContents(base + "/subDirectory/emptySub")) ~> check {
eraseDateTime(responseAs[String]) shouldEqual prep {
"""<html>
|<head><title>Index of /files/</title></head>
|<body>
|<h1>Index of /files/</h1>
|<hr>
|<pre>
|(no files)
|</pre>
|<hr>
|</body>
|</html>
|"""
}
}
}
"reject requests to file resources" in {
Get() ~> listDirectoryContents(base + "subDirectory/empty.pdf") ~> check { handled shouldEqual (false) }
}
}
def prep(s: String) = s.stripMarginWithNewline("\n")
def writeAllText(text: String, file: File): Unit = {
val fos = new FileOutputStream(file)
try {
fos.write(text.getBytes("UTF-8"))
} finally fos.close()
}
def evaluateTo[T](t: T, atMost: Duration = 100.millis)(implicit ec: ExecutionContext): Matcher[Future[T]] =
be(t).compose[Future[T]] { fut
import scala.concurrent.Await
fut.awaitResult(atMost)
}
}

View file

@ -13,6 +13,12 @@ akka.http.routing {
# (Note that akka-http will always produce log messages containing the full error details) # (Note that akka-http will always produce log messages containing the full error details)
verbose-error-messages = off verbose-error-messages = off
# Enables/disables ETag and `If-Modified-Since` support for FileAndResourceDirectives
file-get-conditional = on
# Enables/disables the rendering of the "rendered by" footer in directory listings
render-vanity-footer = yes
# The maximum size between two requested ranges. Ranges with less space in between will be coalesced. # The maximum size between two requested ranges. Ranges with less space in between will be coalesced.
# #
# When multiple ranges are requested, a server may coalesce any of the ranges that overlap or that are separated # When multiple ranges are requested, a server may coalesce any of the ranges that overlap or that are separated

View file

@ -17,7 +17,7 @@ trait Directives extends RouteConcatenation
with DebuggingDirectives with DebuggingDirectives
with CodingDirectives with CodingDirectives
with ExecutionDirectives with ExecutionDirectives
//with FileAndResourceDirectives with FileAndResourceDirectives
//with FormFieldDirectives //with FormFieldDirectives
with FutureDirectives with FutureDirectives
with HeaderDirectives with HeaderDirectives

View file

@ -10,12 +10,16 @@ import akka.http.util._
case class RoutingSettings( case class RoutingSettings(
verboseErrorMessages: Boolean, verboseErrorMessages: Boolean,
fileGetConditional: Boolean,
renderVanityFooter: Boolean,
rangeCountLimit: Int, rangeCountLimit: Int,
rangeCoalescingThreshold: Long) rangeCoalescingThreshold: Long)
object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") { object RoutingSettings extends SettingsCompanion[RoutingSettings]("akka.http.routing") {
def fromSubConfig(c: Config) = apply( def fromSubConfig(c: Config) = apply(
c getBoolean "verbose-error-messages", c getBoolean "verbose-error-messages",
c getBoolean "file-get-conditional",
c getBoolean "render-vanity-footer",
c getInt "range-count-limit", c getInt "range-count-limit",
c getBytes "range-coalescing-threshold") c getBytes "range-coalescing-threshold")

View file

@ -0,0 +1,311 @@
/*
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.http.server
package directives
import java.io.{ File, FileInputStream }
import akka.actor.ActorSystem
import akka.event.LoggingAdapter
import akka.http.marshalling.{ Marshaller, ToEntityMarshaller }
import akka.http.model._
import akka.http.model.headers._
import akka.http.util._
import scala.annotation.tailrec
import scala.concurrent.ExecutionContext
trait FileAndResourceDirectives {
import CacheConditionDirectives._
import MethodDirectives._
import FileAndResourceDirectives._
import RouteDirectives._
import BasicDirectives._
import RouteConcatenation._
import RangeDirectives._
/**
* Completes GET requests with the content of the given file. The actual I/O operation is
* running detached in a `Future`, so it doesn't block the current thread (but potentially
* some other thread !). If the file cannot be found or read the request is rejected.
*/
def getFromFile(fileName: String)(implicit resolver: ContentTypeResolver): Route =
getFromFile(new File(fileName))
/**
* Completes GET requests with the content of the given file. The actual I/O operation is
* running detached in a `Future`, so it doesn't block the current thread (but potentially
* some other thread !). If the file cannot be found or read the request is rejected.
*/
def getFromFile(file: File)(implicit resolver: ContentTypeResolver): Route =
getFromFile(file, resolver(file.getName))
/**
* Completes GET requests with the content of the given file. The actual I/O operation is
* running detached in a `Future`, so it doesn't block the current thread (but potentially
* some other thread !). If the file cannot be found or read the request is rejected.
*/
def getFromFile(file: File, contentType: ContentType): Route =
get {
if (file.isFile && file.canRead)
conditionalFor(file.length, file.lastModified).apply {
withRangeSupport {
extractExecutionContext { implicit ec
complete(HttpEntity.Default(contentType, file.length, StreamUtils.fromInputStreamSource(new FileInputStream(file))))
}
}
}
else reject
}
private def conditionalFor(length: Long, lastModified: Long): Directive0 =
extractSettings.flatMap(settings
if (settings.fileGetConditional) {
val tag = java.lang.Long.toHexString(lastModified ^ java.lang.Long.reverse(length))
val lastModifiedDateTime = DateTime(math.min(lastModified, System.currentTimeMillis))
conditional(EntityTag(tag), lastModifiedDateTime)
} else pass)
/**
* Completes GET requests with the content of the given resource. The actual I/O operation is
* running detached in a `Future`, so it doesn't block the current thread (but potentially
* some other thread !).
* If the resource cannot be found or read the Route rejects the request.
*/
def getFromResource(resourceName: String)(implicit resolver: ContentTypeResolver): Route =
getFromResource(resourceName, resolver(resourceName))
/**
* Completes GET requests with the content of the given resource. The actual I/O operation is
* running detached in a `Future`, so it doesn't block the current thread (but potentially
* some other thread !).
* If the resource cannot be found or read the Route rejects the request.
*/
def getFromResource(resourceName: String, contentType: ContentType, theClassLoader: ClassLoader = classOf[ActorSystem].getClassLoader): Route =
if (!resourceName.endsWith("/"))
get {
theClassLoader.getResource(resourceName) match {
case null reject
case url
val (length, lastModified) = {
val conn = url.openConnection()
try {
conn.setUseCaches(false) // otherwise the JDK will keep the JAR file open when we close!
val len = conn.getContentLength
val lm = conn.getLastModified
len -> lm
} finally conn.getInputStream.close()
}
conditionalFor(length, lastModified).apply {
withRangeSupport {
extractExecutionContext { implicit ec
complete {
HttpEntity.Default(contentType, length, StreamUtils.fromInputStreamSource(url.openStream()))
}
}
}
}
}
}
else reject // don't serve the content of resource "directories"
/**
* Completes GET requests with the content of a file underneath the given directory.
* If the file cannot be read the Route rejects the request.
*/
def getFromDirectory(directoryName: String)(implicit resolver: ContentTypeResolver): Route = {
val base = withTrailingSlash(directoryName)
extractUnmatchedPath { path
extractLog { log
fileSystemPath(base, path, log) match {
case "" reject
case fileName getFromFile(fileName)
}
}
}
}
/**
* Completes GET requests with a unified listing of the contents of all given directories.
* The actual rendering of the directory contents is performed by the in-scope `Marshaller[DirectoryListing]`.
*/
def listDirectoryContents(directories: String*)(implicit renderer: DirectoryRenderer): Route =
get {
extractRequestContext { ctx
val path = ctx.unmatchedPath
val fullPath = ctx.request.uri.path.toString
val matchedLength = fullPath.lastIndexOf(path.toString)
require(matchedLength >= 0)
val pathPrefix = fullPath.substring(0, matchedLength)
val pathString = withTrailingSlash(fileSystemPath("/", path, ctx.log, '/'))
val dirs = directories flatMap { dir
fileSystemPath(withTrailingSlash(dir), path, ctx.log) match {
case "" None
case fileName
val file = new File(fileName)
if (file.isDirectory && file.canRead) Some(file) else None
}
}
import ctx.executionContext
implicit val marshaller: ToEntityMarshaller[DirectoryListing] = renderer.marshaller(ctx.settings.renderVanityFooter)
if (dirs.isEmpty) reject
else complete(DirectoryListing(pathPrefix + pathString, isRoot = pathString == "/", dirs.flatMap(_.listFiles)))
}
}
/**
* Same as `getFromBrowseableDirectories` with only one directory.
*/
def getFromBrowseableDirectory(directory: String)(implicit renderer: DirectoryRenderer, resolver: ContentTypeResolver): Route =
getFromBrowseableDirectories(directory)
/**
* Serves the content of the given directories as a file system browser, i.e. files are sent and directories
* served as browseable listings.
*/
def getFromBrowseableDirectories(directories: String*)(implicit renderer: DirectoryRenderer, resolver: ContentTypeResolver): Route = {
directories.map(getFromDirectory).reduceLeft(_ ~ _) ~ listDirectoryContents(directories: _*)
}
/**
* Same as "getFromDirectory" except that the file is not fetched from the file system but rather from a
* "resource directory".
*/
def getFromResourceDirectory(directoryName: String)(implicit resolver: ContentTypeResolver): Route = {
val base = if (directoryName.isEmpty) "" else withTrailingSlash(directoryName)
extractUnmatchedPath { path
extractLog { log
fileSystemPath(base, path, log, separator = '/') match {
case "" reject
case resourceName getFromResource(resourceName)
}
}
}
}
}
object FileAndResourceDirectives extends FileAndResourceDirectives {
private def withTrailingSlash(path: String): String = if (path endsWith "/") path else path + '/'
private def fileSystemPath(base: String, path: Uri.Path, log: LoggingAdapter, separator: Char = File.separatorChar): String = {
import java.lang.StringBuilder
@tailrec def rec(p: Uri.Path, result: StringBuilder = new StringBuilder(base)): String =
p match {
case Uri.Path.Empty result.toString
case Uri.Path.Slash(tail) rec(tail, result.append(separator))
case Uri.Path.Segment(head, tail)
if (head.indexOf('/') >= 0 || head == "..") {
log.warning("File-system path for base [{}] and Uri.Path [{}] contains suspicious path segment [{}], " +
"GET access was disallowed", base, path, head)
""
} else rec(tail, result.append(head))
}
rec(if (path.startsWithSlash) path.tail else path)
}
trait DirectoryRenderer {
def marshaller(renderVanityFooter: Boolean): ToEntityMarshaller[DirectoryListing]
}
trait LowLevelDirectoryRenderer {
implicit def defaultDirectoryRenderer(implicit ec: ExecutionContext): DirectoryRenderer =
new DirectoryRenderer {
def marshaller(renderVanityFooter: Boolean): ToEntityMarshaller[DirectoryListing] =
DirectoryListing.directoryMarshaller(renderVanityFooter)
}
}
object DirectoryRenderer extends LowLevelDirectoryRenderer {
implicit def liftMarshaller(implicit _marshaller: ToEntityMarshaller[DirectoryListing]): DirectoryRenderer =
new DirectoryRenderer {
def marshaller(renderVanityFooter: Boolean): ToEntityMarshaller[DirectoryListing] = _marshaller
}
}
}
trait ContentTypeResolver {
def apply(fileName: String): ContentType
}
object ContentTypeResolver {
/**
* The default way of resolving a filename to a ContentType is by looking up the file extension in the
* registry of all defined media-types. By default all non-binary file content is assumed to be UTF-8 encoded.
*/
implicit val Default = withDefaultCharset(HttpCharsets.`UTF-8`)
def withDefaultCharset(charset: HttpCharset): ContentTypeResolver =
new ContentTypeResolver {
def apply(fileName: String) = {
val ext = fileName.lastIndexOf('.') match {
case -1 ""
case x fileName.substring(x + 1)
}
val mediaType = MediaTypes.forExtension(ext) getOrElse MediaTypes.`application/octet-stream`
mediaType match {
case x if !x.binary ContentType(x, charset)
case x ContentType(x)
}
}
}
}
case class DirectoryListing(path: String, isRoot: Boolean, files: Seq[File])
object DirectoryListing {
private val html =
"""<html>
|<head><title>Index of $</title></head>
|<body>
|<h1>Index of $</h1>
|<hr>
|<pre>
|$</pre>
|<hr>$
|<div style="width:100%;text-align:right;color:gray">
|<small>rendered by <a href="http://akka.io">Akka Http</a> on $</small>
|</div>$
|</body>
|</html>
|""".stripMarginWithNewline("\n") split '$'
def directoryMarshaller(renderVanityFooter: Boolean)(implicit ec: ExecutionContext): ToEntityMarshaller[DirectoryListing] =
Marshaller.StringMarshaller.wrap(MediaTypes.`text/html`) { listing
val DirectoryListing(path, isRoot, files) = listing
val filesAndNames = files.map(file file -> file.getName).sortBy(_._2)
val deduped = filesAndNames.zipWithIndex.flatMap {
case (fan @ (file, name), ix)
if (ix == 0 || filesAndNames(ix - 1)._2 != name) Some(fan) else None
}
val (directoryFilesAndNames, fileFilesAndNames) = deduped.partition(_._1.isDirectory)
def maxNameLength(seq: Seq[(File, String)]) = if (seq.isEmpty) 0 else seq.map(_._2.length).max
val maxNameLen = math.max(maxNameLength(directoryFilesAndNames) + 1, maxNameLength(fileFilesAndNames))
val sb = new java.lang.StringBuilder
sb.append(html(0)).append(path).append(html(1)).append(path).append(html(2))
if (!isRoot) {
val secondToLastSlash = path.lastIndexOf('/', path.lastIndexOf('/', path.length - 1) - 1)
sb.append("<a href=\"%s/\">../</a>\n" format path.substring(0, secondToLastSlash))
}
def lastModified(file: File) = DateTime(file.lastModified).toIsoLikeDateTimeString
def start(name: String) =
sb.append("<a href=\"").append(path + name).append("\">").append(name).append("</a>")
.append(" " * (maxNameLen - name.length))
def renderDirectory(file: File, name: String) =
start(name + '/').append(" ").append(lastModified(file)).append('\n')
def renderFile(file: File, name: String) = {
val size = akka.http.util.humanReadableByteCount(file.length, si = true)
start(name).append(" ").append(lastModified(file))
sb.append(" ".substring(size.length)).append(size).append('\n')
}
for ((file, name) directoryFilesAndNames) renderDirectory(file, name)
for ((file, name) fileFilesAndNames) renderFile(file, name)
if (isRoot && files.isEmpty) sb.append("(no files)\n")
sb.append(html(3))
if (renderVanityFooter) sb.append(html(4)).append(DateTime.now.toIsoLikeDateTimeString).append(html(5))
sb.append(html(6)).toString
}
}