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 2022/09/29 02:00:10 UTC

[GitHub] [pulsar] mattisonchao commented on a diff in pull request #17831: [fix][proxy] Fix refresh client auth

mattisonchao commented on code in PR #17831:
URL: https://github.com/apache/pulsar/pull/17831#discussion_r982977833


##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -543,6 +577,35 @@ protected void handleAuthResponse(CommandAuthResponse authResponse) {
         }
     }
 
+    private Void requestRefreshClientAuthData() {

Review Comment:
   why return `Void`?



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -513,7 +548,7 @@ remoteAddress, protocolVersionToAdvertise, getRemoteEndpointProtocolVersion(),
 
             authState = authenticationProvider.newAuthState(clientData, remoteAddress, sslSession);
             authenticationData = authState.getAuthDataSource();
-            doAuthentication(clientData);
+            doAuthentication(clientData, Optional.empty());

Review Comment:
   Passing the optional as a parameter is not a good approach I think.



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -423,16 +430,44 @@ public void brokerConnected(DirectProxyHandler directProxyHandler, CommandConnec
     }
 
     // According to auth result, send newConnected or newAuthChallenge command.
-    private void doAuthentication(AuthData clientData) throws Exception {
+    private void doAuthentication(AuthData clientData, Optional<CommandAuthResponse> authResponseOptional)
+            throws Exception {
         AuthData brokerData = authState.authenticate(clientData);
         // authentication has completed, will send newConnected command.
         if (authState.isComplete()) {
             clientAuthRole = authState.getAuthRole();
             if (LOG.isDebugEnabled()) {
                 LOG.debug("[{}] Client successfully authenticated with {} role {}",
-                    remoteAddress, authMethod, clientAuthRole);
+                        remoteAddress, authMethod, clientAuthRole);
+            }
+
+            // First connection
+            if (this.connectionPool == null || state == State.Connecting) {
+                // authentication has completed, will send newConnected command.
+                completeConnect(clientData);
+            } else if (state == State.ProxyLookupRequests) {

Review Comment:
   Could moving this logic out of `doAuthentication` make this method look better? According to its name, only the operation of authentication is done.
   I think we can move this method to `handleAuthResponse`, please let me know what do you think.



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java:
##########
@@ -54,10 +67,33 @@ protected ByteBuf newConnectCommand() throws Exception {
 
         authenticationDataProvider = authentication.getAuthData(remoteHostName);
         AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
-        return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
-            PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
-            clientAuthMethod);
+        return Commands.newConnect(authentication.getAuthMethodName(), authData, protocolVersion,
+                PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
+                clientAuthMethod);
     }
 
-    private static final Logger log = LoggerFactory.getLogger(ProxyClientCnx.class);
+    @Override
+    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
+        checkArgument(authChallenge.hasChallenge());
+        checkArgument(authChallenge.getChallenge().hasAuthData());
+
+        boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
+        if (!forwardClientAuthData || !isRefresh || refreshClientAuthDataNotifier == null) {

Review Comment:
   Question:
   Which condition the `refreshClientAuthDataNotifier` equal null?
   Could we check it when we create the `ProxyClientCnx`? 



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java:
##########
@@ -54,10 +67,33 @@ protected ByteBuf newConnectCommand() throws Exception {
 
         authenticationDataProvider = authentication.getAuthData(remoteHostName);
         AuthData authData = authenticationDataProvider.authenticate(AuthData.INIT_AUTH_DATA);
-        return Commands.newConnect(authentication.getAuthMethodName(), authData, this.protocolVersion,
-            PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
-            clientAuthMethod);
+        return Commands.newConnect(authentication.getAuthMethodName(), authData, protocolVersion,
+                PulsarVersion.getVersion(), proxyToTargetBrokerAddress, clientAuthRole, clientAuthData,
+                clientAuthMethod);
     }
 
-    private static final Logger log = LoggerFactory.getLogger(ProxyClientCnx.class);
+    @Override
+    protected void handleAuthChallenge(CommandAuthChallenge authChallenge) {
+        checkArgument(authChallenge.hasChallenge());
+        checkArgument(authChallenge.getChallenge().hasAuthData());
+
+        boolean isRefresh = Arrays.equals(AuthData.REFRESH_AUTH_DATA_BYTES, authChallenge.getChallenge().getAuthData());
+        if (!forwardClientAuthData || !isRefresh || refreshClientAuthDataNotifier == null) {
+            super.handleAuthChallenge(authChallenge);
+            return;
+        }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Request to refresh the original client authentication data");
+        }
+        try {
+            refreshClientAuthDataNotifier.get();
+            if (state == State.SentConnectFrame) {
+                state = State.Connecting;

Review Comment:
   Does this state conflict with `handleAuthResponse`? Because I see the logic check 
   ```
     // First connection
   if (this.connectionPool == null || state == State.Connecting) {
        // authentication has completed, will send newConnected command.
         completeConnect(clientData);
   } 
   ```
   



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyClientCnx.java:
##########
@@ -18,30 +18,43 @@
  */
 package org.apache.pulsar.proxy.server;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.EventLoopGroup;
+import java.util.Arrays;
+import java.util.function.Supplier;
+import lombok.extern.slf4j.Slf4j;
 import org.apache.pulsar.PulsarVersion;
 import org.apache.pulsar.client.impl.ClientCnx;
 import org.apache.pulsar.client.impl.conf.ClientConfigurationData;
 import org.apache.pulsar.common.api.AuthData;
+import org.apache.pulsar.common.api.proto.CommandAuthChallenge;
 import org.apache.pulsar.common.protocol.Commands;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
+@Slf4j
 public class ProxyClientCnx extends ClientCnx {
+    private final boolean forwardClientAuthData;
+    private final String clientAuthMethod;
+    private final String clientAuthRole;
+    private final AuthData clientAuthData;

Review Comment:
   When we refresh the `authData`, should we update this value?



##########
pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java:
##########
@@ -543,6 +577,35 @@ protected void handleAuthResponse(CommandAuthResponse authResponse) {
         }
     }
 
+    private Void requestRefreshClientAuthData() {
+        if (!service.getConfiguration().isForwardAuthorizationCredentials()) {

Review Comment:
   I'm not sure if we have to check this again. because it looks like already checked at `ProxyClientCnx`. but it's fine.



-- 
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