Skip to content

Commit

Permalink
Merge pull request #3526 from jrudolph/3514-really-move-http2-support…
Browse files Browse the repository at this point in the history
…-into-core
  • Loading branch information
jrudolph authored Oct 15, 2020
2 parents f96426e + aa35811 commit fb9e3e2
Show file tree
Hide file tree
Showing 46 changed files with 59 additions and 177 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
# Removal of internal class
ProblemFilters.exclude[Problem]("akka.http.impl.engine.Http2Shadow*")

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,15 @@
* Copyright (C) 2009-2020 Lightbend Inc. <https://www.lightbend.com>
*/

package akka.http.scaladsl
package akka.http.impl.engine.http2

import akka.actor.{ ActorSystem, ClassicActorSystemProvider, ExtendedActorSystem, Extension, ExtensionId, ExtensionIdProvider }
import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
import akka.event.LoggingAdapter
import akka.http.impl.engine.http2.{ Http2AlpnSupport, Http2Blueprint, ProtocolSwitch }
import akka.http.impl.engine.server.{ MasterServerTerminator, UpgradeToOtherProtocolResponseHeader }
import akka.http.impl.util.LogByteStringTools
import akka.http.scaladsl.{ ConnectionContext, Http, HttpsConnectionContext }
import akka.http.scaladsl.Http.ServerBinding
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers.{ Connection, RawHeader, Upgrade, UpgradeProtocol }
Expand All @@ -30,8 +31,13 @@ import scala.concurrent.duration.Duration
import scala.util.control.NonFatal
import scala.util.{ Failure, Success }

/** Entry point for Http/2 server */
final class Http2Ext(private val config: Config)(implicit val system: ActorSystem)
/**
* INTERNAL API
*
* Internal entry points for Http/2 server
*/
@InternalApi
private[http] final class Http2Ext(private val config: Config)(implicit val system: ActorSystem)
extends akka.actor.Extension {
// FIXME: won't having the same package as top-level break osgi?

Expand Down Expand Up @@ -210,7 +216,9 @@ final class Http2Ext(private val config: Config)(implicit val system: ActorSyste
}
}

