You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/09/06 09:37:41 UTC

[GitHub] [pulsar] tuteng opened a new pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

tuteng opened a new pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931


   
   
   ### Motivation
   
   In some scenarios (e.g. azure cloud), when the client exchanges tokens with the server, an optional scope parameter is required, this pr fixes this issue, to ensure compatibility, when the user does not fill in this parameter, all behavior is the same as before.
   
   ### Modifications
   
   * Add an optional parameter scope when exchanges access token
   
   ### Verifying this change
   
   - [ ] Make sure that the change passes the CI checks.
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   *(or)*
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Added integration tests for end-to-end deployment with large payloads (10MB)*
     - *Extended integration test for recovery after broker failure*
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (yes / no)
     - The public API: (yes / no)
     - The schema: (yes / no / don't know)
     - The default values of configurations: (yes / no)
     - The wire protocol: (yes / no)
     - The rest endpoints: (yes / no)
     - The admin cli options: (yes / no)
     - Anything that affects deployment: (yes / no / don't know)
   
   ### Documentation
   
   Check the box below and label this PR (if you have committer privilege).
   
   Need to update docs? 
   
   - [ ] doc-required 
     
     (If you need help on updating docs, create a doc issue)
     
   - [ ] no-need-doc 
     
     (Please explain why)
     
   - [ ] doc 
     
     (If this PR contains doc changes)
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] sijie commented on pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
sijie commented on pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#issuecomment-915054882


   @eolivelli please review @tuteng 's latest change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704281048



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {

Review comment:
       nit: the request.getScope() is always "test-scope", so looks like a unnecessary check.

##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {
+            bodyMap.put("scope", request.getScope());
+        }
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.execute()).thenReturn(listenableFuture);
+        when(listenableFuture.get()).thenReturn(response);
+        when(response.getStatusCode()).thenReturn(200);
+        TokenResult tokenResult = new TokenResult();
+        tokenResult.setAccessToken("test-access-token");
+        tokenResult.setIdToken("test-id");
+        when(response.getResponseBodyAsBytes()).thenReturn(new Gson().toJson(tokenResult).getBytes());
+        TokenResult tr = tokenClient.exchangeClientCredentials(request);
+        Assert.assertNotNull(tr);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsFailedByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);

