You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by pe...@apache.org on 2022/06/15 07:00:39 UTC

[pulsar] branch master updated: Avoid AuthenticationDataSource mutation for subscription name (#16065)

This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new e6b12c64b04 Avoid AuthenticationDataSource mutation for subscription name (#16065)
e6b12c64b04 is described below

commit e6b12c64b043903eb5ff2dc5186fe8030f157cfc
Author: Matteo Merli <mm...@apache.org>
AuthorDate: Wed Jun 15 00:00:28 2022 -0700

    Avoid AuthenticationDataSource mutation for subscription name (#16065)
    
    ### Motivation
    
    The `authenticationData` field in `ServerCnx` is being mutated to add the `subscription` field that will be passed on to the authorization plugin. The problem is that `authenticationData` is scoped to the whole connection and it should be getting mutated for each consumer that is created on the connection.
    
    The current code leads to a race condition where the subscription name used in the authz plugin is already modified while we're looking at it. Instead, we should create a new object and enforce the final modifier.
---
 .../authentication/AuthenticationDataCommand.java  | 33 +---------
 .../AuthenticationDataSubscription.java            | 74 ++++++++++++++++++++++
 .../apache/pulsar/broker/service/ServerCnx.java    | 32 +++++-----
 3 files changed, 90 insertions(+), 49 deletions(-)

diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataCommand.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataCommand.java
index 02fc76f60e1..0bd03d5daf6 100644
--- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataCommand.java
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataCommand.java
@@ -29,22 +29,9 @@ public class AuthenticationDataCommand implements AuthenticationDataSource {
     protected final String authData;
     protected final SocketAddress remoteAddress;
     protected final SSLSession sslSession;
-    protected String subscription;
 
     public AuthenticationDataCommand(String authData) {
-        this(authData, null, null, null);
-    }
-
-    public AuthenticationDataCommand(String authData, String subscription) {
-        this(authData, null, null, subscription);
-    }
-
-    public AuthenticationDataCommand(String authData, SocketAddress remoteAddress, SSLSession sslSession,
-                                     String subscription) {
-        this.authData = authData;
-        this.remoteAddress = remoteAddress;
-        this.sslSession = sslSession;
-        this.subscription = subscription;
+        this(authData, null, null);
     }
 
     public AuthenticationDataCommand(String authData, SocketAddress remoteAddress, SSLSession sslSession) {
@@ -99,22 +86,4 @@ public class AuthenticationDataCommand implements AuthenticationDataSource {
             return null;
         }
     }
-
-    /*
-     * Subscription
-     */
-    @Override
-    public boolean hasSubscription() {
-        return this.subscription != null;
-    }
-
-    @Override
-    public void setSubscription(String subscription) {
-        this.subscription = subscription;
-    }
-
-    @Override
-    public String getSubscription() {
-        return subscription;
-    }
 }
diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java
new file mode 100644
index 00000000000..f6723609908
--- /dev/null
+++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authentication/AuthenticationDataSubscription.java
@@ -0,0 +1,74 @@
+/**
+ * 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.broker.authentication;
+
+import java.net.SocketAddress;
+import java.security.cert.Certificate;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class AuthenticationDataSubscription implements AuthenticationDataSource {
+    private final AuthenticationDataSource authData;
+    private final String subscription;
+
+    public AuthenticationDataSubscription(AuthenticationDataSource authData, String subscription) {
+        this.authData = authData;
+        this.subscription = subscription;
+    }
+
+    @Override
+    public boolean hasDataFromCommand() {
+        return authData.hasDataFromCommand();
+    }
+
+    @Override
+    public String getCommandData() {
+        return authData.getCommandData();
+    }
+
+    @Override
+    public boolean hasDataFromPeer() {
+        return authData.hasDataFromPeer();
+    }
+
+    @Override
+    public SocketAddress getPeerAddress() {
+        return authData.getPeerAddress();
+    }
+
+    @Override
+    public boolean hasDataFromTls() {
+        return authData.hasDataFromTls();
+    }
+
+    @Override
+    public Certificate[] getTlsCertificates() {
+        return authData.getTlsCertificates();
+    }
+
+    @Override
+    public boolean hasSubscription() {
+        return this.subscription != null;
+    }
+
+    @Override
+    public String getSubscription() {
+        return subscription;
+    }
+}
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
index 74b37213fbd..5edca60de94 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java
@@ -68,8 +68,8 @@ import org.apache.commons.lang3.mutable.MutableLong;
 import org.apache.pulsar.broker.PulsarService;
 import org.apache.pulsar.broker.ServiceConfiguration;
 import org.apache.pulsar.broker.TransactionMetadataStoreService;
-import org.apache.pulsar.broker.authentication.AuthenticationDataCommand;
 import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
+import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription;
 import org.apache.pulsar.broker.authentication.AuthenticationProvider;
 import org.apache.pulsar.broker.authentication.AuthenticationState;
 import org.apache.pulsar.broker.intercept.BrokerInterceptor;
@@ -381,19 +381,20 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
     // // Incoming commands handling
     // ////
 
-    private CompletableFuture<Boolean> isTopicOperationAllowed(TopicName topicName, TopicOperation operation) {
+    private CompletableFuture<Boolean> isTopicOperationAllowed(TopicName topicName, TopicOperation operation,
+                    AuthenticationDataSource authData) {
         if (!service.isAuthorizationEnabled()) {
             return CompletableFuture.completedFuture(true);
         }
         CompletableFuture<Boolean> isProxyAuthorizedFuture;
         if (originalPrincipal != null) {
             isProxyAuthorizedFuture = service.getAuthorizationService().allowTopicOperationAsync(
-                topicName, operation, originalPrincipal, getAuthenticationData());
+                topicName, operation, originalPrincipal, authData);
         } else {
             isProxyAuthorizedFuture = CompletableFuture.completedFuture(true);
         }
         CompletableFuture<Boolean> isAuthorizedFuture = service.getAuthorizationService().allowTopicOperationAsync(
-            topicName, operation, authRole, authenticationData);
+            topicName, operation, authRole, authData);
         return isProxyAuthorizedFuture.thenCombine(isAuthorizedFuture, (isProxyAuthorized, isAuthorized) -> {
             if (!isProxyAuthorized) {
                 log.warn("OriginalRole {} is not authorized to perform operation {} on topic {}",
@@ -410,15 +411,9 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
     private CompletableFuture<Boolean> isTopicOperationAllowed(TopicName topicName, String subscriptionName,
                                                                TopicOperation operation) {
         if (service.isAuthorizationEnabled()) {
-            if (authenticationData == null) {
-                authenticationData = new AuthenticationDataCommand("", subscriptionName);
-            } else {
-                authenticationData.setSubscription(subscriptionName);
-            }
-            if (originalAuthData != null) {
-                originalAuthData.setSubscription(subscriptionName);
-            }
-            return isTopicOperationAllowed(topicName, operation);
+            AuthenticationDataSource authData =
+                    new AuthenticationDataSubscription(getAuthenticationData(), subscriptionName);
+            return isTopicOperationAllowed(topicName, operation, authData);
         } else {
             return CompletableFuture.completedFuture(true);
         }
@@ -453,7 +448,8 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
                 lookupSemaphore.release();
                 return;
             }
-            isTopicOperationAllowed(topicName, TopicOperation.LOOKUP).thenApply(isAuthorized -> {
+            isTopicOperationAllowed(topicName, TopicOperation.LOOKUP, getAuthenticationData()).thenApply(
+                    isAuthorized -> {
                 if (isAuthorized) {
                     lookupTopicAsync(getBrokerService().pulsar(), topicName, authoritative,
                             getPrincipal(), getAuthenticationData(),
@@ -516,7 +512,8 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
                 lookupSemaphore.release();
                 return;
             }
-            isTopicOperationAllowed(topicName, TopicOperation.LOOKUP).thenApply(isAuthorized -> {
+            isTopicOperationAllowed(topicName, TopicOperation.LOOKUP, getAuthenticationData()).thenApply(
+                    isAuthorized -> {
                 if (isAuthorized) {
                     unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName)
                         .handle((metadata, ex) -> {
@@ -1203,12 +1200,13 @@ public class ServerCnx extends PulsarHandler implements TransportCnx {
         }
 
         CompletableFuture<Boolean> isAuthorizedFuture = isTopicOperationAllowed(
-                topicName, TopicOperation.PRODUCE
+                topicName, TopicOperation.PRODUCE, getAuthenticationData()
         );
 
         if (!Strings.isNullOrEmpty(initialSubscriptionName)) {
             isAuthorizedFuture =
-                    isAuthorizedFuture.thenCombine(isTopicOperationAllowed(topicName, TopicOperation.SUBSCRIBE),
+                    isAuthorizedFuture.thenCombine(
+                            isTopicOperationAllowed(topicName, TopicOperation.SUBSCRIBE, getAuthenticationData()),
                             (canProduce, canSubscribe) -> canProduce && canSubscribe);
         }