!htp rename akka.http.routing package to akka.http.server

This commit is contained in:
Mathias 2014-09-17 16:42:31 +02:00
parent 1f32faeb96
commit 47affda004
43 changed files with 70 additions and 70 deletions

View file

@ -13,7 +13,7 @@ import scala.reflect.ClassTag
import org.scalatest.Suite import org.scalatest.Suite
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.http.routing._ import akka.http.server._
import akka.http.unmarshalling._ import akka.http.unmarshalling._
import akka.http.model._ import akka.http.model._
import headers.Host import headers.Host

View file

@ -12,7 +12,7 @@ import scala.concurrent.{ Await, ExecutionContext }
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.http.model.HttpEntity.ChunkStreamPart import akka.http.model.HttpEntity.ChunkStreamPart
import akka.http.routing._ import akka.http.server._
import akka.http.model._ import akka.http.model._
trait RouteTestResultComponent { trait RouteTestResultComponent {

View file

@ -11,7 +11,7 @@ import akka.testkit.TestProbe
import akka.util.Timeout import akka.util.Timeout
import akka.pattern.ask import akka.pattern.ask
import akka.http.model.headers.RawHeader import akka.http.model.headers.RawHeader
import akka.http.routing._ import akka.http.server._
import akka.http.model._ import akka.http.model._
import StatusCodes._ import StatusCodes._
import HttpMethods._ import HttpMethods._

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.http.model.HttpMethods._ import akka.http.model.HttpMethods._
import akka.http.routing.PathMatchers.{ Segment, IntNumber } import akka.http.server.PathMatchers.{ Segment, IntNumber }
class BasicRouteSpecs extends RoutingSpec { class BasicRouteSpecs extends RoutingSpec {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import org.scalatest.{ WordSpec, Suite, Matchers } import org.scalatest.{ WordSpec, Suite, Matchers }
import akka.http.model.HttpResponse import akka.http.model.HttpResponse

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import com.typesafe.config.{ ConfigFactory, Config } import com.typesafe.config.{ ConfigFactory, Config }
import scala.concurrent.duration._ import scala.concurrent.duration._

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
class BasicDirectivesSpec extends RoutingSpec { class BasicDirectivesSpec extends RoutingSpec {

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.directives package akka.http.server.directives
import akka.http.model.{ StatusCodes, HttpMethods } import akka.http.model.{ StatusCodes, HttpMethods }
import akka.http.routing._ import akka.http.server._
class MethodDirectivesSpec extends RoutingSpec { class MethodDirectivesSpec extends RoutingSpec {

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.directives package akka.http.server.directives
import akka.http.routing._ import akka.http.server._
class PathDirectivesSpec extends RoutingSpec { class PathDirectivesSpec extends RoutingSpec {
val echoUnmatchedPath = unmatchedPath { echoComplete } val echoUnmatchedPath = unmatchedPath { echoComplete }

View file

@ -2,13 +2,13 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.directives package akka.http.server.directives
import org.scalatest.FreeSpec import org.scalatest.FreeSpec
import scala.concurrent.Promise import scala.concurrent.Promise
import akka.http.marshalling._ import akka.http.marshalling._
import akka.http.routing._ import akka.http.server._
import akka.http.model._ import akka.http.model._
import akka.http.util._ import akka.http.util._
import headers._ import headers._

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
private[util] abstract class AppendOneInstances { private[util] abstract class AppendOneInstances {
type Aux[P, S, Out0] = AppendOne[P, S] { type Out = Out0 } type Aux[P, S, Out0] = AppendOne[P, S] { type Out = Out0 }

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
import akka.http.routing.Route import akka.http.server.Route
private[util] abstract class ApplyConverterInstances { private[util] abstract class ApplyConverterInstances {
[#implicit def hac1[[#T1#]] = new ApplyConverter[Tuple1[[#T1#]]] { [#implicit def hac1[[#T1#]] = new ApplyConverter[Tuple1[[#T1#]]] {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
trait ConstructFromTupleInstances { trait ConstructFromTupleInstances {
[#implicit def instance1[[#T1#], R](construct: ([#T1#]) => R): ConstructFromTuple[Tuple1[[#T1#]], R] = [#implicit def instance1[[#T1#], R](construct: ([#T1#]) => R): ConstructFromTuple[Tuple1[[#T1#]], R] =

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
private[util] abstract class HeadTailInstances { private[util] abstract class HeadTailInstances {
type Aux[L, H0, T0] = HeadTail[L] { type H = H0; type T = T0 } type Aux[L, H0, T0] = HeadTail[L] { type H = H0; type T = T0 }

View file

@ -10,7 +10,7 @@ import akka.actor.ActorRefFactory
import akka.parboiled2.util.Base64 import akka.parboiled2.util.Base64
import akka.stream.{ FlattenStrategy, FlowMaterializer } import akka.stream.{ FlattenStrategy, FlowMaterializer }
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.http.rendering.BodyPartRenderer import akka.http.engine.rendering.BodyPartRenderer
import akka.http.util.actorSystem import akka.http.util.actorSystem
import akka.http.model._ import akka.http.model._
import MediaTypes._ import MediaTypes._

View file

@ -2,10 +2,10 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.http.routing.directives.RouteDirectives import akka.http.server.directives.RouteDirectives
import akka.http.routing.util._ import akka.http.server.util._
trait ConjunctionMagnet[L] { trait ConjunctionMagnet[L] {
type Out type Out

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import directives._ import directives._

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import scala.util.control.NonFatal import scala.util.control.NonFatal

View file

@ -2,12 +2,12 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import java.util.UUID import java.util.UUID
import scala.util.matching.Regex import scala.util.matching.Regex
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.http.routing.util.{ Tuple, Join } import akka.http.server.util.{ Tuple, Join }
import akka.http.model.Uri.Path import akka.http.model.Uri.Path
import akka.http.util._ import akka.http.util._
import directives.NameReceptacle import directives.NameReceptacle

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.http.model._ import akka.http.model._
import akka.http.model.headers.{ ByteRange, HttpEncoding } import akka.http.model.headers.{ ByteRange, HttpEncoding }
@ -124,7 +124,7 @@ case class UnacceptedResponseContentTypeRejection(supported: Seq[ContentType]) e
case class UnacceptedResponseEncodingRejection(supported: HttpEncoding) extends Rejection case class UnacceptedResponseEncodingRejection(supported: HttpEncoding) extends Rejection
/** /**
* Rejection created by an [[akka.http.routing.authentication.HttpAuthenticator]]. * Rejection created by an [[akka.http.server.authentication.HttpAuthenticator]].
* Signals that the request was rejected because the user could not be authenticated. The reason for the rejection is * Signals that the request was rejected because the user could not be authenticated. The reason for the rejection is
* specified in the cause. * specified in the cause.
*/ */

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.http.model._ import akka.http.model._

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
trait RouteConcatenation { trait RouteConcatenation {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.http.model.HttpResponse import akka.http.model.HttpResponse

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.ActorRefFactory import akka.actor.ActorRefFactory

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import akka.actor.{ ActorSystem, ActorContext } import akka.actor.{ ActorSystem, ActorContext }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
import akka.http.routing.util.Tuple import akka.http.server.util.Tuple
/** /**
* A Route that can be implicitly converted into a Directive (fitting any signature). * A Route that can be implicitly converted into a Directive (fitting any signature).

View file

@ -2,11 +2,11 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
import scala.collection.immutable import scala.collection.immutable
import akka.http.routing.util.Tuple import akka.http.server.util.Tuple
import akka.http.model._ import akka.http.model._
trait BasicDirectives { trait BasicDirectives {

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
trait ExecutionDirectives { trait ExecutionDirectives {
@ -10,7 +10,7 @@ trait ExecutionDirectives {
/** /**
* Transforms exceptions thrown during evaluation of its inner route using the given * Transforms exceptions thrown during evaluation of its inner route using the given
* [[akka.http.routing.ExceptionHandler]]. * [[akka.http.server.ExceptionHandler]].
*/ */
def handleExceptions(handler: ExceptionHandler): Directive0 = def handleExceptions(handler: ExceptionHandler): Directive0 =
mapInnerRoute { inner mapInnerRoute { inner
@ -26,7 +26,7 @@ trait ExecutionDirectives {
/** /**
* Transforms rejections produced by its inner route using the given * Transforms rejections produced by its inner route using the given
* [[akka.http.routing.RejectionHandler]]. * [[akka.http.server.RejectionHandler]].
*/ */
def handleRejections(handler: RejectionHandler): Directive0 = def handleRejections(handler: RejectionHandler): Directive0 =
mapRequestContext { ctx mapRequestContext { ctx

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
import akka.http.model.HttpMethod import akka.http.model.HttpMethod

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
import akka.http.unmarshalling.{ FromStringOptionDeserializer FSOD, Deserializer } import akka.http.unmarshalling.{ FromStringOptionDeserializer FSOD, Deserializer }

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction with ToNameReceptacleEnhancements { trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction with ToNameReceptacleEnhancements {
@ -11,7 +11,7 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
import PathMatcher._ import PathMatcher._
/** /**
* Consumes a leading slash from the unmatched path of the [[akka.http.routing.RequestContext]] * Consumes a leading slash from the unmatched path of the [[akka.http.server.RequestContext]]
* before applying the given matcher. The matcher has to match the remaining path completely * before applying the given matcher. The matcher has to match the remaining path completely
* or leave only a single trailing slash. * or leave only a single trailing slash.
* If matched the value extracted by the PathMatcher is extracted on the directive level. * If matched the value extracted by the PathMatcher is extracted on the directive level.
@ -19,14 +19,14 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
def path[L](pm: PathMatcher[L]): Directive[L] = pathPrefix(pm ~ PathEnd) def path[L](pm: PathMatcher[L]): Directive[L] = pathPrefix(pm ~ PathEnd)
/** /**
* Consumes a leading slash from the unmatched path of the [[akka.http.routing.RequestContext]] * Consumes a leading slash from the unmatched path of the [[akka.http.server.RequestContext]]
* before applying the given matcher. The matcher has to match a prefix of the remaining path. * before applying the given matcher. The matcher has to match a prefix of the remaining path.
* If matched the value extracted by the PathMatcher is extracted on the directive level. * If matched the value extracted by the PathMatcher is extracted on the directive level.
*/ */
def pathPrefix[L](pm: PathMatcher[L]): Directive[L] = rawPathPrefix(Slash ~ pm) def pathPrefix[L](pm: PathMatcher[L]): Directive[L] = rawPathPrefix(Slash ~ pm)
/** /**
* Applies the given matcher directly to the unmatched path of the [[akka.http.routing.RequestContext]] * Applies the given matcher directly to the unmatched path of the [[akka.http.server.RequestContext]]
* (i.e. without implicitly consuming a leading slash). * (i.e. without implicitly consuming a leading slash).
* The matcher has to match a prefix of the remaining path. * The matcher has to match a prefix of the remaining path.
* If matched the value extracted by the PathMatcher is extracted on the directive level. * If matched the value extracted by the PathMatcher is extracted on the directive level.
@ -40,13 +40,13 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
} }
/** /**
* Checks whether the unmatchedPath of the [[akka.http.routing.RequestContext]] has a prefix matched by the * Checks whether the unmatchedPath of the [[akka.http.server.RequestContext]] has a prefix matched by the
* given PathMatcher. In analogy to the `pathPrefix` directive a leading slash is implied. * given PathMatcher. In analogy to the `pathPrefix` directive a leading slash is implied.
*/ */
def pathPrefixTest[L](pm: PathMatcher[L]): Directive[L] = rawPathPrefixTest(Slash ~ pm) def pathPrefixTest[L](pm: PathMatcher[L]): Directive[L] = rawPathPrefixTest(Slash ~ pm)
/** /**
* Checks whether the unmatchedPath of the [[akka.http.routing.RequestContext]] has a prefix matched by the * Checks whether the unmatchedPath of the [[akka.http.server.RequestContext]] has a prefix matched by the
* given PathMatcher. However, as opposed to the `pathPrefix` directive the matched path is not * given PathMatcher. However, as opposed to the `pathPrefix` directive the matched path is not
* actually "consumed". * actually "consumed".
*/ */
@ -59,7 +59,7 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
} }
/** /**
* Rejects the request if the unmatchedPath of the [[akka.http.routing.RequestContext]] does not have a suffix * Rejects the request if the unmatchedPath of the [[akka.http.server.RequestContext]] does not have a suffix
* matched the given PathMatcher. If matched the value extracted by the PathMatcher is extracted * matched the given PathMatcher. If matched the value extracted by the PathMatcher is extracted
* and the matched parts of the path are consumed. * and the matched parts of the path are consumed.
* Note that, for efficiency reasons, the given PathMatcher must match the desired suffix in reversed-segment * Note that, for efficiency reasons, the given PathMatcher must match the desired suffix in reversed-segment
@ -74,7 +74,7 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
} }
/** /**
* Checks whether the unmatchedPath of the [[akka.http.routing.RequestContext]] has a suffix matched by the * Checks whether the unmatchedPath of the [[akka.http.server.RequestContext]] has a suffix matched by the
* given PathMatcher. However, as opposed to the pathSuffix directive the matched path is not * given PathMatcher. However, as opposed to the pathSuffix directive the matched path is not
* actually "consumed". * actually "consumed".
* Note that, for efficiency reasons, the given PathMatcher must match the desired suffix in reversed-segment * Note that, for efficiency reasons, the given PathMatcher must match the desired suffix in reversed-segment
@ -89,7 +89,7 @@ trait PathDirectives extends PathMatchers with ImplicitPathMatcherConstruction w
} }
/** /**
* Rejects the request if the unmatchedPath of the [[akka.http.routing.RequestContext]] is non-empty, * Rejects the request if the unmatchedPath of the [[akka.http.server.RequestContext]] is non-empty,
* or said differently: only passes on the request to its inner route if the request path * or said differently: only passes on the request to its inner route if the request path
* has been matched completely. * has been matched completely.
*/ */

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing package akka.http.server
package directives package directives
import akka.http.marshalling.ToResponseMarshallable import akka.http.marshalling.ToResponseMarshallable

View file

@ -6,7 +6,7 @@ package akka.http
import akka.http.util.Deferrable import akka.http.util.Deferrable
package object routing { package object server {
type Route = RequestContext Deferrable[RouteResult] type Route = RequestContext Deferrable[RouteResult]
type RouteGenerator[T] = T Route type RouteGenerator[T] = T Route

View file

@ -2,9 +2,9 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
import akka.http.routing._ import akka.http.server._
/** /**
* ApplyConverter allows generic conversion of functions of type `(T1, T2, ...) => Route` to * ApplyConverter allows generic conversion of functions of type `(T1, T2, ...) => Route` to

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
trait ConstructFromTuple[T, R] { trait ConstructFromTuple[T, R] {
def apply(t: T): R def apply(t: T): R

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
/** /**
* `Join` is implicit evidence that tuples `P` and `S` can be joined into tuple `Out` * `Join` is implicit evidence that tuples `P` and `S` can be joined into tuple `Out`

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
/** /**
* Phantom type providing implicit evidence that a given type is a Tuple or Unit. * Phantom type providing implicit evidence that a given type is a Tuple or Unit.

View file

@ -2,7 +2,7 @@
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.http.routing.util package akka.http.server.util
/** /**
* Provides a way to convert a value into an Tuple. * Provides a way to convert a value into an Tuple.
@ -23,7 +23,7 @@ object Tupler extends LowerPriorityTupler {
} }
} }
private[routing] abstract class LowerPriorityTupler { private[server] abstract class LowerPriorityTupler {
implicit def forAnyRef[T] = implicit def forAnyRef[T] =
new Tupler[T] { new Tupler[T] {
type Out = Tuple1[T] type Out = Tuple1[T]

View file

@ -4,7 +4,7 @@
package akka.http.unmarshalling package akka.http.unmarshalling
import akka.http.routing import akka.http.server
import scala.concurrent.{ ExecutionContext, Future } import scala.concurrent.{ ExecutionContext, Future }
@ -27,8 +27,8 @@ object Deserializer extends FromStringDeserializers {
} }
} }
implicit def liftFromStringDeserializer[T: FromStringDeserializer]: FromStringOptionDeserializer[T] = routing.FIXME implicit def liftFromStringDeserializer[T: FromStringDeserializer]: FromStringOptionDeserializer[T] = server.FIXME
implicit def liftFromStringDeserializerConversion[T](f: FromStringDeserializer[T]): FromStringOptionDeserializer[T] = routing.FIXME implicit def liftFromStringDeserializerConversion[T](f: FromStringDeserializer[T]): FromStringOptionDeserializer[T] = server.FIXME
} }
trait DeserializationError extends RuntimeException trait DeserializationError extends RuntimeException
@ -39,7 +39,7 @@ case class UnsupportedContentType(errorMessage: String) extends DeserializationE
case class MalformedContent(errorMessage: String, cause: Option[Throwable] = None) extends DeserializationError case class MalformedContent(errorMessage: String, cause: Option[Throwable] = None) extends DeserializationError
trait FromStringDeserializers { trait FromStringDeserializers {
implicit def String2IntConverter: FromStringDeserializer[Int] = routing.FIXME implicit def String2IntConverter: FromStringDeserializer[Int] = server.FIXME
def HexInt: FromStringDeserializer[Int] = routing.FIXME def HexInt: FromStringDeserializer[Int] = server.FIXME
} }
object FromStringDeserializers extends FromStringDeserializers object FromStringDeserializers extends FromStringDeserializers

View file

@ -8,7 +8,7 @@ import org.reactivestreams.Publisher
import akka.actor.ActorRefFactory import akka.actor.ActorRefFactory
import akka.stream.FlowMaterializer import akka.stream.FlowMaterializer
import akka.stream.scaladsl.Flow import akka.stream.scaladsl.Flow
import akka.http.parsing.BodyPartParser import akka.http.engine.parsing.BodyPartParser
import akka.http.model._ import akka.http.model._
import akka.http.util._ import akka.http.util._
import MediaRanges._ import MediaRanges._