Skip to content

Commit

Permalink
WIP: Huge refactoring and cleanup of decoding and encoding for HttpCo…
Browse files Browse the repository at this point in the history
…dec (zio#1670)

* Beginning attempt to consolidate encoding/decoding logic and clean up bit-rotted code

* fix lazy initializaiton error

* ongoing work

* Refactoring middleware

* Fix some problems

* get tests passing

* Fix issue

* Fix another error
  • Loading branch information
jdegoes authored Oct 23, 2022
1 parent d7f451e commit 1da8838
Show file tree
Hide file tree
Showing 20 changed files with 638 additions and 423 deletions.
2 changes: 1 addition & 1 deletion .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,6 @@ node_modules/
.bsp/
.DS_Store
web/.DS_Store

project/metals.sbt
project/.bloop
profiling/src/main/scala/Main.scala
2 changes: 1 addition & 1 deletion project/metals.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -2,5 +2,5 @@

// This file enables sbt-bloop to create bloop config files.

addSbtPlugin("ch.epfl.scala" % "sbt-bloop" % "1.5.4")
addSbtPlugin("ch.epfl.scala" % "sbt-bloop" % "1.5.3")

8 changes: 8 additions & 0 deletions zio-http/src/main/scala/zio/http/Body.scala
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,8 @@ object Body {
override def asStream(implicit trace: Trace): ZStream[Any, Throwable, Byte] = ZStream.empty
override def write(ctx: Ctx)(implicit trace: Trace): Task[Boolean] = ZIO.succeed(false)
override def isComplete: Boolean = true

override def toString(): String = "Body.empty"
}

/**
Expand All @@ -72,6 +74,8 @@ object Body {

override def write(ctx: Ctx)(implicit trace: Trace): Task[Boolean] =
ZIO.attempt(ctx.write(Unpooled.wrappedBuffer(asciiString.array())): Unit).as(false)

override def toString(): String = s"Body.fromAsciiString($asciiString)"
}

/**
Expand All @@ -89,6 +93,8 @@ object Body {

override def write(ctx: Ctx)(implicit trace: Trace): Task[Boolean] =
ZIO.attempt(ctx.write(byteBuf): Unit).as(false)

override def toString(): String = s"Body.fromByteBuf($byteBuf)"
}

/**
Expand All @@ -110,6 +116,8 @@ object Body {

override def write(ctx: Ctx)(implicit trace: Trace): Task[Boolean] =
ZIO.attempt(ctx.write(Unpooled.wrappedBuffer(data.toArray))).as(false)

override def toString(): String = s"Body.fromChunk($data)"
}

/**
Expand Down
8 changes: 6 additions & 2 deletions zio-http/src/main/scala/zio/http/Path.scala
Original file line number Diff line number Diff line change
Expand Up @@ -199,7 +199,9 @@ object Path {
else Path(path.split("/", -1).toVector.map(Segment(_)))
}

sealed trait Segment extends Product with Serializable
sealed trait Segment extends Product with Serializable {
def text: String
}

object Segment {
def apply(text: String): Segment = text match {
Expand All @@ -211,7 +213,9 @@ object Path {

final case class Text(text: String) extends Segment

case object Root extends Segment
case object Root extends Segment {
def text = ""
}
}

}
8 changes: 8 additions & 0 deletions zio-http/src/main/scala/zio/http/Response.scala
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,9 @@ final case class Response private (
def isWebSocket: Boolean =
self.status.asJava.code() == Status.SwitchingProtocols.asJava.code() && self.attribute.socketApp.nonEmpty

def patch(p: Response.Patch): Response =
copy(headers = self.headers ++ p.addHeaders, status = p.setStatus.getOrElse(self.status))

/**
* Sets the response attributes
*/
Expand Down Expand Up @@ -112,6 +115,11 @@ final case class Response private (
}

object Response {
final case class Patch(addHeaders: Headers, setStatus: Option[Status]) { self =>
def ++(that: Patch): Patch =
Patch(self.addHeaders ++ that.addHeaders, self.setStatus.orElse(that.setStatus))
}

def apply[R, E](
status: Status = Status.Ok,
headers: Headers = Headers.empty,
Expand Down
25 changes: 19 additions & 6 deletions zio-http/src/main/scala/zio/http/api/EndpointError.scala
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package zio.http.api

import zio.stacktracer.TracingImplicits.disableAutoTrace // scalafix:ok;
import zio.http.Path
sealed trait EndpointError extends Exception {
def message: String

Expand All @@ -12,20 +13,32 @@ object EndpointError {
final case class NotFound(message: String, api: EndpointSpec[_, _]) extends ClientError
final case class MalformedResponseBody(message: String, api: EndpointSpec[_, _]) extends ClientError

sealed trait ServerError extends EndpointError
final case class MissingHeader(headerName: String) extends ServerError {
sealed trait ServerError extends EndpointError
final case class MissingHeader(headerName: String) extends ServerError {
def message = s"Missing header $headerName"
}
final case class MalformedHeader(headerName: String, textCodec: TextCodec[_]) extends ServerError {
final case class MalformedMethod(methodName: String, textCodec: TextCodec[_]) extends ServerError {
def message = s"Malformed method $methodName failed to decode using $textCodec"
}
final case class PathTooShort(path: Path, textCodec: TextCodec[_]) extends ServerError {
def message = s"Expected to find ${textCodec} but found pre-mature end to the path ${path}"
}
final case class MalformedRoute(path: Path, segment: Path.Segment, textCodec: TextCodec[_]) extends ServerError {
def message = "Malformed route \"" + segment.text + "\" of " + s"${path} failed to decode using $textCodec"
}
final case class MalformedStatus(status: String, textCodec: TextCodec[_]) extends ServerError {
def message = s"Malformed status $status failed to decode using $textCodec"
}
final case class MalformedHeader(headerName: String, textCodec: TextCodec[_]) extends ServerError {
def message = s"Malformed header $headerName failed to decode using $textCodec"
}
final case class MissingQueryParam(queryParamName: String) extends ServerError {
final case class MissingQueryParam(queryParamName: String) extends ServerError {
def message = s"Missing query parameter $queryParamName"
}
final case class MalformedQueryParam(queryParamName: String, textCodec: TextCodec[_]) extends ServerError {
final case class MalformedQueryParam(queryParamName: String, textCodec: TextCodec[_]) extends ServerError {
def message = s"Malformed query parameter $queryParamName failed to decode using $textCodec"
}
final case class MalformedRequestBody(api: EndpointSpec[_, _]) extends ServerError {
final case class MalformedRequestBody(api: EndpointSpec[_, _]) extends ServerError {
def message = s"Malformed request body failed to decode"
}
}
15 changes: 15 additions & 0 deletions zio-http/src/main/scala/zio/http/api/EndpointSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,11 @@ final case class EndpointSpec[Input, Output](
def implement[R, E](f: Input => ZIO[R, E, Output]): Endpoints[R, E, this.type] =
Endpoints.HandledEndpoint[R, E, Input, Output, this.type](self, f)

def header[A](codec: HeaderCodec[A])(implicit
combiner: Combiner[Input, A],
): EndpointSpec[combiner.Out, Output] =
copy(input = self.input ++ codec)

/**
* Adds a new element of input to the API, which can come from the portion of
* the HTTP path not yet consumed, the query string parameters, or the HTTP
Expand Down Expand Up @@ -125,6 +130,16 @@ final case class EndpointSpec[Input, Output](
*/
def outStream[Output2: Schema]: EndpointSpec[Input, ZStream[Any, Throwable, Output2]] =
copy(output = HttpCodec.BodyStream(implicitly[Schema[Output2]]))

def query[A](codec: QueryCodec[A])(implicit
combiner: Combiner[Input, A],
): EndpointSpec[combiner.Out, Output] =
copy(input = self.input ++ codec)

def route[A](codec: RouteCodec[A])(implicit
combiner: Combiner[Input, A],
): EndpointSpec[combiner.Out, Output] =
copy(input = self.input ++ codec)
}

object EndpointSpec {
Expand Down
169 changes: 117 additions & 52 deletions zio-http/src/main/scala/zio/http/api/HttpCodec.scala
Original file line number Diff line number Diff line change
@@ -1,76 +1,129 @@
package zio.http.api

import zio.http.URL

import scala.language.implicitConversions
import zio.Chunk

import zio.http._
import zio.http.model._

import zio._
import zio.stacktracer.TracingImplicits.disableAutoTrace // scalafix:ok;
import zio.stream.ZStream

import zio.schema.Schema
import zio.stacktracer.TracingImplicits.disableAutoTrace // scalafix:ok;
import zio.schema.codec.Codec

/**
* A [[zio.http.api.HttpCodec]] represents a codec for a part of an HTTP
* request. In the HTTP protocol, these parts may be the unconsumed portion of
* the HTTP path (a route codec), the query string parameters (a query codec),
* the request headers (a header codec), or the request body (a body codec).
* request. HttpCodec the HTTP protocol, these parts may be the unconsumed
* portion of the HTTP path (a route codec), the query string parameters (a
* query codec), the request headers (a header codec), or the request body (a
* body codec).
*
* A `HttpCodec` is a purely declarative description of an input, and therefore,
* A HttpCodec is a purely declarative description of an input, and therefore,
* it can be used to generate documentation, clients, and client libraries.
*
* HttpCodecs are a bit like invertible multi-channel parsers.
*/
sealed trait HttpCodec[-AtomTypes, Input] {
sealed trait HttpCodec[-AtomTypes, Value] {
self =>
private lazy val encoderDecoder = zio.http.api.internal.EncoderDecoder(self)

def ??(doc: Doc): HttpCodec[AtomTypes, Input] = HttpCodec.WithDoc(self, doc)
def ??(doc: Doc): HttpCodec[AtomTypes, Value] = HttpCodec.WithDoc(self, doc)

def ++[AtomTypes1 <: AtomTypes, Input2](that: HttpCodec[AtomTypes1, Input2])(implicit
combiner: Combiner[Input, Input2],
def ++[AtomTypes1 <: AtomTypes, Value2](that: HttpCodec[AtomTypes1, Value2])(implicit
combiner: Combiner[Value, Value2],
): HttpCodec[AtomTypes1, combiner.Out] =
HttpCodec.Combine[AtomTypes1, AtomTypes1, Input, Input2, combiner.Out](self, that, combiner)
HttpCodec.Combine[AtomTypes1, AtomTypes1, Value, Value2, combiner.Out](self, that, combiner)

def &[Value2](
that: QueryCodec[Value2],
)(implicit
combiner: Combiner[Value, Value2],
ev: CodecType.Query <:< AtomTypes,
): QueryCodec[combiner.Out] =
self.asQuery ++ that

def /[Input2](
that: RouteCodec[Input2],
def /[Value2](
that: RouteCodec[Value2],
)(implicit
combiner: Combiner[Input, Input2],
combiner: Combiner[Value, Value2],
ev: CodecType.Route <:< AtomTypes,
): RouteCodec[combiner.Out] =
self.asRoute ++ that

def asRoute(implicit ev: CodecType.Route <:< AtomTypes): RouteCodec[Input] =
self.asInstanceOf[RouteCodec[Input]]
final def asQuery(implicit ev: CodecType.Query <:< AtomTypes): QueryCodec[Value] =
self.asInstanceOf[QueryCodec[Value]]

final def asRoute(implicit ev: CodecType.Route <:< AtomTypes): RouteCodec[Value] =
self.asInstanceOf[RouteCodec[Value]]

final def decodeRequest(codec: Codec)(request: Request)(implicit trace: Trace): Task[Value] =
decode(codec)(request.url, Status.Ok, request.method, request.headers, request.body)

final def decodeResponse(codec: Codec)(response: Response)(implicit trace: Trace): Task[Value] =
decode(codec)(URL.empty, response.status, Method.GET, response.headers, response.body)

private final def decode(codec: Codec)(url: URL, status: Status, method: Method, headers: Headers, body: Body)(
implicit trace: Trace,
): Task[Value] =
encoderDecoder.decode(codec)(url, status, method, headers, body)

final def encodeRequest[Z](codec: Codec)(value: Value): Request =
encodeWith(codec)(value)((url, status, method, headers, body) =>
Request(
url = url,
method = method.getOrElse(Method.GET),
headers = headers,
body = body,
version = Version.Http_1_1,
remoteAddress = None,
),
)

final def encodeResponse[Z](codec: Codec)(value: Value): Response =
encodeWith(codec)(value)((url, status, method, headers, body) => Response(headers = headers, body = body))

final def encodeResponsePatch[Z](value: Value): Response.Patch =
encodeWith(zio.schema.codec.JsonCodec)(value)((url, status, method, headers, body) =>
Response.Patch(addHeaders = headers, setStatus = status),
)

private final def encodeWith[Z](codec: Codec)(value: Value)(
f: (URL, Option[Status], Option[Method], Headers, Body) => Z,
): Z =
encoderDecoder.encodeWith(codec)(value)(f)

/**
* Transforms the type parameter of this `In` from `Input` to `Input2`. Due to
* the fact that `In` is invariant in its type parameter, the transformation
* requires not just a function from `Input` to `Input2`, but also, a function
* from `Input2` to `Input`.
* Transforms the type parameter of this HttpCodec from `Value` to `Value2`.
* Due to the fact that HttpCodec is invariant in its type parameter, the
* transformation requires not just a function from `Value` to `Value2`, but
* also, a function from `Value2` to `Value`.
*
* One of these functions will be used in decoding, for example, when the
* endpoint is invoked on the server. The other of these functions will be
* used in encoding, for example, when a client calls the endpoint on the
* server.
*/
def transform[Input2](f: Input => Input2, g: Input2 => Input): HttpCodec[AtomTypes, Input2] =
HttpCodec.TransformOrFail[AtomTypes, Input, Input2](self, in => Right(f(in)), output => Right(g(output)))

def transformOrFail[Input2](
f: Input => Either[String, Input2],
g: Input2 => Either[String, Input],
): HttpCodec[AtomTypes, Input2] =
HttpCodec.TransformOrFail[AtomTypes, Input, Input2](self, f, g)

def transformOrFailLeft[Input2](
f: Input => Either[String, Input2],
g: Input2 => Input,
): HttpCodec[AtomTypes, Input2] =
HttpCodec.TransformOrFail[AtomTypes, Input, Input2](self, f, output => Right(g(output)))

def transformOrFailRight[Input2](
f: Input => Input2,
g: Input2 => Either[String, Input],
): HttpCodec[AtomTypes, Input2] =
HttpCodec.TransformOrFail[AtomTypes, Input, Input2](self, in => Right(f(in)), g)
def transform[Value2](f: Value => Value2, g: Value2 => Value): HttpCodec[AtomTypes, Value2] =
HttpCodec.TransformOrFail[AtomTypes, Value, Value2](self, in => Right(f(in)), output => Right(g(output)))

def transformOrFail[Value2](
f: Value => Either[String, Value2],
g: Value2 => Either[String, Value],
): HttpCodec[AtomTypes, Value2] =
HttpCodec.TransformOrFail[AtomTypes, Value, Value2](self, f, g)

def transformOrFailLeft[Value2](
f: Value => Either[String, Value2],
g: Value2 => Value,
): HttpCodec[AtomTypes, Value2] =
HttpCodec.TransformOrFail[AtomTypes, Value, Value2](self, f, output => Right(g(output)))

def transformOrFailRight[Value2](
f: Value => Value2,
g: Value2 => Either[String, Value],
): HttpCodec[AtomTypes, Value2] =
HttpCodec.TransformOrFail[AtomTypes, Value, Value2](self, in => Right(f(in)), g)

}

Expand All @@ -80,17 +133,29 @@ object HttpCodec extends HeaderCodecs with QueryCodecs with RouteCodecs {
def empty: HttpCodec[Any, Unit] =
Empty

private[api] sealed trait Atom[-AtomTypes, Input0] extends HttpCodec[AtomTypes, Input0]

private[api] case object Empty extends Atom[Any, Unit]
private[api] final case class Status[A](textCodec: TextCodec[A]) extends Atom[CodecType.Status, A]
private[api] final case class Route[A](textCodec: TextCodec[A]) extends Atom[CodecType.Route, A]
private[api] final case class Body[A](input: Schema[A]) extends Atom[CodecType.Body, A]
private[api] final case class BodyStream[A](element: Schema[A])
private[api] sealed trait Atom[-AtomTypes, Value0] extends HttpCodec[AtomTypes, Value0]

private[api] case object Empty extends Atom[Any, Unit]
private[api] final case class Status[A](textCodec: TextCodec[A]) extends Atom[CodecType.Status, A] { self =>
def erase: Status[Any] = self.asInstanceOf[Status[Any]]
}
private[api] final case class Route[A](textCodec: TextCodec[A]) extends Atom[CodecType.Route, A] { self =>
def erase: Route[Any] = self.asInstanceOf[Route[Any]]
}
private[api] final case class Body[A](schema: Schema[A]) extends Atom[CodecType.Body, A]
private[api] final case class BodyStream[A](schema: Schema[A])
extends Atom[CodecType.Body, ZStream[Any, Throwable, A]]
private[api] final case class Query[A](name: String, textCodec: TextCodec[A]) extends Atom[CodecType.Query, A]
private[api] final case class Method[A](methodCodec: TextCodec[A]) extends Atom[CodecType.Method, A]
private[api] final case class Header[A](name: String, textCodec: TextCodec[A]) extends Atom[CodecType.Header, A]
private[api] final case class Query[A](name: String, textCodec: TextCodec[A]) extends Atom[CodecType.Query, A] {
self =>
def erase: Query[Any] = self.asInstanceOf[Query[Any]]
}
private[api] final case class Method[A](methodCodec: TextCodec[A]) extends Atom[CodecType.Method, A] { self =>
def erase: Method[Any] = self.asInstanceOf[Method[Any]]
}
private[api] final case class Header[A](name: String, textCodec: TextCodec[A]) extends Atom[CodecType.Header, A] {
self =>
def erase: Header[Any] = self.asInstanceOf[Header[Any]]
}
private[api] final case class IndexedAtom[AtomType, A](atom: Atom[AtomType, A], index: Int) extends Atom[AtomType, A]
private[api] final case class WithDoc[AtomType, A](in: HttpCodec[AtomType, A], doc: Doc)
extends HttpCodec[AtomType, A]
Expand Down
Loading

0 comments on commit 1da8838

Please sign in to comment.