Review comment:
       @tuteng Could you please provide more context about this test? I do not fully understand the purpose of this test. Here we only return the `boundRequestBuilder` if the body equals the body map, but the request always has the `scope`, so that we will always return null here. But looks like the test will not touch any response handling steps.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704755658



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java
##########
@@ -37,10 +37,24 @@
      * @return an Authentication object
      */
     public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl, String audience) {
+        return clientCredentials(issuerUrl, credentialsUrl, audience, null);
+    }
+
+    /**
+     * Authenticate with client credentials.
+     *
+     * @param issuerUrl the issuer URL
+     * @param credentialsUrl the credentials URL
+     * @param audience the audience identifier
+     * @param scope the scope

Review comment:
       Makes sense, thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng merged pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng merged pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704755431



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -76,6 +77,9 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re
         bodyMap.put("client_id", req.getClientId());
         bodyMap.put("client_secret", req.getClientSecret());
         bodyMap.put("audience", req.getAudience());

Review comment:
       I'll submit a PR to change the behavior. My main point is that the RFC for this flow (https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.2) does not mention `audience`. The only IDP I know of that requires the `audience` field is Auth0. As such, we shouldn't require that audience be set. I don't think it will break client compatibility to demote a field from being required to optional.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] Anonymitaet commented on pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
Anonymitaet commented on pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#issuecomment-913923473


   @tuteng Thanks for your contribution. Please do not forget to add docs accordingly to allow users to know your great code changes. And you can ping me to review the docs, thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r703759687



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -44,7 +45,7 @@
     protected static final int DEFAULT_READ_TIMEOUT_IN_SECONDS = 30;
 
     private final URL tokenUrl;
-    private final AsyncHttpClient httpClient;

Review comment:
       Instead of removing the `private final`, why not add a new class constructor to allow for passing in an already created `AsyncHttpClient`? That way the tests are cleaner and the class gets to maintain immutability for the http client.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -76,6 +77,9 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re
         bodyMap.put("client_id", req.getClientId());
         bodyMap.put("client_secret", req.getClientSecret());
         bodyMap.put("audience", req.getAudience());

Review comment:
       It seems to me that the `audience` field originates from the `Auth0` implementation, and is not necessary in every case. Since it is optional and since the `bodyMap` does not allow for `null` values, I think we should only put this field in the `bodyMap` if it is non empty and non null.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java
##########
@@ -37,10 +37,24 @@
      * @return an Authentication object
      */
     public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl, String audience) {
+        return clientCredentials(issuerUrl, credentialsUrl, audience, null);
+    }
+
+    /**
+     * Authenticate with client credentials.
+     *
+     * @param issuerUrl the issuer URL
+     * @param credentialsUrl the credentials URL
+     * @param audience the audience identifier
+     * @param scope the scope

Review comment:
       I think it'd be helpful to update these comments. The audience field seems tied to auth0, not oauth2.0 itself, so you might even reference the auth0 spec: https://auth0.com/docs/authorization/flows/call-your-api-using-the-client-credentials-flow. The scope is an optional part of the oauth2.0 spec defined here: https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.2.
   
   It'd also be helpful to know what is optional and what is required.

##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url);
+        tokenClient.httpClient = defaultAsyncHttpClient;
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {
+            bodyMap.put("scope", request.getScope());
+        }
+        String body = bodyMap.entrySet().stream()
+            .map(e -> {
+                try {
+                    return URLEncoder.encode(
+                            e.getKey(),
+                            "UTF-8") + '=' + URLEncoder.encode(e.getValue(), "UTF-8");
+                } catch (UnsupportedEncodingException e1) {
+                    throw new RuntimeException(e1);
+                }
+            })
+        .collect(Collectors.joining("&"));

Review comment:
       This is logic is copied twice in this test and is already part of the `TokenClient` class's `exchangeClientCredentials` method. I think we should pull it out into a utility method named something like `buildClientCredentialsBody` and have it take the `ClientCredentialsExchangeRequest`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704312859



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {
+            bodyMap.put("scope", request.getScope());
+        }
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.execute()).thenReturn(listenableFuture);
+        when(listenableFuture.get()).thenReturn(response);
+        when(response.getStatusCode()).thenReturn(200);
+        TokenResult tokenResult = new TokenResult();
+        tokenResult.setAccessToken("test-access-token");
+        tokenResult.setIdToken("test-id");
+        when(response.getResponseBodyAsBytes()).thenReturn(new Gson().toJson(tokenResult).getBytes());
+        TokenResult tr = tokenClient.exchangeClientCredentials(request);
+        Assert.assertNotNull(tr);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsFailedByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);

Review comment:
       The purpose of this test is to test that the `scope` is correctly encoded in the request body, because it is a unit test, so there is no way to get the real response, In this unit test, only this part of the logic in this function `exchangeClientCredentials` can be tested
   
   ```
          Map<String, String> bodyMap = new TreeMap<>();
           bodyMap.put("grant_type", "client_credentials");
           bodyMap.put("client_id", req.getClientId());
           bodyMap.put("client_secret", req.getClientSecret());
           bodyMap.put("audience", req.getAudience());
           if (!StringUtils.isBlank(req.getScope())) {
               bodyMap.put("scope", req.getScope());
           }
           String body = buildClientCredentialsBody(bodyMap);
   ```
   because it has no way to actually send an http request to a server, all http-related operations are mocked, 
   So there is the following mock call:
   ```
   when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
   ```
   Note that there is a body that is mocked here, and the test will pass when it has the same value as the body in the following function, otherwise an exception will be thrown, the body in the following function is generated in the exchangeClientCredentials function:
   
   ```
   Response res = httpClient.preparePost(tokenUrl.toString())
                       .setHeader("Accept", "application/json")
                       .setHeader("Content-Type", "application/x-www-form-urlencoded")
                       .setBody(body)
                       .execute()
                       .get();
   ```
   
   This is the current way to verify that the scope is encoded correctly
   
   @codelipenghui 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704122834



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -76,6 +77,9 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re
         bodyMap.put("client_id", req.getClientId());
         bodyMap.put("client_secret", req.getClientSecret());
         bodyMap.put("audience", req.getAudience());

Review comment:
       I haven't seen the audience related specification yet, if I want to change this configuration, there might be other parts that need to be updated, for example https://github.com/apache/pulsar/blob/master/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java#L61, maybe other needs need to be updated as well, I'm worried about breaking client compatibility, I want to fixed it in the next pr




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#issuecomment-913672203


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#issuecomment-913941557


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704312859



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {
+            bodyMap.put("scope", request.getScope());
+        }
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.execute()).thenReturn(listenableFuture);
+        when(listenableFuture.get()).thenReturn(response);
+        when(response.getStatusCode()).thenReturn(200);
+        TokenResult tokenResult = new TokenResult();
+        tokenResult.setAccessToken("test-access-token");
+        tokenResult.setIdToken("test-id");
+        when(response.getResponseBodyAsBytes()).thenReturn(new Gson().toJson(tokenResult).getBytes());
+        TokenResult tr = tokenClient.exchangeClientCredentials(request);
+        Assert.assertNotNull(tr);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsFailedByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);

Review comment:
       In the unit test, only this part of the logic in this function `exchangeClientCredentials` can be tested
   
   ```
          Map<String, String> bodyMap = new TreeMap<>();
           bodyMap.put("grant_type", "client_credentials");
           bodyMap.put("client_id", req.getClientId());
           bodyMap.put("client_secret", req.getClientSecret());
           bodyMap.put("audience", req.getAudience());
           if (!StringUtils.isBlank(req.getScope())) {
               bodyMap.put("scope", req.getScope());
           }
           String body = buildClientCredentialsBody(bodyMap);
   ```
   because it has no way to actually send an http request to a server, all http-related operations are mocked, 
   So there is the following mock call:
   ```
   when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
   ```
   Note that there is a body that is mocked here, and the test will pass when it has the same value as the body in the following function, otherwise an exception will be thrown, the body in the following function is generated in the exchangeClientCredentials function:
   
   ```
   Response res = httpClient.preparePost(tokenUrl.toString())
                       .setHeader("Accept", "application/json")
                       .setHeader("Content-Type", "application/x-www-form-urlencoded")
                       .setBody(body)
                       .execute()
                       .get();
   ```
   
   This is the current way to verify that the scope is encoded correctly
   
   @codelipenghui 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704122834



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -76,6 +77,9 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re
         bodyMap.put("client_id", req.getClientId());
         bodyMap.put("client_secret", req.getClientSecret());
         bodyMap.put("audience", req.getAudience());

Review comment:
       I haven't seen the audience related specification yet, if I want to change this configuration, there might be other parts that need to be updated, for example https://github.com/apache/pulsar/blob/master/pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/FlowBase.java#L61, maybe other language need to be updated as well, I'm worried about breaking client compatibility, I want to fixed it in the next pr




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704123411



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationFactoryOAuth2.java
##########
@@ -37,10 +37,24 @@
      * @return an Authentication object
      */
     public static Authentication clientCredentials(URL issuerUrl, URL credentialsUrl, String audience) {
+        return clientCredentials(issuerUrl, credentialsUrl, audience, null);
+    }
+
+    /**
+     * Authenticate with client credentials.
+     *
+     * @param issuerUrl the issuer URL
+     * @param credentialsUrl the credentials URL
+     * @param audience the audience identifier
+     * @param scope the scope

Review comment:
       Fixed `scope`, but without changing the audience




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#issuecomment-915059246


   /pulsarbot run-failure-checks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] codelipenghui commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704320570



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClientTest.java
##########
@@ -0,0 +1,125 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.client.impl.auth.oauth2.protocol;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+import org.asynchttpclient.BoundRequestBuilder;
+import org.asynchttpclient.DefaultAsyncHttpClient;
+import org.asynchttpclient.ListenableFuture;
+import org.asynchttpclient.Response;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutionException;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.testng.Assert.fail;
+
+/**
+ * Token client exchange token mock test.
+ */
+public class TokenClientTest {
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsSuccessByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        if (!StringUtils.isBlank(request.getScope())) {
+            bodyMap.put("scope", request.getScope());
+        }
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.execute()).thenReturn(listenableFuture);
+        when(listenableFuture.get()).thenReturn(response);
+        when(response.getStatusCode()).thenReturn(200);
+        TokenResult tokenResult = new TokenResult();
+        tokenResult.setAccessToken("test-access-token");
+        tokenResult.setIdToken("test-id");
+        when(response.getResponseBodyAsBytes()).thenReturn(new Gson().toJson(tokenResult).getBytes());
+        TokenResult tr = tokenClient.exchangeClientCredentials(request);
+        Assert.assertNotNull(tr);
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void exchangeClientCredentialsFailedByScopeTest() throws
+            IOException, TokenExchangeException, ExecutionException, InterruptedException {
+        DefaultAsyncHttpClient defaultAsyncHttpClient = mock(DefaultAsyncHttpClient.class);
+        URL url = new URL("http://localhost");
+        TokenClient tokenClient = new TokenClient(url, defaultAsyncHttpClient);
+        Map<String, String> bodyMap = new TreeMap<>();
+        ClientCredentialsExchangeRequest request = ClientCredentialsExchangeRequest.builder()
+                .audience("test-audience")
+                .clientId("test-client-id")
+                .clientSecret("test-client-secret")
+                .scope("test-scope")
+                .build();
+        bodyMap.put("grant_type", "client_credentials");
+        bodyMap.put("client_id", request.getClientId());
+        bodyMap.put("client_secret", request.getClientSecret());
+        bodyMap.put("audience", request.getAudience());
+        String body = tokenClient.buildClientCredentialsBody(bodyMap);
+        BoundRequestBuilder boundRequestBuilder = mock(BoundRequestBuilder.class);
+        Response response = mock(Response.class);
+        ListenableFuture<Response> listenableFuture = mock(ListenableFuture.class);
+        when(defaultAsyncHttpClient.preparePost(url.toString())).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Accept", "application/json")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setHeader("Content-Type", "application/x-www-form-urlencoded")).thenReturn(boundRequestBuilder);
+        when(boundRequestBuilder.setBody(body)).thenReturn(boundRequestBuilder);

Review comment:
       I see. Thanks for the explanation




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r704119227



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -44,7 +45,7 @@
     protected static final int DEFAULT_READ_TIMEOUT_IN_SECONDS = 30;
 
     private final URL tokenUrl;
-    private final AsyncHttpClient httpClient;

Review comment:
       Fixed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] tuteng commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