object Http2 extends ExtensionId[Http2Ext] with ExtensionIdProvider {
/** INTERNAL API */
@InternalApi
private[http] object Http2 extends ExtensionId[Http2Ext] with ExtensionIdProvider {
val streamId = AttributeKey[Int]("x-http2-stream-id")

override def get(system: ActorSystem): Http2Ext = super.get(system)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import akka.http.impl.engine.http2.hpack.{ HeaderCompression, HeaderDecompressio
import akka.http.impl.engine.parsing.HttpHeaderParser
import akka.http.impl.util.LogByteStringTools.logTLSBidiBySetting
import akka.http.impl.util.StreamUtils
import akka.http.scaladsl.Http2
import akka.http.scaladsl.model._
import akka.http.scaladsl.settings.{ ClientConnectionSettings, Http2CommonSettings, ParserSettings, ServerSettings }
import akka.stream.TLSProtocol._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ import javax.net.ssl.SSLSession
import akka.annotation.InternalApi
import akka.http.impl.engine.parsing.HttpHeaderParser
import akka.http.impl.engine.server.HttpAttributes
import akka.http.scaladsl.{ Http2, model }
import akka.http.scaladsl.model
import akka.http.scaladsl.model._
import akka.http.scaladsl.model.headers.{ `Remote-Address`, `Tls-Session-Info` }
import akka.http.scaladsl.settings.ServerSettings
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import akka.http.scaladsl.settings.ServerSettings
import scala.collection.immutable
import scala.collection.immutable.VectorBuilder
import FrameEvent.ParsedHeadersFrame
import akka.http.scaladsl.Http2

private[http2] object ResponseRendering {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,8 +10,7 @@ import java.util.concurrent.atomic.AtomicInteger
import akka.annotation.InternalApi
import akka.event.LoggingAdapter
import akka.http.impl.engine.http2.FrameEvent.ParsedHeadersFrame
import akka.http.scaladsl.model.http2.RequestResponseAssociation
import akka.http.scaladsl.model.{ ContentTypes, HttpEntity, HttpRequest }
import akka.http.scaladsl.model.{ HttpRequest, RequestResponseAssociation }

import scala.collection.immutable.VectorBuilder

Expand Down
4 changes: 2 additions & 2 deletions akka-http-core/src/main/scala/akka/http/scaladsl/Http.scala
Original file line number Diff line number Diff line change
Expand Up @@ -13,9 +13,9 @@ import akka.annotation.DoNotInherit
import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
import akka.event.{ Logging, LoggingAdapter }
import akka.http.impl.engine.Http2Shadow
import akka.http.impl.engine.HttpConnectionIdleTimeoutBidi
import akka.http.impl.engine.client._
import akka.http.impl.engine.http2.Http2
import akka.http.impl.engine.server._
import akka.http.impl.engine.ws.WebSocketClientBlueprint
import akka.http.impl.settings.{ ConnectionPoolSetup, HostConnectionPoolSetup }
Expand Down Expand Up @@ -341,7 +341,7 @@ class HttpExt private[http] (private val config: Config)(implicit val system: Ex
if (parallelism > 0) settings.mapHttp2Settings(_.withMaxConcurrentStreams(parallelism))
else if (parallelism < 0) throw new IllegalArgumentException("Only positive values allowed for `parallelism`.")
else settings
Http2Shadow.bindAndHandleAsync(handler, interface, port, connectionContext, definitiveSettings, definitiveSettings.http2Settings.maxConcurrentStreams, log)(fm)
Http2().bindAndHandleAsync(handler, interface, port, connectionContext, definitiveSettings, definitiveSettings.http2Settings.maxConcurrentStreams, log)(fm)
} else {
val definitiveParallelism =
if (parallelism > 0) parallelism
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,10 +2,18 @@
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/

package akka.http.scaladsl.model.http2
package akka.http.scaladsl.model

import akka.annotation.ApiMayChange

/**
* A marker trait for attribute values that should be (automatically) carried over from request to response.
*/
@ApiMayChange
trait RequestResponseAssociation

/**
* A simple value holder class implementing RequestResponseAssociation.
*/
@ApiMayChange
final case class SimpleRequestResponseAttribute[T](value: T) extends RequestResponseAssociation
Original file line number Diff line number Diff line change
Expand Up @@ -4,18 +4,21 @@

package akka.http.scaladsl.model.http2

import akka.annotation.ApiMayChange
import akka.http.impl.engine.http2.Http2Protocol.ErrorCode

import scala.util.control.NoStackTrace

/**
* Base class for HTTP2 exceptions.
*/
@ApiMayChange
class Http2Exception(msg: String) extends RuntimeException(msg)

/**
* Exception that will be reported on the request entity stream when the peer closed the stream.
*/
@ApiMayChange
class PeerClosedStreamException(val streamId: Int, val errorCode: String, val numericErrorCode: Int)
extends Http2Exception(f"Stream with ID [$streamId%d] was closed by peer with code $errorCode%s(0x$numericErrorCode%02x)") with NoStackTrace {
private[http] def this(streamId: Int, errorCode: ErrorCode) = this(streamId, errorCode.toString, errorCode.id)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,7 @@
import akka.http.impl.util.ExampleHttpContexts;
import akka.http.javadsl.model.HttpRequest;
import akka.http.javadsl.model.HttpResponse;
import akka.http.javadsl.HttpsConnectionContext;
import akka.japi.function.Function;
import akka.stream.ActorMaterializer;
import akka.stream.Materializer;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;

Expand All @@ -31,7 +28,6 @@ public static void main(String[] args) {
"akka.http.server.preview.enable-http2 = on\n"
);
ActorSystem system = ActorSystem.create("ServerTest", testConf);
Materializer materializer = ActorMaterializer.create(system);

Function<HttpRequest, CompletionStage<HttpResponse>> handler =
request -> CompletableFuture.completedFuture(HttpResponse.create().withEntity(request.entity()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,14 +2,14 @@
* Copyright (C) 2009-2020 Lightbend Inc. <https://www.lightbend.com>
*/

package akka.http.h2spec
package akka.http.impl.engine.http2

import java.io.File
import java.util.concurrent.atomic.AtomicBoolean

import akka.http.impl.util.{ ExampleHttpContexts, WithLogCapturing }
import akka.http.scaladsl.Http
import akka.http.scaladsl.server.Directives
import akka.http.scaladsl.Http2
import akka.stream.ActorMaterializer
import akka.testkit._
import akka.util.ByteString
Expand All @@ -25,6 +25,7 @@ class H2SpecIntegrationSpec extends AkkaSpec(
loglevel = DEBUG
loggers = ["akka.http.impl.util.SilenceAllTestEventListener"]
http.server.log-unencrypted-network-bytes = off
http.server.preview.enable-http2 = on
http.server.http2.log-frames = on
actor.serialize-creators = off
Expand All @@ -43,7 +44,12 @@ class H2SpecIntegrationSpec extends AkkaSpec(
complete(data)
}

val binding = Http2().bindAndHandleAsync(echo, "127.0.0.1", 0, ExampleHttpContexts.exampleServerContext).futureValue
val binding =
Http()
.newServerAt("127.0.0.1", 0)
.enableHttps(ExampleHttpContexts.exampleServerContext)
.bind(echo)
.futureValue
val port = binding.localAddress.getPort

"H2Spec" must {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,7 @@ package akka.http.impl.engine.http2

import akka.http.impl.util._
import akka.http.scaladsl.model.{ HttpResponse, StatusCodes }
import akka.http.scaladsl.{ Http2, HttpConnectionContext }
import akka.http.scaladsl.HttpConnectionContext
import akka.stream.scaladsl.{ Source, Tcp }
import akka.util.ByteString

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,21 +2,20 @@
* Copyright (C) 2020 Lightbend Inc. <https://www.lightbend.com>
*/

package akka.http.scaladsl
package akka.http.impl.engine.http2

import akka.actor.ActorSystem
import akka.http.scaladsl.model.headers
import akka.http.scaladsl.model.headers.HttpEncodings
import akka.http.scaladsl.model.http2.RequestResponseAssociation
import akka.http.scaladsl.model.{ AttributeKey, HttpRequest, HttpResponse }
import akka.http.scaladsl.model.{ AttributeKey, HttpRequest, HttpResponse, RequestResponseAssociation, headers }
import akka.stream.OverflowStrategy
import akka.stream.scaladsl.{ Flow, Sink, Source }
import com.typesafe.config.ConfigFactory

import scala.concurrent.{ Future, Promise }
import scala.concurrent.duration._
import scala.concurrent.{ Future, Promise }

/** A small example app that shows how to use the HTTP/2 client API currently against actual internet servers */
// needs to live in impl.engine.http2 for now as we have no public access to the internal HTTP2 client
object Http2ClientApp extends App {
val config =
ConfigFactory.parseString(
Expand All @@ -30,7 +29,7 @@ object Http2ClientApp extends App {
implicit val system = ActorSystem("Http2ClientApp", config)
implicit val ec = system.dispatcher

val dispatch = singleRequest(Http2().outgoingConnection("doc.akka.io"))
val dispatch = singleRequest(Http2().outgoingConnection("doc.akka.io")) // FIXME: replace with public API

dispatch(HttpRequest(uri = "https://doc.akka.io/api/akka/current/akka/actor/typed/scaladsl/index.html", headers = headers.`Accept-Encoding`(HttpEncodings.gzip) :: Nil)).onComplete { res =>
println(s"[1] Got index.html: $res")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,13 +6,11 @@ package akka.http.impl.engine.http2

import akka.http.impl.engine.ws.ByteStringSinkProbe
import akka.http.impl.util.{ AkkaSpecWithMaterializer, ExampleHttpContexts }
import akka.http.scaladsl.model.headers
import akka.http.scaladsl.model.{ AttributeKey, ContentTypes, HttpEntity, HttpHeader, HttpMethod, HttpMethods, HttpRequest, HttpResponse, RequestResponseAssociation, StatusCode, StatusCodes, Uri, headers }
import akka.http.scaladsl.model.headers.HttpEncodings
import akka.http.scaladsl.model.http2.RequestResponseAssociation
import akka.http.scaladsl.model.{ AttributeKey, ContentTypes, HttpEntity, HttpHeader, HttpMethod, HttpMethods, HttpRequest, HttpResponse, StatusCode, StatusCodes, Uri }
import akka.http.scaladsl.settings.ClientConnectionSettings
import akka.http.scaladsl.unmarshalling.Unmarshal
import akka.http.scaladsl.{ Http, Http2 }
import akka.http.scaladsl.Http
import akka.stream.scaladsl.{ Sink, Source }
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.testkit.TestProbe
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ import akka.event.Logging
import akka.http.impl.engine.http2.FrameEvent._
import akka.http.impl.engine.http2.Http2Protocol.ErrorCode
import akka.http.impl.util.{ AkkaSpecWithMaterializer, LogByteStringTools }
import akka.http.scaladsl.Http2
import akka.http.scaladsl.model.HttpEntity.Strict
import akka.http.scaladsl.model._
import akka.http.scaladsl.settings.ClientConnectionSettings
Expand Down Expand Up @@ -58,10 +57,9 @@ class Http2ClientSpec extends AkkaSpecWithMaterializer("""
"GET request in one HEADERS frame" in new SimpleRequestResponseRoundtripSetup {
requestResponseRoundtrip(
streamId = 1,
request = HttpRequest(uri = "http://www.example.com/"),
request = HttpRequest(uri = "https://www.example.com/"),
expectedHeaders = Seq(
":method" -> "GET",
// TODO check if this makes sense?
":scheme" -> "https",
":authority" -> "www.example.com",
":path" -> "/",
Expand Down Expand Up @@ -93,7 +91,7 @@ class Http2ClientSpec extends AkkaSpecWithMaterializer("""
"GOAWAY when the response to a second request on different stream has an invalid headers frame" in new SimpleRequestResponseRoundtripSetup {
requestResponseRoundtrip(
streamId = 1,
request = HttpRequest(uri = "http://www.example.com/"),
request = HttpRequest(uri = "https://www.example.com/"),
expectedHeaders = Seq(
":method" -> "GET",
// TODO check if this makes sense?
Expand All @@ -111,7 +109,7 @@ class Http2ClientSpec extends AkkaSpecWithMaterializer("""
.withEntity(Strict(ContentTypes.NoContentType, ByteString.empty))
)

emitRequest(3, HttpRequest(uri = "http://www.example.com/"))
emitRequest(3, HttpRequest(uri = "https://www.example.com/"))
expectFrame() shouldBe a[HeadersFrame]

val incorrectHeaderBlock = hex"00 00 01 01 05 00 00 00 01 40"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,8 @@ import akka.http.impl.engine.http2.Http2Protocol.{ ErrorCode, Flags, FrameType,
import akka.http.impl.engine.http2.framing.FrameRenderer
import akka.util.{ ByteString, ByteStringBuilder }

trait Http2FrameSending {
def sendBytes(bytes: ByteString)
private[http2] trait Http2FrameSending {
def sendBytes(bytes: ByteString): Unit

def sendFrame(frame: FrameEvent): Unit =
sendBytes(FrameRenderer.render(frame))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import scala.concurrent.{ Await, Future, Promise }
import scala.concurrent.duration._
import FrameEvent._
import akka.event.Logging
import akka.http.scaladsl.Http2
import akka.stream.Attributes
import akka.stream.Attributes.LogLevels
import akka.stream.testkit.scaladsl.StreamTestKit
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import akka.testkit.AkkaSpec
import akka.util.ByteString
import org.scalatest.{ Inside, Inspectors }
import FrameEvent._
import akka.http.scaladsl.Http2

class RequestParsingSpec extends AkkaSpec() with Inside with Inspectors {

Expand Down
Loading

0 comments on commit fb9e3e2

Please sign in to comment.