GEARPUMP-5, Add additional authorization check like checking user-organization 
for cloudfoundry OAuth2 Authenticator.


Project: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/repo
Commit: 
http://git-wip-us.apache.org/repos/asf/incubator-gearpump/commit/37e0a381
Tree: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/tree/37e0a381
Diff: http://git-wip-us.apache.org/repos/asf/incubator-gearpump/diff/37e0a381

Branch: refs/heads/master
Commit: 37e0a38136d14a9cf96814fced72d4733415c8d0
Parents: 77a5bf7
Author: Sean Zhong <[email protected]>
Authored: Wed Mar 30 15:22:42 2016 +0800
Committer: manuzhang <[email protected]>
Committed: Tue Apr 26 14:24:22 2016 +0800

----------------------------------------------------------------------
 conf/gear.conf                                  |  37 ++++++-
 core/src/main/resources/geardefault.conf        |  38 ++++++-
 docs/deployment-ui-authentication.md            |  20 +++-
 .../yarn/appmaster/CommandSpec.scala            |   4 +-
 .../io/gearpump/services/SecurityService.scala  |  23 ++--
 .../security/oauth2/OAuth2Authenticator.scala   |   9 +-
 .../oauth2/impl/BaseOAuth2Authenticator.scala   |  89 ++++++++-------
 .../CloudFoundryUAAOAuth2Authenticator.scala    | 110 +++++++++++++++++--
 ...CloudFoundryUAAOAuth2AuthenticatorSpec.scala |  12 +-
 .../oauth2/GoogleOAuth2AuthenticatorSpec.scala  |   7 +-
 10 files changed, 262 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/conf/gear.conf
----------------------------------------------------------------------
diff --git a/conf/gear.conf b/conf/gear.conf
index d8e8b4c..e7f14af 100644
--- a/conf/gear.conf
+++ b/conf/gear.conf
@@ -488,12 +488,12 @@ gearpump-ui {
         ## Please replace "127.0.0.1:8090" with your address of UI service.
         "callback" = 
"http://127.0.0.1:8090/login/oauth2/cloudfoundryuaa/callback";
 
-        ## Client Id and client secret you applied on Google.
+        ## Client Id and client secret you applied on UAA.
         ##
         ## !!NOTE!! Replace clientID and clientSecret with your own 
application to avoid
         ## potential privacy leakage, the values set here serves as a test 
application.
-        "clientid" = "gearpump_test2"
-        "clientsecret" = "gearpump_test2"
+        "clientid" = "<your client id registered on UAA>"
+        "clientsecret" = "<your client secret registered on UAA>"
 
         ## The default role we assign to user when user get authenticated by 
UAA.
         ##
@@ -508,13 +508,38 @@ gearpump-ui {
         ## Login icon disiplayed on UI server frontend
         icon = "/icons/uaa.png"
 
-        ## The hostname of cloudfoudry UAA server prefixed by "http://"; or 
"https://";
-        ## !!NOTE!! Please relace uaahost with your actual Cloudfounudry UAA 
server, the
+        ## Cloud Foundry UAA Login Endpoint. Prefixed by "http://"; or 
"https://";
+        ##
+        ## !!NOTE!! Please relace uaahost with your actual Cloudfounudry UAA 
login endpoint, the
         ## value set here serves as an example.
-        uaahost = "http://login.gearpump.gotapaas.eu";
+        uaahost = "http://<cloud foundry login endpoint>"
+
+
+        ## Whether to enable additional authorization check.
+        ## If the user fails the check, then Gearpump would log user out.
+        additional-authenticator-enabled = true
+
+        ## Define how to do additional authorization check. The class should 
implement
+        ## interface CloudFoundryUAAOAuth2Authenticator.AdditionalAuthenticator
+        additional-authenticator = {
+          "class" = 
"io.gearpump.services.security.oauth2.impl.CloudFoundryUAAOAuth2Authenticator$OrganizationAccessChecker"
+
+          ## Please fill the Cloud Foundry API endpoint and organization GUID
+          "organization-url" = "http://<cloud foundry api 
endpoint>/v2/organizations/<organization-guid>"
+        }
       }
     }
   }
