splitting up TLS APIs

This commit is contained in:
Endre Sándor Varga 2016-02-16 18:19:30 +01:00 committed by Roland Kuhn
parent 650e94ba30
commit 4ba4c27b0f
34 changed files with 651 additions and 544 deletions

View file

@ -9,7 +9,7 @@ import akka.http.scaladsl.settings.{ ClientConnectionSettings, ParserSettings }
import language.existentials
import scala.annotation.tailrec
import scala.collection.mutable.ListBuffer
import akka.stream.io.{ SessionBytes, SslTlsInbound, SendBytes }
import akka.stream.TLSProtocol._
import akka.util.ByteString
import akka.event.LoggingAdapter
import akka.stream._

View file

@ -6,7 +6,7 @@ package akka.http.impl.engine.parsing
import javax.net.ssl.SSLSession
import akka.stream.io.SessionBytes
import akka.stream.TLSProtocol._
import scala.annotation.tailrec
import scala.collection.mutable.ListBuffer

View file

@ -16,7 +16,7 @@ import akka.japi.Function
import akka.event.LoggingAdapter
import akka.util.ByteString
import akka.stream._
import akka.stream.io._
import akka.stream.TLSProtocol._
import akka.stream.scaladsl._
import akka.stream.stage._
import akka.http.scaladsl.settings.ServerSettings
@ -150,10 +150,9 @@ private[http] object HttpServerBluePrint {
}
}
def createEntity(creator: EntityCreator[RequestOutput, RequestEntity]): RequestEntity =
creator match {
case StrictEntityCreator(entity) entity
case StrictEntityCreator(entity) entity
case StreamedEntityCreator(creator) streamRequestEntity(creator)
}
@ -253,7 +252,7 @@ private[http] object HttpServerBluePrint {
}
class RequestTimeoutSupport(initialTimeout: FiniteDuration)
extends GraphStage[BidiShape[HttpRequest, HttpRequest, HttpResponse, HttpResponse]] {
extends GraphStage[BidiShape[HttpRequest, HttpRequest, HttpResponse, HttpResponse]] {
private val requestIn = Inlet[HttpRequest]("requestIn")
private val requestOut = Outlet[HttpRequest]("requestOut")
private val responseIn = Inlet[HttpResponse]("responseIn")
@ -309,7 +308,7 @@ private[http] object HttpServerBluePrint {
private class TimeoutAccessImpl(request: HttpRequest, initialTimeout: FiniteDuration, requestEnd: Future[Unit],
trigger: AsyncCallback[(TimeoutAccess, HttpResponse)], materializer: Materializer)
extends AtomicReference[Future[TimeoutSetup]] with TimeoutAccess with (HttpRequest HttpResponse) { self
extends AtomicReference[Future[TimeoutSetup]] with TimeoutAccess with (HttpRequest HttpResponse) { self
import materializer.executionContext
set {
@ -351,7 +350,7 @@ private[http] object HttpServerBluePrint {
}
class ControllerStage(settings: ServerSettings, log: LoggingAdapter)
extends GraphStage[BidiShape[RequestOutput, RequestOutput, HttpResponse, ResponseRenderingContext]] {
extends GraphStage[BidiShape[RequestOutput, RequestOutput, HttpResponse, ResponseRenderingContext]] {
private val requestParsingIn = Inlet[RequestOutput]("requestParsingIn")
private val requestPrepOut = Outlet[RequestOutput]("requestPrepOut")
private val httpResponseIn = Inlet[HttpResponse]("httpResponseIn")
@ -533,7 +532,7 @@ private[http] object HttpServerBluePrint {
One2OneBidiFlow[HttpRequest, HttpResponse](pipeliningLimit).reversed
private class ProtocolSwitchStage(settings: ServerSettings, log: LoggingAdapter)
extends GraphStage[BidiShape[ResponseRenderingOutput, ByteString, SessionBytes, SessionBytes]] {
extends GraphStage[BidiShape[ResponseRenderingOutput, ByteString, SessionBytes, SessionBytes]] {
private val fromNet = Inlet[SessionBytes]("fromNet")
private val toNet = Outlet[ByteString]("toNet")

View file

@ -4,9 +4,9 @@
package akka.http.impl.engine.ws
import akka.stream.impl.io.ByteStringParser
import akka.util.ByteString
import scala.annotation.tailrec
import akka.stream.io.ByteStringParser
import akka.stream.Attributes
/**

View file

@ -13,8 +13,8 @@ import akka.util.ByteString
import akka.event.LoggingAdapter
import akka.stream.stage._
import akka.stream.BidiShape
import akka.stream.io.{ SessionBytes, SendBytes, SslTlsInbound }
import akka.stream._
import akka.stream.TLSProtocol._
import akka.stream.scaladsl._
import akka.http.scaladsl.settings.ClientConnectionSettings

View file

@ -6,7 +6,7 @@ package akka.http.javadsl
import java.util.{ Collection JCollection, Optional }
import javax.net.ssl.{ SSLContext, SSLParameters }
import akka.http.scaladsl
import akka.stream.io.ClientAuth
import akka.stream.TLSClientAuth
import scala.compat.java8.OptionConverters
@ -18,7 +18,7 @@ object ConnectionContext {
/** Used to serve HTTPS traffic. */
def https(sslContext: SSLContext, enabledCipherSuites: Optional[JCollection[String]],
enabledProtocols: Optional[JCollection[String]], clientAuth: Optional[ClientAuth], sslParameters: Optional[SSLParameters]) =
enabledProtocols: Optional[JCollection[String]], clientAuth: Optional[TLSClientAuth], sslParameters: Optional[SSLParameters]) =
scaladsl.ConnectionContext.https(sslContext, sslParameters = OptionConverters.toScala(sslParameters))
//#https-context-creation
@ -47,7 +47,7 @@ abstract class HttpsConnectionContext extends akka.http.javadsl.ConnectionContex
/** Java API */
def getEnabledProtocols: Optional[JCollection[String]]
/** Java API */
def getClientAuth: Optional[ClientAuth]
def getClientAuth: Optional[TLSClientAuth]
/** Java API */
def getSslContext: SSLContext

View file

@ -11,7 +11,7 @@ import akka.http.impl.util.JavaMapping.HttpsConnectionContext
import akka.http.javadsl.model.ws._
import akka.http.javadsl.settings.{ ConnectionPoolSettings, ClientConnectionSettings, ServerSettings }
import akka.{ NotUsed, stream }
import akka.stream.io.{ SslTlsInbound, SslTlsOutbound }
import akka.stream.TLSProtocol._
import scala.language.implicitConversions
import scala.concurrent.Future
import scala.util.Try

View file

@ -4,7 +4,8 @@
package akka.http.scaladsl
import akka.stream.io.{ ClientAuth, NegotiateNewSession }
import akka.stream.TLSClientAuth
import akka.stream.TLSProtocol._
import scala.collection.JavaConverters._
import java.util.{ Optional, Collection JCollection }
@ -22,7 +23,7 @@ object ConnectionContext {
def https(sslContext: SSLContext,
enabledCipherSuites: Option[immutable.Seq[String]] = None,
enabledProtocols: Option[immutable.Seq[String]] = None,
clientAuth: Option[ClientAuth] = None,
clientAuth: Option[TLSClientAuth] = None,
sslParameters: Option[SSLParameters] = None) = {
new HttpsConnectionContext(sslContext, enabledCipherSuites, enabledProtocols, clientAuth, sslParameters)
}
@ -35,7 +36,7 @@ final class HttpsConnectionContext(
val sslContext: SSLContext,
val enabledCipherSuites: Option[immutable.Seq[String]] = None,
val enabledProtocols: Option[immutable.Seq[String]] = None,
val clientAuth: Option[ClientAuth] = None,
val clientAuth: Option[TLSClientAuth] = None,
val sslParameters: Option[SSLParameters] = None)
extends akka.http.javadsl.HttpsConnectionContext with ConnectionContext {
@ -44,7 +45,7 @@ final class HttpsConnectionContext(
override def getSslContext = sslContext
override def getEnabledCipherSuites: Optional[JCollection[String]] = enabledCipherSuites.map(_.asJavaCollection).asJava
override def getEnabledProtocols: Optional[JCollection[String]] = enabledProtocols.map(_.asJavaCollection).asJava
override def getClientAuth: Optional[ClientAuth] = clientAuth.asJava
override def getClientAuth: Optional[TLSClientAuth] = clientAuth.asJava
override def getSslParameters: Optional[SSLParameters] = sslParameters.asJava
}

View file

@ -22,8 +22,8 @@ import akka.http.scaladsl.model.ws.{ Message, WebSocketRequest, WebSocketUpgrade
import akka.http.scaladsl.settings.{ ServerSettings, ClientConnectionSettings, ConnectionPoolSettings }
import akka.http.scaladsl.util.FastFuture
import akka.{ Done, NotUsed }
import akka.stream.Materializer
import akka.stream.io._
import akka.stream._
import akka.stream.TLSProtocol._
import akka.stream.scaladsl._
import com.typesafe.config.Config
import com.typesafe.sslconfig.akka._
@ -608,10 +608,10 @@ class HttpExt(private val config: Config)(implicit val system: ActorSystem) exte
}
/** Creates real or placebo SslTls stage based on if ConnectionContext is HTTPS or not. */
private[http] def sslTlsStage(connectionContext: ConnectionContext, role: Role, hostInfo: Option[(String, Int)] = None) =
private[http] def sslTlsStage(connectionContext: ConnectionContext, role: TLSRole, hostInfo: Option[(String, Int)] = None) =
connectionContext match {
case hctx: HttpsConnectionContext SslTls(hctx.sslContext, hctx.firstSession, role, hostInfo = hostInfo)
case other SslTlsPlacebo.forScala // if it's not HTTPS, we don't enable SSL/TLS
case hctx: HttpsConnectionContext TLS(hctx.sslContext, hctx.firstSession, role, hostInfo = hostInfo)
case other TLSPlacebo() // if it's not HTTPS, we don't enable SSL/TLS
}
}
@ -746,7 +746,7 @@ object Http extends ExtensionId[HttpExt] with ExtensionIdProvider {
* TLS configuration for an HTTPS server binding or client connection.
* For the sslContext please refer to the com.typeasfe.ssl-config library.
* The remaining four parameters configure the initial session that will
* be negotiated, see [[akka.stream.io.NegotiateNewSession]] for details.
* be negotiated, see [[NegotiateNewSession]] for details.
*/
trait DefaultSSLContextCreation {
@ -786,9 +786,9 @@ trait DefaultSSLContextCreation {
import com.typesafe.sslconfig.ssl.{ ClientAuth SslClientAuth }
val clientAuth = config.sslParametersConfig.clientAuth match {
case SslClientAuth.Default None
case SslClientAuth.Want Some(ClientAuth.Want)
case SslClientAuth.Need Some(ClientAuth.Need)
case SslClientAuth.None Some(ClientAuth.None)
case SslClientAuth.Want Some(TLSClientAuth.Want)
case SslClientAuth.Need Some(TLSClientAuth.Need)
case SslClientAuth.None Some(TLSClientAuth.None)
}
// hostname!
defaultParams.setEndpointIdentificationAlgorithm("https")

View file

@ -9,12 +9,13 @@ import java.net.InetSocketAddress
import java.security.MessageDigest
import java.util
import javax.net.ssl.SSLSession
import akka.stream.scaladsl.ScalaSessionAPI
import scala.reflect.ClassTag
import scala.util.{ Failure, Success, Try }
import scala.annotation.tailrec
import scala.collection.immutable
import akka.parboiled2.util.Base64
import akka.stream.io.ScalaSessionAPI
import akka.http.impl.util._
import akka.http.javadsl.{ model jm }
import akka.http.scaladsl.model._

View file

@ -18,8 +18,8 @@ import akka.util.ByteString
import akka.http.scaladsl.{ TestUtils, Http }
import akka.http.impl.util.{ SingletonException, StreamUtils }
import akka.http.scaladsl.settings.{ ClientConnectionSettings, ConnectionPoolSettings, ServerSettings }
import akka.stream.io.{ SessionBytes, SendBytes, SslTlsOutbound }
import akka.stream.{ BidiShape, ActorMaterializer }
import akka.stream.TLSProtocol._
import akka.stream.testkit.{ TestPublisher, TestSubscriber, AkkaSpec }
import akka.stream.scaladsl._
import akka.http.scaladsl.model.headers._

View file

@ -9,10 +9,10 @@ import scala.reflect.ClassTag
import org.scalatest.Inside
import org.scalatest.concurrent.ScalaFutures
import akka.http.scaladsl.settings.ClientConnectionSettings
import akka.stream.io.{ SessionBytes, SslTlsOutbound, SendBytes }
import akka.util.ByteString
import akka.event.NoLogging
import akka.stream.{ ClosedShape, ActorMaterializer }
import akka.stream.{ClosedShape, ActorMaterializer}
import akka.stream.TLSProtocol._
import akka.stream.testkit._
import akka.stream.scaladsl._
import akka.http.scaladsl.model.HttpEntity._

View file

@ -7,8 +7,7 @@ package akka.http.impl.engine.client
import akka.NotUsed
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.concurrent.ScalaFutures
import akka.stream.ActorMaterializer
import akka.stream.io._
import akka.stream.{ Server, Client, ActorMaterializer }
import akka.stream.scaladsl._
import akka.stream.testkit.AkkaSpec
import akka.http.impl.util._

View file

@ -18,7 +18,7 @@ import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.scaladsl._
import akka.stream.io.{ SslTlsPlacebo, SessionBytes }
import akka.stream.TLSProtocol._
import org.scalatest.matchers.Matcher
import org.scalatest.{ BeforeAndAfterAll, FreeSpec, Matchers }
@ -482,7 +482,7 @@ class RequestParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
def multiParse(parser: HttpRequestParser)(input: Seq[String]): Seq[Either[RequestOutput, StrictEqualHttpRequest]] =
Source(input.toList)
.map(bytes SessionBytes(SslTlsPlacebo.dummySession, ByteString(bytes)))
.map(bytes SessionBytes(TLSPlacebo.dummySession, ByteString(bytes)))
.transform(() parser.stage).named("parser")
.splitWhen(x x.isInstanceOf[MessageStart] || x.isInstanceOf[EntityStreamError])
.prefixAndTail(1)

View file

@ -7,7 +7,7 @@ package akka.http.impl.engine.parsing
import akka.NotUsed
import akka.http.scaladsl.settings.ParserSettings
import akka.http.scaladsl.util.FastFuture
import akka.stream.io.{ SslTlsPlacebo, SessionBytes }
import akka.stream.TLSProtocol._
import com.typesafe.config.{ ConfigFactory, Config }
import scala.concurrent.{ Future, Await }
import scala.concurrent.duration._
@ -292,7 +292,7 @@ class ResponseParserSpec extends FreeSpec with Matchers with BeforeAndAfterAll {
def rawParse(requestMethod: HttpMethod, input: String*): Source[Either[ResponseOutput, HttpResponse], NotUsed] =
Source(input.toList)
.map(bytes SessionBytes(SslTlsPlacebo.dummySession, ByteString(bytes)))
.map(bytes SessionBytes(TLSPlacebo.dummySession, ByteString(bytes)))
.transform(() newParserStage(requestMethod)).named("parser")
.splitWhen(x x.isInstanceOf[MessageStart] || x.isInstanceOf[EntityStreamError])
.prefixAndTail(1)

View file

@ -7,18 +7,17 @@ package akka.http.impl.engine.server
import java.net.InetSocketAddress
import akka.http.impl.engine.ws.ByteStringSinkProbe
import akka.http.scaladsl.settings.ServerSettings
import akka.stream.io.{ SendBytes, SslTlsOutbound, SessionBytes }
import akka.stream.TLSProtocol._
import scala.concurrent.duration.FiniteDuration
import akka.actor.ActorSystem
import akka.event.NoLogging
import akka.util.ByteString
import akka.stream.{ ClosedShape, Materializer }
import akka.stream._
import akka.stream.scaladsl._
import akka.stream.testkit.{ TestPublisher, TestSubscriber }
import akka.http.impl.util._
import akka.http.scaladsl.model.headers.{ ProductVersion, Server }
import akka.http.scaladsl.model.{ HttpResponse, HttpRequest }
import akka.stream.OverflowStrategy
abstract class HttpServerTestSetupBase {
implicit def system: ActorSystem

View file

@ -12,7 +12,6 @@ import spray.json._
import akka.actor.ActorSystem
import akka.stream.ActorMaterializer
import akka.stream.io.SslTlsPlacebo
import akka.stream.stage.{ TerminationDirective, Context, SyncDirective, PushStage }
import akka.stream.scaladsl._

View file

@ -9,6 +9,7 @@ import java.util.Random
import akka.NotUsed
import akka.http.scaladsl.model.ws.{ InvalidUpgradeResponse, WebSocketUpgradeResponse }
import akka.stream.ClosedShape
import akka.stream.TLSProtocol._
import scala.concurrent.duration._
@ -17,7 +18,6 @@ import akka.http.scaladsl.Http
import akka.http.scaladsl.model.headers.{ ProductVersion, `User-Agent` }
import akka.http.scaladsl.model.ws._
import akka.http.scaladsl.model.Uri
import akka.stream.io._
import akka.stream.scaladsl._
import akka.stream.testkit.{ TestSubscriber, TestPublisher }
import akka.util.ByteString

View file

@ -17,7 +17,6 @@ import akka.stream.scaladsl._
import akka.stream.testkit._
import akka.stream.scaladsl.GraphDSL.Implicits._
import org.scalatest.concurrent.Eventually
import akka.stream.io.SslTlsPlacebo
import java.net.InetSocketAddress
import akka.stream.impl.fusing.GraphStages
import akka.util.ByteString
@ -78,7 +77,7 @@ class WebSocketIntegrationSpec extends AkkaSpec("akka.stream.materializer.debug.
Source.empty
.viaMat {
Http().webSocketClientLayer(WebSocketRequest("ws://localhost:" + myPort))
.atop(SslTlsPlacebo.forScala)
.atop(TLSPlacebo())
.joinMat(Flow.fromGraph(GraphStages.breaker[ByteString]).via(
Tcp().outgoingConnection(new InetSocketAddress("localhost", myPort), halfClose = true)))(Keep.both)
}(Keep.right)
@ -158,7 +157,7 @@ class WebSocketIntegrationSpec extends AkkaSpec("akka.stream.materializer.debug.
Source.maybe
.viaMat {
Http().webSocketClientLayer(WebSocketRequest("ws://localhost:" + myPort))
.atop(SslTlsPlacebo.forScala)
.atop(TLSPlacebo())
// the resource leak of #19398 existed only for severed websocket connections
.atopMat(GraphStages.bidiBreaker[ByteString, ByteString])(Keep.right)
.join(Tcp().outgoingConnection(new InetSocketAddress("localhost", myPort), halfClose = true))

View file

@ -10,7 +10,7 @@ import javax.net.ssl.{ SSLParameters, SSLContext }
import akka.http.javadsl.model.headers.Cookie
import akka.http.scaladsl.model
import akka.http.scaladsl.model.headers.BasicHttpCredentials
import akka.stream.io.ClientAuth
import akka.stream.TLSClientAuth
import org.scalatest.{ FreeSpec, MustMatchers }
import scala.collection.immutable
@ -64,7 +64,7 @@ class JavaApiTestCaseSpecs extends FreeSpec with MustMatchers {
akka.http.javadsl.ConnectionContext.https(SSLContext.getDefault,
Optional.empty[java.util.Collection[String]],
Optional.empty[java.util.Collection[String]],
Optional.empty[ClientAuth],
Optional.empty[TLSClientAuth],
Optional.empty[SSLParameters]) mustNot be(null)
}
}