Skip to content

Commit

Permalink
finagle-http: ensure server returns 400 with non-ascii characters in URL
Browse files Browse the repository at this point in the history
Problem

HTTP requests that contain invalid (non-ascii) characters in the URI are not
consistently handled between HTTP/1.1 and HTTP/2 servers.

Solution

Add filtering earlier in the Netty pipeline that will ensure that HTTP requests
containing invalid characters are rejected with a `400 Bad Request` response.

Result

Finagle HTTP servers will more consistently handle requests containing
invalid characters, regardless of HTTP version.

JIRA Issues: CSL-8018

Differential Revision: https://phabricator.twitter.biz/D312009
  • Loading branch information
enbnt authored and jenkins committed May 16, 2019
1 parent 8b92e57 commit a7dae7e
Show file tree
Hide file tree
Showing 18 changed files with 316 additions and 35 deletions.
6 changes: 6 additions & 0 deletions CHANGELOG.rst
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,12 @@ New Features
subtype, ignoring their charset, and `MediaType#addUtf8Charset` for easily
setting a utf-8 charset. ``PHAB_ID=D308761``

Bug Fixes
~~~~~~~~~

* finagle-http: Ensure server returns 400 Bad Request when
non-ASCII characters are present in the HTTP request URI path. ``PHAB_ID=D312009``

Runtime Behavior Changes
~~~~~~~~~~~~~~~~~~~~~~~~

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,7 @@ import com.twitter.finagle._
import com.twitter.finagle.builder.ClientBuilder
import com.twitter.finagle.context.{Contexts, Deadline, Retries}
import com.twitter.finagle.filter.ServerAdmissionControl
import com.twitter.finagle.http.service.HttpResponseClassifier
import com.twitter.finagle.http.service.{HttpResponseClassifier, NullService}
import com.twitter.finagle.http2.param.EncoderIgnoreMaxHeaderListSize
import com.twitter.finagle.liveness.{FailureAccrualFactory, FailureDetector}
import com.twitter.finagle.service._
Expand Down Expand Up @@ -36,6 +36,7 @@ abstract class AbstractEndToEndTest
object ClientAbort extends Feature
object NoBodyMessage extends Feature
object MaxHeaderSize extends Feature
object RequiresAsciiFilter extends Feature

var saveBase: Dtab = Dtab.empty
var statsRecv: InMemoryStatsReceiver = new InMemoryStatsReceiver()
Expand Down Expand Up @@ -1303,6 +1304,25 @@ abstract class AbstractEndToEndTest
await(service.close())
}

testIfImplemented(RequiresAsciiFilter)(
"server responds with 400 Bad Request if non-ascii character is present in uri") {
val service = NullService
val server = serverImpl().withStatsReceiver(NullStatsReceiver).serve("localhost:*", service)
val addr = server.boundAddress.asInstanceOf[InetSocketAddress]

val client = clientImpl()
.withStatsReceiver(NullStatsReceiver)
.newService(s"${addr.getHostName}:${addr.getPort}", "client")

try {
val rep = await(client(Request("/DSC02175拷貝.jpg")))
assert(rep.status == Status.BadRequest)
} finally {
await(client.close())
await(server.close())
}
}