+
+  ## Akka http dispatcher settings.
+  akka.stream.materializer.dispatcher = "gearpump.ui-default-dispatcher"
+
+  ## Timeout settings for akka-http
+  akka.http.server.idle-timeout = 300 s
+
+  ## When uploading a large job jar, it takes a long time to launch the job 
jar in cluster
+  ## before we respond to client.
+  akka.http.server.request-timeout = 600 s
 }
 
 ## Configurations only visible on Linux or Mac.

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/core/src/main/resources/geardefault.conf
----------------------------------------------------------------------
diff --git a/core/src/main/resources/geardefault.conf 
b/core/src/main/resources/geardefault.conf
index 663c244..d34a656 100644
--- a/core/src/main/resources/geardefault.conf
+++ b/core/src/main/resources/geardefault.conf
@@ -111,6 +111,18 @@ gearpump {
   ### If you want to use metrics, please change
   ###########################
 
+  ## Akka Http Dispatcher settings for UI service.
+  ui-default-dispatcher {
+    type = "Dispatcher"
+    executor = "thread-pool-executor"
+    throughput = 1
+
+    thread-pool-executor {
+      core-pool-size-min = 2
+      core-pool-size-factor = 2.0
+      core-pool-size-max = 32
+    }
+  }
 
   ### Flag to enable metrics
   metrics {
@@ -487,12 +499,12 @@ gearpump-ui {
         ## Please replace "127.0.0.1:8090" with your address of UI service.
         "callback" = 
"http://127.0.0.1:8090/login/oauth2/cloudfoundryuaa/callback";
 
-        ## Client Id and client secret you applied on Google.
+        ## Client Id and client secret you applied on UAA.
         ##
         ## !!NOTE!! Replace clientID and clientSecret with your own 
application to avoid
         ## potential privacy leakage, the values set here serves as a test 
application.
-        "clientid" = "gearpump_test2"
-        "clientsecret" = "gearpump_test2"
+        "clientid" = "<your client id registered on UAA>"
+        "clientsecret" = "<your client secret registered on UAA>"
 
         ## The default role we assign to user when user get authenticated by 
UAA.
         ##
@@ -507,10 +519,24 @@ gearpump-ui {
         ## Login icon disiplayed on UI server frontend
         icon = "/icons/uaa.png"
 
-        ## The hostname of cloudfoudry UAA server prefixed by "http://"; or 
"https://";
-        ## !!NOTE!! Please relace uaahost with your actual Cloudfounudry UAA 
server, the
+        ## Cloud Foundry UAA Login Endpoint. Prefixed by "http://"; or 
"https://";
+        ##
+        ## !!NOTE!! Please relace uaahost with your actual Cloudfounudry UAA 
login endpoint, the
         ## value set here serves as an example.
-        uaahost = "http://login.gearpump.gotapaas.eu";
+        uaahost = "http://<cloud foundry login endpoint>"
+
+        ## Whether to enable additional authorization check.
+        ## If the user fails the check, then Gearpump would log user out.
+        additional-authenticator-enabled = false
+
+        ## Define how to do additional authorization check. The class should 
implement
+        ## interface CloudFoundryUAAOAuth2Authenticator.AdditionalAuthenticator
+        additional-authenticator = {
+          "class" = 
"io.gearpump.services.security.oauth2.impl.CloudFoundryUAAOAuth2Authenticator$OrganizationAccessChecker"
+
+          ## Please fill the Cloud Foundry API endpoint and organization GUID
+          "organization-url" = "http://<cloud foundry api 
endpoint>/v2/organizations/<organization-guid>"
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/docs/deployment-ui-authentication.md
----------------------------------------------------------------------
diff --git a/docs/deployment-ui-authentication.md 
b/docs/deployment-ui-authentication.md
index ac5a511..75fc75a 100644
--- a/docs/deployment-ui-authentication.md
+++ b/docs/deployment-ui-authentication.md
@@ -218,17 +218,21 @@ To use Google OAuth2 Authenticator, there are several 
steps:
 #### Step1: Register your application to UAA with `uaac`
 
 1. Check tutorial on uaac at 
[https://docs.cloudfoundry.org/adminguide/uaa-user-management.html](https://docs.cloudfoundry.org/adminguide/uaa-user-management.html)
-2. Open a bash shell, and login in as user admin by
+2. Open a bash shell, set the UAA server by command `uaac target`
+    ```
+      uaac target [your uaa server url]
+    ```
+3. Login in as user admin by
 
    ```
      uaac token client get admin -s MyAdminPassword
    ```
-3. Create a new Application (Client) in UAA,
+4. Create a new Application (Client) in UAA,
    ```
     uaac client add [your_client_id]
-      --scope openid
+      --scope "openid cloud_controller.read"
       --authorized_grant_types "authorization_code client_credentials 
refresh_token"
-      --authorities openid
+      --authorities "openid cloud_controller.read"
       --redirect_uri [your_redirect_url]
       --autoapprove true
       --secret [your_client_secret]
@@ -252,6 +256,14 @@ For guide of how to configure web proxy for UI server, 
please refer to please re
 
 #### Step4: Restart the UI server and try to click the CloudFoundry login icon 
on UI server.
 
+#### Step5: You can also enable additional authenticator for CloudFoundry UAA 
by setting config:
+
+```
+additional-authenticator-enabled = true
+```
+
+Please see description in gear.conf for more information.
+
 #### Extends OAuth2Authenticator to support new Authorization service like 
Facebook, or Twitter.
 
 You can follow the Google OAuth2 example code to define a custom 
OAuth2Authenticator. Basically, the steps includes:

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/experiments/yarn/src/test/scala/io/gearpump/experiments/yarn/appmaster/CommandSpec.scala
----------------------------------------------------------------------
diff --git 
a/experiments/yarn/src/test/scala/io/gearpump/experiments/yarn/appmaster/CommandSpec.scala
 
b/experiments/yarn/src/test/scala/io/gearpump/experiments/yarn/appmaster/CommandSpec.scala
index 67047d2..3f1aed1 100644
--- 
a/experiments/yarn/src/test/scala/io/gearpump/experiments/yarn/appmaster/CommandSpec.scala
+++ 
b/experiments/yarn/src/test/scala/io/gearpump/experiments/yarn/appmaster/CommandSpec.scala
@@ -68,14 +68,14 @@ class CommandSpec extends FlatSpec with Matchers with 
BeforeAndAfterAll {
     val version = "gearpump-0.1"
     val master = MasterCommand(config, version, HostPort("127.0.0.1", 8080))
 
-    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp 
conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH
 -Dgearpump.cluster.masters.0=127.0.0.1:8080 -Dgearpump.hostname=127.0.0.1 
-Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 
-Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR>  
io.gearpump.cluster.main.Master -ip 127.0.0.1 -port 8080 2>&1 | /usr/bin/tee -a 
<LOG_DIR>/stderr"
+    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp 
conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH
 -Dgearpump.cluster.masters.0=127.0.0.1:8080 -Dgearpump.hostname=127.0.0.1 
-Dgearpump.master-resource-manager-container-id={{CONTAINER_ID}} 
-Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 
-Dgearpump.log.daemon.dir=<LOG_DIR> -Dgearpump.log.application.dir=<LOG_DIR>  
io.gearpump.cluster.main.Master -ip 127.0.0.1 -port 8080 2>&1 | /usr/bin/tee -a 
<LOG_DIR>/stderr"
     assert(master.get == expected)
   }
 
   "WorkerCommand" should "create correct command line" in {
     val version = "gearpump-0.1"
     val worker = WorkerCommand(config, version, HostPort("127.0.0.1", 8080), 
"worker-machine")
-    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp 
conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH
 -Dgearpump.cluster.masters.0=127.0.0.1:8080 
-Dgearpump.log.daemon.dir=<LOG_DIR> 
-Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 
-Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname=worker-machine  
io.gearpump.cluster.main.Worker  2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
+    val expected = "$JAVA_HOME/bin/java  -Xmx512m -cp 
conf:pack/gearpump-0.1/conf:pack/gearpump-0.1/lib/daemon/*:pack/gearpump-0.1/lib/*:$CLASSPATH
 -Dgearpump.cluster.masters.0=127.0.0.1:8080 
-Dgearpump.log.daemon.dir=<LOG_DIR> 
-Dgearpump.worker-resource-manager-container-id={{CONTAINER_ID}} 
-Dgearpump.home={{LOCAL_DIRS}}/{{CONTAINER_ID}}/pack/gearpump-0.1 
-Dgearpump.log.application.dir=<LOG_DIR> -Dgearpump.hostname=worker-machine  
io.gearpump.cluster.main.Worker  2>&1 | /usr/bin/tee -a <LOG_DIR>/stderr"
     assert(worker.get == expected)
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/main/scala/io/gearpump/services/SecurityService.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/main/scala/io/gearpump/services/SecurityService.scala 
b/services/jvm/src/main/scala/io/gearpump/services/SecurityService.scala
index 387ff45..8e03c43 100644
--- a/services/jvm/src/main/scala/io/gearpump/services/SecurityService.scala
+++ b/services/jvm/src/main/scala/io/gearpump/services/SecurityService.scala
@@ -21,7 +21,7 @@ package io.gearpump.services
 import akka.actor.{ActorSystem}
 import akka.http.scaladsl.model.{Uri, StatusCodes, RemoteAddress}
 import akka.http.scaladsl.model.headers.{HttpCookiePair, HttpCookie, 
HttpChallenge}
-import 
akka.http.scaladsl.server.AuthenticationFailedRejection.{CredentialsMissing}
+import 
akka.http.scaladsl.server.AuthenticationFailedRejection.{CredentialsRejected, 
CredentialsMissing}
 import akka.http.scaladsl.server._
 import akka.http.scaladsl.server.Directives._
 import akka.http.scaladsl.server.directives.FormFieldDirectives.FieldMagnet
@@ -55,10 +55,9 @@ import scala.util.{Failure, Success}
  */
 class SecurityService(inner: RouteService, implicit val system: ActorSystem) 
extends RouteService {
 
-  // Use scheme "xBasic" to avoid popping up web browser native authentication 
box.
-  private val challenge = HttpChallenge(scheme = "xBasic", realm = "gearpump", 
params = Map.empty)
+  // Use scheme "GearpumpBasic" to avoid popping up web browser native 
authentication box.
+  private val challenge = HttpChallenge(scheme = "GearpumpBasic", realm = 
"gearpump", params = Map.empty)
 
-  private val authFailedRejection = 
AuthenticationFailedRejection(CredentialsMissing, challenge)
   val LOG = LogUtil.getLogger(getClass, "AUDIT")
 
   private val config = system.settings.config
@@ -100,8 +99,12 @@ class SecurityService(inner: RouteService, implicit val 
system: ActorSystem) ext
     }
   }
 
-  private def authenticationFailed: Route = {
-    reject(authFailedRejection)
+  private def rejectMissingCredentials: Route = {
+    reject(AuthenticationFailedRejection(CredentialsMissing, challenge))
+  }
+
+  private def rejectWrongCredentials: Route = {
+    reject(AuthenticationFailedRejection(CredentialsRejected, challenge))
   }
 
   private def requireAuthentication(inner: UserSession => Route): Route = {
@@ -111,7 +114,7 @@ class SecurityService(inner: RouteService, implicit val 
system: ActorSystem) ext
           inner(session)
         }
         case None =>
-          authenticationFailed
+          rejectMissingCredentials
       }
     }
   }
@@ -180,7 +183,7 @@ class SecurityService(inner: RouteService, implicit val 
system: ActorSystem) ext
                   case Some(session) =>
                     login(session, ip.toString)
                   case None =>
-                    authenticationFailed
+                    rejectWrongCredentials
                 }
               }
             }
@@ -193,7 +196,7 @@ class SecurityService(inner: RouteService, implicit val 
system: ActorSystem) ext
         // Support OAUTH Authentication
         pathPrefix ("oauth2"/ Segment) {providerName =>
         // Resolve OAUTH Authentication Provider
-        val oauthService = OAuth2Authenticator.get(config, providerName)
+        val oauthService = OAuth2Authenticator.get(config, providerName, ec)
 
           if (oauthService == null) {
             // OAuth2 is disabled.
@@ -207,7 +210,7 @@ class SecurityService(inner: RouteService, implicit val 
system: ActorSystem) ext
                   login(session, ip.toString, redirectToRoot = true)
                 case Failure(ex) => {
                   LOG.info(s"Failed to login user from ${ip.toString}", ex)
-                  failWith(ex)
+                  rejectWrongCredentials
                 }
               }
             }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/OAuth2Authenticator.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/OAuth2Authenticator.scala
 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/OAuth2Authenticator.scala
index f44d67a..edaefa7 100644
--- 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/OAuth2Authenticator.scala
+++ 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/OAuth2Authenticator.scala
@@ -22,7 +22,7 @@ import com.typesafe.config.Config
 import io.gearpump.services.SecurityService.UserSession
 import io.gearpump.util.Constants
 import io.gearpump.util.Constants._
-import scala.concurrent.Future
+import scala.concurrent.{ExecutionContext, Future}
 
 /**
  *
@@ -72,8 +72,9 @@ trait OAuth2Authenticator {
    * @note '''Thread-Safety''': Framework ensures this call is synchronized.
    *
    * @param config Client Id, client secret, callback URL and etc..
+   * @param executionContext ExecutionContext from hosting environment.
    */
-  def init(config: Config): Unit
+  def init(config: Config, executionContext: ExecutionContext): Unit
 
   /**
    * Returns the OAuth Authorization URL so for redirection to that address to 
do OAuth2
@@ -115,7 +116,7 @@ object OAuth2Authenticator {
    * @param provider, Name for the OAuth2 Authentication Service.
    * @return Returns null if the OAuth2 Authentication is disabled.
    */
-  def get(config: Config, provider: String): OAuth2Authenticator = {
+  def get(config: Config, provider: String, executionContext: 
ExecutionContext): OAuth2Authenticator = {
 
     if (providers.contains(provider)) {
       providers(provider)
@@ -131,7 +132,7 @@ object OAuth2Authenticator {
             val authenticatorClass = 
authenticatorConfig.getString(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_CLASS)
             val clazz = 
Thread.currentThread().getContextClassLoader.loadClass(authenticatorClass)
             val authenticator = 
clazz.newInstance().asInstanceOf[OAuth2Authenticator]
-            authenticator.init(authenticatorConfig)
+            authenticator.init(authenticatorConfig, executionContext)
             providers += provider -> authenticator
             authenticator
           }

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/BaseOAuth2Authenticator.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/BaseOAuth2Authenticator.scala
 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/BaseOAuth2Authenticator.scala
index 851053d..f0ba400 100644
--- 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/BaseOAuth2Authenticator.scala
+++ 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/BaseOAuth2Authenticator.scala
@@ -35,7 +35,8 @@ import io.gearpump.util.Constants._
 import io.gearpump.util.Util
 
 import scala.collection.mutable.StringBuilder
-import scala.concurrent.{Future, Promise}
+import scala.concurrent.{ExecutionContext, Future, Promise}
+import scala.util.{Failure, Success}
 
 /**
  * Uses Ning AsyncClient to connect to OAuth2 service.
@@ -64,12 +65,14 @@ abstract class BaseOAuth2Authenticator extends 
OAuth2Authenticator {
     new BaseApi20(authorizeUrl, accessTokenEndpoint)
   }
 
-  private var oauthService: OAuth20Service = null
+  protected var oauthService: OAuth20Service = null
+
+  protected var executionContext: ExecutionContext = null
 
   private var defaultPermissionLevel = Authenticator.Guest.permissionLevel
 
   // Synchronization ensured by the caller
-  override def init(config: Config): Unit = {
+  override def init(config: Config, executionContext: ExecutionContext): Unit 
= {
     if (this.oauthService == null) {
       val callback = 
config.getString(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_CALLBACK)
       val clientId = 
config.getString(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_CLIENT_ID)
@@ -84,6 +87,7 @@ abstract class BaseOAuth2Authenticator extends 
OAuth2Authenticator {
         }
       }
       this.oauthService = buildOAuth2Service(clientId, clientSecret, callback)
+      this.executionContext = executionContext
     }
   }
 
@@ -101,60 +105,66 @@ abstract class BaseOAuth2Authenticator extends 
OAuth2Authenticator {
     oauthService.getAuthorizationUrl()
   }
 
-  override def authenticate(parameters: Map[String, String]): 
Future[UserSession] = {
 
+  protected def authenticateWithAccessToken(accessToken: OAuth2AccessToken): 
Future[UserSession] = {
     val promise = Promise[UserSession]()
-    val code = 
parameters.get(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_AUTHORIZATION_CODE)
-    val accessToken = 
parameters.get(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_ACCESS_TOKEN)
-
-    def authenticateWithAccessToken(accessToken: OAuth2AccessToken): Unit = {
-
-      val request = new OAuthRequestAsync(Verb.GET, protectedResourceUrl, 
oauthService)
-      oauthService.signRequest(accessToken, request)
-      request.sendAsync {
-        new OAuthAsyncRequestCallback[Response] {
-          override def onCompleted(response: Response): Unit = {
-            try {
-              val user = extractUserName(response.getBody)
-              promise.success(new UserSession(user, defaultPermissionLevel))
-            } catch {
-              case ex: Throwable =>
-                promise.failure(ex)
-            }
+    val request = new OAuthRequestAsync(Verb.GET, protectedResourceUrl, 
oauthService)
+    oauthService.signRequest(accessToken, request)
+    request.sendAsync {
+      new OAuthAsyncRequestCallback[Response] {
+        override def onCompleted(response: Response): Unit = {
+          try {
+            val user = extractUserName(response.getBody)
+            promise.success(new UserSession(user, defaultPermissionLevel))
+          } catch {
+            case ex: Throwable =>
+              promise.failure(ex)
           }
+        }
 
-          override def onThrowable(throwable: Throwable): Unit = {
-            promise.failure(throwable)
-          }
+        override def onThrowable(throwable: Throwable): Unit = {
+          promise.failure(throwable)
         }
       }
     }
+    promise.future
+  }
 
-    def authenticateWithAuthorizationCode(code: String): Unit = {
-      oauthService.getAccessTokenAsync(code,
+  protected def authenticateWithAuthorizationCode(code: String): 
Future[UserSession] = {
 
-        new OAuthAsyncRequestCallback[OAuth2AccessToken] {
-          override def onCompleted(accessToken: OAuth2AccessToken): Unit = {
-            authenticateWithAccessToken(accessToken)
-          }
+    implicit val ec: ExecutionContext = executionContext
 
-          override def onThrowable(throwable: Throwable): Unit = {
-            promise.failure(throwable)
+    val promise = Promise[UserSession]()
+    oauthService.getAccessTokenAsync(code,
+
+      new OAuthAsyncRequestCallback[OAuth2AccessToken] {
+        override def onCompleted(accessToken: OAuth2AccessToken): Unit = {
+          authenticateWithAccessToken(accessToken).onComplete{
+            case Success(user) => promise.success(user)
+            case Failure(ex) => promise.failure(ex)
           }
-        })
-    }
+        }
+
+        override def onThrowable(throwable: Throwable): Unit = {
+          promise.failure(throwable)
+        }
+      })
+    promise.future
+  }
+
+  override def authenticate(parameters: Map[String, String]): 
Future[UserSession] = {
+
+    val code = 
parameters.get(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_AUTHORIZATION_CODE)
+    val accessToken = 
parameters.get(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_ACCESS_TOKEN)
 
     if (accessToken.isDefined) {
       authenticateWithAccessToken(new OAuth2AccessToken(accessToken.get))
-    }
-    else if (code.isDefined) {
+    } else if (code.isDefined) {
       authenticateWithAuthorizationCode(code.get)
     } else {
       // Fails authentication if code not exist
-      promise.failure(new Exception("Fail to authenticate user as there is no 
code parameter in URL"))
+      Future.failed(new Exception("Fail to authenticate user as there is no 
code parameter in URL"))
     }
-
-    promise.future
   }
 
   private def buildOAuth2Service(clientId: String, clientSecret: String, 
callback: String): OAuth20Service = {
@@ -179,6 +189,7 @@ abstract class BaseOAuth2Authenticator extends 
OAuth2Authenticator {
 
     service
   }
+
 }
 
 object BaseOAuth2Authenticator {

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/CloudFoundryUAAOAuth2Authenticator.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/CloudFoundryUAAOAuth2Authenticator.scala
 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/CloudFoundryUAAOAuth2Authenticator.scala
index 75fa07a..27b66d2 100644
--- 
a/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/CloudFoundryUAAOAuth2Authenticator.scala
+++ 
b/services/jvm/src/main/scala/io/gearpump/services/security/oauth2/impl/CloudFoundryUAAOAuth2Authenticator.scala
@@ -19,15 +19,20 @@
 package io.gearpump.services.security.oauth2.impl
 
 import com.github.scribejava.core.builder.api.DefaultApi20
-import com.github.scribejava.core.model.{AbstractRequest, OAuthConfig, 
OAuthConstants}
+import com.github.scribejava.core.model._
 import com.github.scribejava.core.oauth.OAuth20Service
+import com.ning.http.client
+import com.ning.http.client.{AsyncCompletionHandler, AsyncHttpClient}
 import com.typesafe.config.Config
+import io.gearpump.services.SecurityService.UserSession
 import io.gearpump.services.security.oauth2.OAuth2Authenticator
 import 
io.gearpump.services.security.oauth2.impl.BaseOAuth2Authenticator.BaseApi20
-import 
io.gearpump.services.security.oauth2.impl.CloudFoundryUAAOAuth2Authenticator.CloudFoundryUAAService
+import io.gearpump.util.Constants._
 import spray.json.{JsString, _}
 import sun.misc.BASE64Encoder
 
+import scala.concurrent.{ExecutionContext, Promise, Future}
+
 /**
  *
  * Does authentication with CloudFoundry UAA service. Currently it only
@@ -41,16 +46,21 @@ import sun.misc.BASE64Encoder
  *
  * Step1: Register your website to UAA with tool uaac.
  * 1) Check tutorial on uaac at 
[[https://docs.cloudfoundry.org/adminguide/uaa-user-management.html]]
- * 2) Open a bash shell, and login in as user admin by
+ * 2) Open a bash shell, set the UAA server by command `uaac target`
+ * {{{
+ *   uaac target [your uaa server url]
+ * }}}
+ * NOTE: [your uaa server url] should match the uaahost settings in gear.conf
+ * 3) Login in as user admin by
  * {{{
  *    uaac token client get admin -s MyAdminPassword
  * }}}
- * 3) Create a new Application (Client) in UAA,
+ * 4) Create a new Application (Client) in UAA,
  * {{{
  *   uaac client add [your_client_id]
- *     --scope openid
+ *     --scope "openid cloud_controller.read"
  *     --authorized_grant_types "authorization_code client_credentials 
refresh_token"
- *     --authorities openid
+ *     --authorities "openid cloud_controller.read"
  *     --redirect_uri [your_redirect_url]
  *     --autoapprove true
  *     --secret [your_client_secret]
@@ -63,7 +73,7 @@ import sun.misc.BASE64Encoder
  * 3) Config gear.conf 
"gearpump.ui-security.oauth2-authenticators.cloudfoundryuaa" section.
  * Please make sure class name, client ID, client Secret, and callback URL are 
set properly.
  *
- * @note The callback URL here should matche what you set on CloudFoundry UAA 
in step1.
+ * @note The callback URL here should match what you set on CloudFoundry UAA 
in step1.
  *
  * Step3: Restart the UI service and try the "social login" button for UAA.
  *
@@ -74,6 +84,8 @@ import sun.misc.BASE64Encoder
  */
 class CloudFoundryUAAOAuth2Authenticator extends BaseOAuth2Authenticator {
 
+  import CloudFoundryUAAOAuth2Authenticator._
+
   private var host: String = null
 
   protected override def authorizeUrl: String = 
s"$host/oauth/authorize?response_type=%s&client_id=%s&redirect_uri=%s&scope=%s"
@@ -82,11 +94,22 @@ class CloudFoundryUAAOAuth2Authenticator extends 
BaseOAuth2Authenticator {
 
   protected override def protectedResourceUrl: String = s"$host/userinfo"
 
-  protected override def scope: String = "openid"
+  protected override def scope: String = "openid,cloud_controller.read"
+
+  private var additionalAuthenticator: Option[AdditionalAuthenticator] = None
 
-  override def init(config: Config): Unit = {
+  override def init(config: Config, executionContext: ExecutionContext): Unit 
= {
     host = config.getString("uaahost")
-    super.init(config)
+    super.init(config, executionContext)
+
+    if (config.getBoolean(ADDITIONAL_AUTHENTICATOR_ENABLED)) {
+      val additionalAuthenticatorConfig = 
config.getConfig(ADDITIONAL_AUTHENTICATOR)
+      val authenticatorClass = 
additionalAuthenticatorConfig.getString(GEARPUMP_UI_OAUTH2_AUTHENTICATOR_CLASS)
+      val clazz = 
Thread.currentThread().getContextClassLoader.loadClass(authenticatorClass)
+      val authenticator = 
clazz.newInstance().asInstanceOf[AdditionalAuthenticator]
+      authenticator.init(additionalAuthenticatorConfig, executionContext)
+      additionalAuthenticator = Option(authenticator)
+    }
   }
 
   protected override def extractUserName(body: String): String = {
@@ -98,11 +121,27 @@ class CloudFoundryUAAOAuth2Authenticator extends 
BaseOAuth2Authenticator {
   protected override def oauth2Api(): DefaultApi20 = {
     new CloudFoundryUAAService(authorizeUrl, accessTokenEndpoint)
   }
+
+  protected override def authenticateWithAccessToken(accessToken: 
OAuth2AccessToken): Future[UserSession] = {
+
+    implicit val ec: ExecutionContext = executionContext
+
+    if (additionalAuthenticator.isDefined) {
+      super.authenticateWithAccessToken(accessToken).flatMap{user =>
+        
additionalAuthenticator.get.authenticate(oauthService.getAsyncHttpClient, 
accessToken, user)
+      }
+    } else {
+      super.authenticateWithAccessToken(accessToken)
+    }
+  }
 }
 
 object CloudFoundryUAAOAuth2Authenticator {
   private val RESPONSE_TYPE = "response_type"
 
+  val ADDITIONAL_AUTHENTICATOR_ENABLED = "additional-authenticator-enabled"
+  val ADDITIONAL_AUTHENTICATOR = "additional-authenticator"
+
   private class CloudFoundryUAAService(authorizeUrl: String, 
accessTokenEndpoint: String)
     extends BaseApi20(authorizeUrl, accessTokenEndpoint) {
 
@@ -134,4 +173,55 @@ object CloudFoundryUAAOAuth2Authenticator {
       }
     }
   }
+
+  /**
+   * Additional authenticator to check more credential attributes of user 
before logging in.
+   * This authenticator is applied AFTER user pass the initial (default) 
authenticator.
+   */
+  trait AdditionalAuthenticator {
+
+    /**
+     * @param config configurations specifically used for this authenticator.
+     * @param executionContext execution Context to use to run futures.
+     */
+    def init(config: Config, executionContext: ExecutionContext): Unit
+
+    /**
+     *
+     * @param accessToken, the accessToken for the UAA
+     * @param user user session returned by previous authenticator
+     * @return an updated UserSession
+     */
+    def authenticate(asyncClient: AsyncHttpClient, accessToken: 
OAuth2AccessToken, user: UserSession): Future[UserSession]
+  }
+
+  val ORGANIZATION_URL = "organization-url"
+
+  class OrganizationAccessChecker extends AdditionalAuthenticator {
+    private var organizationUrl: String = null
+    private implicit var executionContext: ExecutionContext = null
+
+    override def init(config: Config, executionContext: ExecutionContext): 
Unit = {
+      this.organizationUrl = config.getString(ORGANIZATION_URL)
+      this.executionContext = executionContext
+    }
+
+    override def authenticate(asyncClient: AsyncHttpClient, accessToken: 
OAuth2AccessToken,
+        user: UserSession): Future[UserSession] = {
+
+      val promise = Promise[UserSession]()
+      val builder = asyncClient.prepareGet(organizationUrl)
+      builder.addHeader("Authorization", s"bearer 
${accessToken.getAccessToken}")
+      builder.execute(new AsyncCompletionHandler[Unit] {
+        override def onCompleted(response: client.Response): Unit = {
+          if (response.getStatusCode == 200) {
+            promise.success(user)
+          } else {
+            promise.failure(new Exception(response.getResponseBody))
+          }
+        }
+      })
+      promise.future
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/CloudFoundryUAAOAuth2AuthenticatorSpec.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/CloudFoundryUAAOAuth2AuthenticatorSpec.scala
 
b/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/CloudFoundryUAAOAuth2AuthenticatorSpec.scala
index df996e7..68a3506 100644
--- 
a/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/CloudFoundryUAAOAuth2AuthenticatorSpec.scala
+++ 
b/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/CloudFoundryUAAOAuth2AuthenticatorSpec.scala
@@ -46,19 +46,23 @@ class CloudFoundryUAAOAuth2AuthenticatorSpec extends 
FlatSpec with ScalatestRout
     "clientsecret" -> "gearpump_test2",
     "default-userrole" -> "user",
     "icon" -> "/icons/uaa.png",
-    "uaahost" -> serverHost)
+    "uaahost" -> serverHost,
+    "additional-authenticator-enabled" -> "false")
 
   val configString = ConfigFactory.parseMap(configMap.asJava)
 
-  private val uaa = new CloudFoundryUAAOAuth2Authenticator
-  uaa.init(configString)
+  lazy val uaa = {
+    val uaa = new CloudFoundryUAAOAuth2Authenticator
+    uaa.init(configString, system.dispatcher)
+    uaa
+  }
 
   it should "generate the correct authorization request" in {
     val parameters = Uri(uaa.getAuthorizationUrl()).query().toMap
     assert(parameters("response_type") == "code")
     assert(parameters("client_id") == configMap("clientid"))
     assert(parameters("redirect_uri") == configMap("callback"))
-    assert(parameters("scope") == "openid")
+    assert(parameters("scope") == "openid,cloud_controller.read")
   }
 
   it should "authenticate the authorization code and return the correct 
profile" in {

http://git-wip-us.apache.org/repos/asf/incubator-gearpump/blob/37e0a381/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/GoogleOAuth2AuthenticatorSpec.scala
----------------------------------------------------------------------
diff --git 
a/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/GoogleOAuth2AuthenticatorSpec.scala
 
b/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/GoogleOAuth2AuthenticatorSpec.scala
index ff57baf..58b4a34 100644
--- 
a/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/GoogleOAuth2AuthenticatorSpec.scala
+++ 
b/services/jvm/src/test/scala/io/gearpump/services/security/oauth2/GoogleOAuth2AuthenticatorSpec.scala
@@ -51,8 +51,11 @@ class GoogleOAuth2AuthenticatorSpec extends FlatSpec with 
ScalatestRouteTest {
 
   val configString = ConfigFactory.parseMap(configMap.asJava)
 
-  private val google = new MockGoogleAuthenticator(serverHost)
-  google.init(configString)
+  private lazy val google = {
+    val google = new MockGoogleAuthenticator(serverHost)
+    google.init(configString, system.dispatcher)
+    google
+  }
 
   it should "generate the correct authorization request" in {
     val parameters = Uri(google.getAuthorizationUrl()).query().toMap

Reply via email to