Skip to content

Commit

Permalink
Introduce SPI to be able to support different authentication directiv…
Browse files Browse the repository at this point in the history
…es. (apache#3829)
  • Loading branch information
mhenke1 authored and markusthoemmes committed Jul 16, 2018
1 parent b074a85 commit 9333557
Show file tree
Hide file tree
Showing 7 changed files with 98 additions and 82 deletions.
2 changes: 2 additions & 0 deletions ansible/group_vars/all
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,8 @@ controller:
seedNodes: "{{ groups['controllers'] | map('extract', hostvars, 'ansible_host') | list }}"
loadbalancer:
spi: "{{ controller_loadbalancer_spi | default('') }}"
authentication:
spi: "{{ controller_authentication_spi | default('') }}"
loglevel: "{{ controller_loglevel | default(whisk_loglevel) | default('INFO') }}"
entitlement:
spi: "{{ controller_entitlement_spi | default('') }}"
Expand Down
1 change: 1 addition & 0 deletions ansible/roles/controller/tasks/deploy.yml
Original file line number Diff line number Diff line change
Expand Up @@ -233,6 +233,7 @@
"{{ controller.loadbalancer.spi }}"
"CONFIG_whisk_spi_EntitlementSpiProvider": "{{ controller.entitlement.spi }}"

"CONFIG_whisk_spi_AuthenticationDirectiveProvider": "{{ controller.authentication.spi }}"
"CONFIG_logback_log_level": "{{ controller.loglevel }}"

"CONFIG_whisk_transactions_header": "{{ transactions.header }}"
Expand Down
1 change: 1 addition & 0 deletions common/scala/src/main/resources/reference.conf
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ whisk.spi {
LogStoreProvider = whisk.core.containerpool.logging.DockerToActivationLogStoreProvider
LoadBalancerProvider = whisk.core.loadBalancer.ShardingContainerPoolBalancer
EntitlementSpiProvider = whisk.core.entitlement.LocalEntitlementProvider
AuthenticationDirectiveProvider = whisk.core.controller.BasicAuthenticationDirective
}

dispatchers {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,40 +17,11 @@

package whisk.core.controller

import scala.concurrent.ExecutionContext
import scala.concurrent.Future

import akka.http.scaladsl.server.Route
import akka.http.scaladsl.server.Directives._
import akka.http.scaladsl.model.headers.BasicHttpCredentials
import akka.http.scaladsl.server.directives._
import akka.http.scaladsl.server.directives.AuthenticationResult
import akka.http.scaladsl.model.headers._

import whisk.common.TransactionId
import whisk.core.entity.Identity

/** A common trait for secured routes */
trait AuthenticatedRoute {

/** An execution context for futures */
protected implicit val executionContext: ExecutionContext

/** Creates HTTP BasicAuth handler */
def basicAuth[A](verify: Option[BasicHttpCredentials] => Future[Option[A]]) = {
authenticateOrRejectWithChallenge[BasicHttpCredentials, A] { creds =>
verify(creds).map {
case Some(t) => AuthenticationResult.success(t)
case None => AuthenticationResult.failWithChallenge(HttpChallenges.basic("OpenWhisk secure realm"))
}
}
}

/** Validates credentials against database of subjects */
protected def validateCredentials(credentials: Option[BasicHttpCredentials])(
implicit transid: TransactionId): Future[Option[Identity]]
}

/** A trait for authenticated routes. */
trait AuthenticatedRouteProvider {
def routes(user: Identity)(implicit transid: TransactionId): Route
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,35 +17,23 @@

package whisk.core.controller

import akka.http.scaladsl.model.headers.BasicHttpCredentials
import akka.http.scaladsl.model.headers._

import scala.concurrent.ExecutionContext
import scala.concurrent.Future
import scala.util.Try

import whisk.common.Logging
import whisk.common.TransactionId
import akka.http.scaladsl.server.Directives._
import akka.http.scaladsl.server.directives.{AuthenticationDirective, AuthenticationResult}
import whisk.common.{Logging, TransactionId}
import whisk.core.database.NoDocumentException
import whisk.core.entity._
import whisk.core.entity.types.AuthStore
import whisk.core.entity.BasicAuthenticationAuthKey
import whisk.core.entity.Identity
import whisk.core.entity.Secret
import whisk.core.entity.UUID

trait Authenticate {
protected implicit val executionContext: ExecutionContext
protected implicit val logging: Logging
import scala.concurrent.{ExecutionContext, Future}
import scala.util.Try

/** Database service to lookup credentials */
protected val authStore: AuthStore
object BasicAuthenticationDirective extends AuthenticationDirectiveProvider {

/**
* Validates credentials against the authentication database; may be used in
* authentication directive.
*/
def validateCredentials(credentials: Option[BasicHttpCredentials])(
implicit transid: TransactionId): Future[Option[Identity]] = {
def validateCredentials(credentials: Option[BasicHttpCredentials])(implicit transid: TransactionId,
ec: ExecutionContext,
logging: Logging,
authStore: AuthStore): Future[Option[Identity]] = {
credentials flatMap { pw =>
Try {
// authkey deserialization is wrapped in a try to guard against malformed values
Expand All @@ -72,4 +60,24 @@ trait Authenticate {
Future.successful(None)
}
}

/** Creates HTTP BasicAuth handler */
def basicAuth[A](verify: Option[BasicHttpCredentials] => Future[Option[A]]): AuthenticationDirective[A] = {
extractExecutionContext.flatMap { implicit ec =>
authenticateOrRejectWithChallenge[BasicHttpCredentials, A] { creds =>
verify(creds).map {
case Some(t) => AuthenticationResult.success(t)
case None => AuthenticationResult.failWithChallenge(HttpChallenges.basic("OpenWhisk secure realm"))
}
}
}
}

def authenticate(implicit transid: TransactionId,
authStore: AuthStore,
logging: Logging): AuthenticationDirective[Identity] = {
extractExecutionContext.flatMap { implicit ec =>
basicAuth(validateCredentials)
}
}
}
47 changes: 28 additions & 19 deletions core/controller/src/main/scala/whisk/core/controller/RestAPIs.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._
import akka.http.scaladsl.model.StatusCodes._
import akka.http.scaladsl.model.Uri
import akka.http.scaladsl.model.headers._
import akka.http.scaladsl.server.directives.AuthenticationDirective
import akka.http.scaladsl.server.{Directives, Route}
import akka.stream.ActorMaterializer
import pureconfig.loadConfigOrThrow
Expand All @@ -37,6 +38,7 @@ import whisk.core.entity.types._
import whisk.core.loadBalancer.LoadBalancer
import whisk.core.{ConfigKeys, WhiskConfig}
import whisk.http.Messages
import whisk.spi.{Spi, SpiLoader}

import scala.concurrent.ExecutionContext
import scala.util.{Failure, Success, Try}
Expand Down Expand Up @@ -168,13 +170,14 @@ class RestAPIVersion(config: WhiskConfig, apiPath: String, apiVersion: String)(
implicit val logStore: LogStore,
implicit val whiskConfig: WhiskConfig)
extends SwaggerDocs(Uri.Path(apiPath) / apiVersion, "apiv1swagger.json")
with Authenticate
with AuthenticatedRoute
with RespondWithHeaders {
implicit val executionContext = actorSystem.dispatcher
implicit val authStore = WhiskAuthStore.datastore()
val whiskInfo = loadConfigOrThrow[WhiskInformation](ConfigKeys.buildInformation)

private implicit val authenticationDirectiveProvider =
SpiLoader.get[AuthenticationDirectiveProvider]

def prefix = pathPrefix(apiPath / apiVersion)

/**
Expand All @@ -194,33 +197,33 @@ class RestAPIVersion(config: WhiskConfig, apiPath: String, apiVersion: String)(
def routes(implicit transid: TransactionId): Route = {
prefix {
sendCorsHeaders {
info ~ basicAuth(validateCredentials) { user =>
namespaces.routes(user) ~
pathPrefix(Collection.NAMESPACES) {
actions.routes(user) ~
triggers.routes(user) ~
rules.routes(user) ~
activations.routes(user) ~
packages.routes(user)
}
} ~ {
info ~
authenticationDirectiveProvider.authenticate(transid, authStore, logging) { user =>
namespaces.routes(user) ~
pathPrefix(Collection.NAMESPACES) {
actions.routes(user) ~
triggers.routes(user) ~
rules.routes(user) ~
activations.routes(user) ~
packages.routes(user)
}
} ~
swaggerRoutes
}
} ~ {
// web actions are distinct to separate the cors header
// and allow the actions themselves to respond to options
basicAuth(validateCredentials) { user =>
authenticationDirectiveProvider.authenticate(transid, authStore, logging) { user =>
web.routes(user)
} ~ {
web.routes()
} ~ options {
sendCorsHeaders {
complete(OK)
} ~
options {
sendCorsHeaders {
complete(OK)
}
}
}
}
}

}

private val namespaces = new NamespacesApi(apiPath, apiVersion)
Expand Down Expand Up @@ -315,3 +318,9 @@ class RestAPIVersion(config: WhiskConfig, apiPath: String, apiVersion: String)(
extends WhiskWebActionsApi
with WhiskServices
}

trait AuthenticationDirectiveProvider extends Spi {
def authenticate(implicit transid: TransactionId,
authStore: AuthStore,
logging: Logging): AuthenticationDirective[Identity]
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import org.scalatest.junit.JUnitRunner

import akka.http.scaladsl.model.headers.BasicHttpCredentials

import whisk.core.controller.Authenticate
import whisk.core.controller.BasicAuthenticationDirective
import whisk.core.entity._
import whisk.core.entitlement.Privilege

Expand All @@ -41,7 +41,7 @@ import whisk.core.entitlement.Privilege
* "using Specs2RouteTest DSL to chain HTTP requests for unit testing, as in ~>"
*/
@RunWith(classOf[JUnitRunner])
class AuthenticateTests extends ControllerTestCommon with Authenticate {
class BasicAuthenticateTests extends ControllerTestCommon {
behavior of "Authenticate"

it should "authorize a known user using different namespaces and cache key, and reject invalid secret" in {
Expand All @@ -66,15 +66,21 @@ class AuthenticateTests extends ControllerTestCommon with Authenticate {
withClue(s"Trying to login to $ns") {
waitOnView(authStore, ns.authkey, 1) // wait for the view to be updated
val pass = BasicHttpCredentials(ns.authkey.uuid.asString, ns.authkey.key.asString)
val user = Await.result(validateCredentials(Some(pass)), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
user.get shouldBe Identity(subject, ns.namespace, ns.authkey, Privilege.ALL)

// first lookup should have been from datastore
stream.toString should include(s"serving from datastore: ${CacheKey(ns.authkey)}")
stream.reset()

// repeat query, now should be served from cache
val cachedUser = Await.result(validateCredentials(Some(pass))(transid()), dbOpTimeout)
val cachedUser = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
cachedUser.get shouldBe Identity(subject, ns.namespace, ns.authkey, Privilege.ALL)

stream.toString should include(s"serving from cache: ${CacheKey(ns.authkey)}")
Expand All @@ -87,7 +93,10 @@ class AuthenticateTests extends ControllerTestCommon with Authenticate {
val key = ns.authkey.key.asString
Seq(key.drop(1), key.dropRight(1), key + "x", BasicAuthenticationAuthKey().key.asString).foreach { k =>
val pass = BasicHttpCredentials(ns.authkey.uuid.asString, k)
val user = Await.result(validateCredentials(Some(pass)), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
user shouldBe empty
}
}
Expand All @@ -96,7 +105,9 @@ class AuthenticateTests extends ControllerTestCommon with Authenticate {
implicit val tid = transid()
val creds = WhiskAuthHelpers.newIdentity()
val pass = creds.authkey.getCredentials.asInstanceOf[Option[BasicHttpCredentials]]
val user = Await.result(validateCredentials(pass), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective.validateCredentials(pass)(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
stream.toString should not include pass.get.password
}
Expand All @@ -105,34 +116,47 @@ class AuthenticateTests extends ControllerTestCommon with Authenticate {
implicit val tid = transid()
val creds = WhiskAuthHelpers.newIdentity()
val pass = creds.authkey.getCredentials.asInstanceOf[Option[BasicHttpCredentials]]
val user = Await.result(validateCredentials(pass), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective.validateCredentials(pass)(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
}

it should "not authorize when no user creds are provided" in {
implicit val tid = transid()
val user = Await.result(validateCredentials(None), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective.validateCredentials(None)(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
}

it should "not authorize when malformed user is provided" in {
implicit val tid = transid()
val pass = BasicHttpCredentials("x", Secret().asString)
val user = Await.result(validateCredentials(Some(pass)), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
}

it should "not authorize when malformed secret is provided" in {
implicit val tid = transid()
val pass = BasicHttpCredentials(UUID().asString, "x")
val user = Await.result(validateCredentials(Some(pass)), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
}

it should "not authorize when malformed creds are provided" in {
implicit val tid = transid()
val pass = BasicHttpCredentials("x", "y")
val user = Await.result(validateCredentials(Some(pass)), dbOpTimeout)
val user = Await.result(
BasicAuthenticationDirective
.validateCredentials(Some(pass))(transid, executionContext, logging, authStore),
dbOpTimeout)
user should be(None)
}
}

0 comments on commit 9333557

Please sign in to comment.