test("server responds 500 if an invalid header is being served") {
val service = new HttpService {
def apply(request: Request): Future[Response] = {
Expand All @@ -1323,6 +1343,7 @@ abstract class AbstractEndToEndTest

val rep = await(client(Request("/")))
assert(rep.status == Status.InternalServerError)

}

testIfImplemented(MaxHeaderSize)("client respects MaxHeaderSize in response") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,13 +3,17 @@ package com.twitter.finagle.http
class ClassicHttp2AlpnTest extends AbstractHttp2AlpnTest {
def implName: String = "alpn http/2"
def useMultiplexCodec: Boolean = false

override def featureImplemented(feature: Feature): Boolean =
feature != RequiresAsciiFilter && super.featureImplemented(feature)
}

class MultiplexedHttp2AlpnTest extends AbstractHttp2AlpnTest {
def implName: String = "alpn http/2-multiplex"
def useMultiplexCodec: Boolean = true

// Should be fixed when https://github.com/netty/netty/issues/8434 is fixed.
// MaxHeaderSize should be allowed when https://github.com/netty/netty/issues/8434 is fixed.
// The RequiresAsciiFilter is due to Netty filtering non-ASCII characters in the h2 pipeline.
override def featureImplemented(feature: Feature): Boolean =
feature != MaxHeaderSize && super.featureImplemented(feature)
feature != MaxHeaderSize && feature != RequiresAsciiFilter && super.featureImplemented(feature)
}
Original file line number Diff line number Diff line change
Expand Up @@ -3,13 +3,17 @@ package com.twitter.finagle.http
class ClassicHttp2PriorKnowledgeTest extends AbstractHttp2PriorKnowledgeTest {
def implName: String = "prior knowledge http/2"
def useMultiplexCodec: Boolean = false

override def featureImplemented(feature: Feature): Boolean =
feature != RequiresAsciiFilter && super.featureImplemented(feature)
}

class Http2PriorKnowledgeTest extends AbstractHttp2PriorKnowledgeTest {
def implName: String = "prior knowledge http/2"
def useMultiplexCodec: Boolean = true

// Should be fixed when https://github.com/netty/netty/issues/8434 is fixed.
// MaxHeaderSize should be allowed when https://github.com/netty/netty/issues/8434 is fixed.
// The RequiresAsciiFilter is due to Netty filtering non-ASCII characters in the h2 pipeline.
override def featureImplemented(feature: Feature): Boolean =
feature != MaxHeaderSize && super.featureImplemented(feature)
feature != MaxHeaderSize && feature != RequiresAsciiFilter && super.featureImplemented(feature)
}
Original file line number Diff line number Diff line change
Expand Up @@ -2,12 +2,13 @@ package com.twitter.finagle.http2

import com.twitter.finagle.Stack
import com.twitter.finagle.http.Fields
import com.twitter.finagle.http2.transport.{H2Filter, H2StreamChannelInit, PriorKnowledgeHandler}
import com.twitter.finagle.http2.transport.{H2StreamChannelInit, PriorKnowledgeHandler}
import com.twitter.finagle.netty4.http.HttpCodecName
import com.twitter.finagle.param.{Stats, Timer => TimerParam}
import com.twitter.finagle.netty4.http.util.UriUtils
import com.twitter.finagle.param.Stats
import com.twitter.logging.Logger
import io.netty.channel.socket.SocketChannel
import io.netty.channel._
import io.netty.channel.socket.SocketChannel
import io.netty.handler.codec.http.HttpServerUpgradeHandler.{
SourceCodec,
UpgradeCodec,
Expand Down Expand Up @@ -56,9 +57,7 @@ final private[finagle] class Http2CleartextServerInitializer(

// we insert immediately after the Http2MultiplexCodec#0, which we know is the
// last Http2 frames before they're converted to Http/1.1
val timer = params[TimerParam].timer
ctx.pipeline
.addAfter(MultiplexCodecName, H2Filter.HandlerName, new H2Filter(timer))
Http2PipelineInitializer.setup(ctx, params, MultiplexCodecName)
}
}
} else null
Expand All @@ -74,7 +73,7 @@ final private[finagle] class Http2CleartextServerInitializer(
val msg = s"Unexpected codec found: ${other.getClass.getSimpleName}. " +
"Aborting channel initialization"
val ex = new IllegalStateException(msg)
Logger.get(this.getClass).error(ex, msg)
log.error(ex, msg)
throw ex
}
p.addBefore(
Expand Down Expand Up @@ -121,10 +120,16 @@ private object Http2CleartextServerInitializer {
val Name: String = "upgradeHandler"
val MultiplexCodecName: String = "multiplexCodec"

val log = Logger.get()

// For an HTTP/1.x request to have a body it must have either a content-length or a
// transfer-encoding header, otherwise the server can't be sure when the message will end.
private def dontUpgrade(req: HttpRequest): Boolean =
req.protocolVersion != HttpVersion.HTTP_1_1 ||
(req.headers.contains(Fields.ContentLength) && HttpUtil.getContentLength(req) != 0) ||
req.headers.contains(Fields.TransferEncoding)
req.headers.contains(Fields.TransferEncoding) ||
// We need to validate here, as `UriValidatorHandler` requires `BadRequestHandler` in the
// pipeline. If we rework the pipeline, it's possible this can be removed in the future.
!UriUtils.isValidUri(req.uri)

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
package com.twitter.finagle.http2

import com.twitter.finagle.Stack
import com.twitter.finagle.http2.transport.{H2Filter, H2UriValidatorHandler}
import com.twitter.finagle.netty4.http.handler.UriValidatorHandler
import com.twitter.finagle.param.Timer
import io.netty.channel.ChannelHandlerContext

private[http2] object Http2PipelineInitializer {

/**
* Install Finagle specific filters and handlers common across all HTTP/2 only pipelines
*
* @param ctx
* @param params
* @param codecName The name of the handler where the remaining handlers will be added after
*/
def setup(ctx: ChannelHandlerContext, params: Stack.Params, codecName: String): Unit = {
// we insert immediately after the Http2MultiplexCodec#0, which we know are the
// last Http2 frames before they're converted to Http/1.1
val timer = params[Timer].timer
ctx.pipeline
.addAfter(codecName, H2Filter.HandlerName, new H2Filter(timer))
ctx.pipeline
.addAfter(H2Filter.HandlerName, H2UriValidatorHandler.HandlerName, H2UriValidatorHandler)

ctx.pipeline
.remove(UriValidatorHandler.HandlerName)

}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
package com.twitter.finagle.http2.transport

import com.twitter.finagle.netty4.http.util.UriUtils
import io.netty.channel.ChannelHandler.Sharable
import io.netty.channel.{ChannelHandlerContext, ChannelInboundHandlerAdapter}
import io.netty.handler.codec.http.HttpResponseStatus
import io.netty.handler.codec.http2.{DefaultHttp2ResetFrame, Http2HeadersFrame}
import io.netty.util.ReferenceCountUtil

/**
* HTTP URI validation that acts upon [[Http2HeadersFrame]] messages in the Netty HTTP/2 pipeline.
*
* @see [[com.twitter.finagle.netty4.http.handler.UriValidatorHandler]] for HTTP 1.1 handling
*/
@Sharable
final private[http2] object H2UriValidatorHandler extends ChannelInboundHandlerAdapter {

val HandlerName: String = "h2UriValidationHandler"

override def channelRead(ctx: ChannelHandlerContext, msg: Object): Unit = msg match {
case headers: Http2HeadersFrame =>
if (!UriUtils.isValidUri(headers.headers().path())) {
ReferenceCountUtil.release(msg)

// If the URI isn't valid, we want to retain consistency between our HTTP/2 and HTTP/1
// pipelines by returning a 400 Bad Request response instead of continuing down the
// Netty pipeline, which has inconsistent behavior
val frame = new DefaultHttp2ResetFrame(HttpResponseStatus.BAD_REQUEST.code())
ctx.writeAndFlush(frame)
} else {
ctx.fireChannelRead(msg)
}
case _ =>
ctx.fireChannelRead(msg)
}
}
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
package com.twitter.finagle.http2.transport

import com.twitter.finagle.Stack
import com.twitter.finagle.http2.MultiplexCodecBuilder
import com.twitter.finagle.http2.{Http2PipelineInitializer, MultiplexCodecBuilder}
import com.twitter.finagle.netty4.http._
import com.twitter.finagle.param.{Stats, Timer => TimerParam}
import com.twitter.finagle.param.Stats
import com.twitter.logging.Logger
import io.netty.buffer.{ByteBuf, ByteBufUtil}
import io.netty.channel.{
Expand All @@ -14,6 +14,10 @@ import io.netty.channel.{
}
import io.netty.handler.codec.http2.Http2CodecUtil.connectionPrefaceBuf

private[http2] object PriorKnowledgeHandler {
val logger = Logger.get()
}

/**
* This handler allows an instant upgrade to HTTP/2 if the first bytes received from the client
* matches the fixed HTTP/2 client preface. If so we upgrade to HTTP/2 right away otherwise we keep
Expand All @@ -29,6 +33,8 @@ final private[http2] class PriorKnowledgeHandler(
params: Stack.Params)
extends ChannelInboundHandlerAdapter {

import PriorKnowledgeHandler._

val prefaceToRead: ByteBuf = connectionPrefaceBuf
var bytesConsumed: Integer = 0

Expand Down Expand Up @@ -81,8 +87,8 @@ final private[http2] class PriorKnowledgeHandler(

p.replace(HttpCodecName, Http2CodecName, http2MultiplexCodec)
p.remove("upgradeHandler")
val timer = params[TimerParam].timer
p.addAfter(Http2CodecName, H2Filter.HandlerName, new H2Filter(timer))

Http2PipelineInitializer.setup(ctx, params, Http2CodecName)

// Since we changed the pipeline, our current ctx points to the wrong handler
// but we can still use this handler as the reference point in the new pipeline
Expand All @@ -103,12 +109,7 @@ final private[http2] class PriorKnowledgeHandler(
case _ =>
// Not sure if there are valid cases for this. Allow it for now but log it.

Logger
.get(this.getClass)
.warning(
s"Unexpected non ByteBuf message read: " +
s"${msg.getClass.getName} - $msg"
)
logger.warning(s"Unexpected non ByteBuf message read: ${msg.getClass.getName} - $msg")
ctx.fireChannelRead(msg)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import io.netty.handler.codec.http2.HttpConversionUtil.ExtensionHeaderNames.SCHE
* or a port, so we add the special header on behalf of users.
*/
private[http2] class SchemifyingHandler(defaultScheme: String) extends ChannelDuplexHandler {

override def userEventTriggered(ctx: ChannelHandlerContext, event: Any): Unit = {
event match {
case _ @UpgradeEvent.UPGRADE_REJECTED =>
Expand Down
Original file line number Diff line number Diff line change
@@ -1,9 +1,9 @@
package com.twitter.finagle.http2.transport

import com.twitter.finagle.Stack
import com.twitter.finagle.http2.MultiplexCodecBuilder
import com.twitter.finagle.http2.{Http2PipelineInitializer, MultiplexCodecBuilder}
import com.twitter.finagle.netty4.http._
import com.twitter.finagle.param.{Stats, Timer => TimerParam}
import com.twitter.finagle.param.Stats
import io.netty.channel.{Channel, ChannelHandlerContext, ChannelInitializer}
import io.netty.handler.ssl.{ApplicationProtocolNames, ApplicationProtocolNegotiationHandler}

Expand All @@ -26,8 +26,8 @@ final private[http2] class ServerNpnOrAlpnHandler(
val http2MultiplexCodec = MultiplexCodecBuilder.serverMultiplexCodec(params, initializer)
MultiplexCodecBuilder.addStreamsGauge(statsReceiver, http2MultiplexCodec, ctx.channel)
ctx.pipeline.replace(HttpCodecName, Http2CodecName, http2MultiplexCodec)
val timer = params[TimerParam].timer
ctx.pipeline.addAfter(Http2CodecName, H2Filter.HandlerName, new H2Filter(timer))

Http2PipelineInitializer.setup(ctx, params, Http2CodecName)

case ApplicationProtocolNames.HTTP_1_1 =>
// The Http codec is already in the pipeline, so we are good!
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,4 +74,25 @@ class Http2ListenerTest extends FunSuite {
assert(req.headers.get("x-hello") == "world")
await(close())
})

test("Http2Listener should not upgrade with an invalid URI")(new Ctx {

await(write(s"""GET http:///DSC02175拷貝.jpg HTTP/1.1
|x-http2-stream-id: 1
|upgrade: h2c
|HTTP2-Settings: AAEAABAAAAIAAAABAAN_____AAQAAP__AAUAAEAAAAZ_____
|connection: HTTP2-Settings,upgrade
|content-length: 0
|x-hello: world
|
|""".stripMargin.replaceAll("\n", "\r\n")))

assert(await(read()).get == """HTTP/1.0 400 Bad Request
|Connection: close
|Content-Length: 0
|
|""".stripMargin.replaceAll("\n", "\r\n"))

await(close())
})
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
package com.twitter.finagle.http2.transport

import io.netty.channel.embedded.EmbeddedChannel
import io.netty.handler.codec.http2.{Http2Headers, Http2HeadersFrame, Http2ResetFrame}
import org.mockito.Mockito._
import org.scalatest.FunSuite
import org.scalatest.mockito.MockitoSugar

class H2UriValidatorHandlerTest extends FunSuite with MockitoSugar {

test("Accepts valid URI") {
val channel = new EmbeddedChannel(H2UriValidatorHandler)

val frame = mock[Http2HeadersFrame]
val headers = mock[Http2Headers]
when(frame.headers()).thenReturn(headers)
when(headers.path()).thenReturn("/abc.jpg")

assert(channel.writeInbound(frame))
assert(channel.readInbound[Http2HeadersFrame].headers().path() == "/abc.jpg")
}

test("Rejects invalid URI") {
val channel = new EmbeddedChannel(H2UriValidatorHandler)

val frame = mock[Http2HeadersFrame]
val headers = mock[Http2Headers]
when(frame.headers()).thenReturn(headers)
when(headers.path()).thenReturn("/DSC02175拷貝.jpg")

assert(channel.writeInbound(frame) == false)
assert(channel.readOutbound[Http2ResetFrame].errorCode() == 400)
}

}
Loading

0 comments on commit a7dae7e

Please sign in to comment.