tuteng commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r703511128



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java
##########
@@ -86,6 +86,7 @@ public void testConfigure() throws Exception {
         params.put("privateKey", "data:base64,e30=");
         params.put("issuerUrl", "http://localhost");
         params.put("audience", "http://localhost");
+        params.put("scope", "test-scope");

Review comment:
       @addisonj  I have added some tests, please take a look at this, thanks




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] michaeljmarshall commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r705661252



##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/auth/oauth2/protocol/TokenClient.java
##########
@@ -76,6 +77,9 @@ public TokenResult exchangeClientCredentials(ClientCredentialsExchangeRequest re
         bodyMap.put("client_id", req.getClientId());
         bodyMap.put("client_secret", req.getClientSecret());
         bodyMap.put("audience", req.getAudience());

Review comment:
       I submitted the PR here: https://github.com/apache/pulsar/pull/11988




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar] eolivelli commented on a change in pull request #11931: [pulsar-client]Add a optional params scope for pulsar oauth2 client

Posted by GitBox <gi...@apache.org>.
eolivelli commented on a change in pull request #11931:
URL: https://github.com/apache/pulsar/pull/11931#discussion_r703209752



##########
File path: pulsar-client/src/test/java/org/apache/pulsar/client/impl/auth/oauth2/AuthenticationOAuth2Test.java
##########
@@ -86,6 +86,7 @@ public void testConfigure() throws Exception {
         params.put("privateKey", "data:base64,e30=");
         params.put("issuerUrl", "http://localhost");
         params.put("audience", "http://localhost");
+        params.put("scope", "test-scope");

Review comment:
       can you please add more testing about this value ?
   I see this only added here and there is not follow up verification




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org