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 2021/11/15 00:49:55 UTC

[pulsar] branch master updated: Modify authz tests of GET_TOPICS and PACKAGES op (#12771)

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 22f2270  Modify authz tests of GET_TOPICS and PACKAGES op (#12771)
22f2270 is described below

commit 22f2270a5d833d7b6173a198d81d0964cc4fdcb6
Author: Ruguo Yu <ji...@163.com>
AuthorDate: Mon Nov 15 08:49:03 2021 +0800

    Modify authz tests of GET_TOPICS and PACKAGES op (#12771)
    
    ### Motivation
    Integration authz test of [GetTopicsOfNamespaceWithAuthTest.java](https://github.com/apache/pulsar/blob/master/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/GetTopicsOfNamespaceWithAuthTest.java) and [PackagesOpsWithAuthTest.java](https://github.com/apache/pulsar/blob/master/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/PackagesOpsWithAuthTest.java) is too much heavyweight for check GET_TOPICS and PACKAGES namespace op.
    
    In fact, we can add a small amount of code in [AuthorizationProducerConsumerTest.java](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java) to replace with above integration test.
---
 .../api/AuthorizationProducerConsumerTest.java     |  23 ++-
 .../admin/GetTopicsOfNamespaceWithAuthTest.java    | 208 --------------------
 .../auth/admin/PackagesOpsWithAuthTest.java        | 212 ---------------------
 .../integration/src/test/resources/pulsar-auth.xml |   1 -
 4 files changed, 22 insertions(+), 422 deletions(-)

diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
index 0654c14..63e32cb 100644
--- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
+++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java
@@ -60,6 +60,7 @@ import org.apache.pulsar.common.policies.data.TenantInfoImpl;
 import org.apache.pulsar.common.policies.data.TenantOperation;
 import org.apache.pulsar.common.policies.data.TopicOperation;
 import org.apache.pulsar.common.util.RestException;
+import org.apache.pulsar.packages.management.core.MockedPackagesStorageProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -172,6 +173,8 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
     public void testSubscriberPermission() throws Exception {
         log.info("-- Starting {} test --", methodName);
 
+        conf.setEnablePackagesManagement(true);
+        conf.setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName());
         conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName());
         setup();
 
@@ -261,15 +264,33 @@ public class AuthorizationProducerConsumerTest extends ProducerConsumerBase {
             assertTrue(e.getMessage().startsWith(
                     "Unauthorized to validateNamespaceOperation for operation [UNSUBSCRIBE]"));
         }
+        try {
+            sub1Admin.namespaces().getTopics(namespace);
+            fail("should have failed with authorization exception");
+        } catch (Exception e) {
+            assertTrue(e.getMessage().startsWith(
+                    "Unauthorized to validateNamespaceOperation for operation [GET_TOPICS]"));
+        }
+        try {
+            sub1Admin.packages().listPackages("function", namespace);
+            fail("should have failed with authorization exception");
+        } catch (Exception e) {
+            assertTrue(e.getMessage().startsWith(
+                    "Role sub1-role has not the 'package' permission to do the packages operations"));
+        }
 
         // grant namespace-level authorization to the subscriptionRole
         tenantAdmin.namespaces().grantPermissionOnNamespace(namespace, subscriptionRole,
-                Collections.singleton(AuthAction.consume));
+                Sets.newHashSet(AuthAction.consume, AuthAction.packages));
 
         // now, subscriptionRole have consume authorization on namespace, so it will successfully unsubscribe namespace
         superAdmin.namespaces().unsubscribeNamespaceBundle(namespace, "0x00000000_0xffffffff", subscriptionName2);
         subscriptions = sub1Admin.topics().getSubscriptions(topicName);
         assertEquals(subscriptions.size(), 1);
+        List<String> topics = sub1Admin.namespaces().getTopics(namespace);
+        assertEquals(topics.size(), 1);
+        List<String> packages = sub1Admin.packages().listPackages("function", namespace);
+        assertEquals(packages.size(), 0);
 
         // subscriptionRole has namespace-level authorization
         sub1Admin.topics().resetCursor(topicName, subscriptionName, 10);
diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/GetTopicsOfNamespaceWithAuthTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/GetTopicsOfNamespaceWithAuthTest.java
deleted file mode 100644
index 68de70d..0000000
--- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/GetTopicsOfNamespaceWithAuthTest.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/**
- * 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.tests.integration.auth.admin;
-
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.fail;
-import com.google.common.io.Files;
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import lombok.Cleanup;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
-import org.apache.pulsar.client.admin.PulsarAdmin;
-import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.client.api.AuthenticationFactory;
-import org.apache.pulsar.client.impl.auth.AuthenticationToken;
-import org.apache.pulsar.common.policies.data.AuthAction;
-import org.apache.pulsar.tests.TestRetrySupport;
-import org.apache.pulsar.tests.integration.containers.PulsarContainer;
-import org.apache.pulsar.tests.integration.containers.ZKContainer;
-import org.apache.pulsar.tests.integration.topologies.PulsarCluster;
-import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec;
-import org.apache.pulsar.tests.integration.utils.DockerUtils;
-import org.elasticsearch.common.collect.Set;
-import org.testcontainers.containers.Network;
-import org.testcontainers.shaded.org.apache.commons.lang.RandomStringUtils;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-/**
- * GetTopicsOfNamespaceWithAuthTest will test GetTopics operation with and without the proper permission.
- */
-@Slf4j
-public class GetTopicsOfNamespaceWithAuthTest extends TestRetrySupport {
-
-    private static final String CLUSTER_PREFIX = "get-topics-auth";
-    private static final String PRIVATE_KEY_PATH_INSIDE_CONTAINER = "/tmp/private.key";
-    private static final String PUBLIC_KEY_PATH_INSIDE_CONTAINER = "/tmp/public.key";
-
-    private static final String SUPER_USER_ROLE = "super-user";
-    private String superUserAuthToken;
-    private static final String PROXY_ROLE = "proxy";
-    private String proxyAuthToken;
-    private static final String REGULAR_USER_ROLE = "client";
-    private String clientAuthToken;
-    private File publicKeyFile;
-
-    private PulsarCluster pulsarCluster;
-    private PulsarContainer cmdContainer;
-
-    @Override
-    @BeforeClass(alwaysRun = true)
-    protected void setup() throws Exception {
-        incrementSetupNumber();
-        // Before starting the cluster, generate the secret key and the token
-        // Use Zk container to have 1 container available before starting the cluster
-        final String clusterName = String.format("%s-%s", CLUSTER_PREFIX, RandomStringUtils.randomAlphabetic(6));
-        final String cliContainerName = String.format("%s-%s", "cli", RandomStringUtils.randomAlphabetic(6));
-        cmdContainer = new ZKContainer<>(cliContainerName);
-        cmdContainer
-                .withNetwork(Network.newNetwork())
-                .withNetworkAliases(ZKContainer.NAME)
-                .withEnv("zkServers", ZKContainer.NAME);
-        cmdContainer.start();
-
-        createKeysAndTokens(cmdContainer);
-
-        PulsarClusterSpec spec = PulsarClusterSpec.builder()
-                .numBookies(2)
-                .numBrokers(2)
-                .numProxies(1)
-                .clusterName(clusterName)
-                .brokerEnvs(getBrokerSettingsEnvs())
-                .proxyEnvs(getProxySettingsEnvs())
-                .brokerMountFiles(Collections.singletonMap(publicKeyFile.toString(), PUBLIC_KEY_PATH_INSIDE_CONTAINER))
-                .proxyMountFiles(Collections.singletonMap(publicKeyFile.toString(), PUBLIC_KEY_PATH_INSIDE_CONTAINER))
-                .build();
-
-        pulsarCluster = PulsarCluster.forSpec(spec);
-        pulsarCluster.start();
-    }
-
-    @Override
-    @AfterClass(alwaysRun = true)
-    public void cleanup() {
-        markCurrentSetupNumberCleaned();
-        if (cmdContainer != null) {
-            cmdContainer.stop();
-        }
-        if (pulsarCluster != null) {
-            pulsarCluster.stop();
-        }
-    }
-
-    private Map<String, String> getBrokerSettingsEnvs() {
-        Map<String, String> envs = new HashMap<>();
-        envs.put("authenticationEnabled", "true");
-        envs.put("authenticationProviders", AuthenticationProviderToken.class.getName());
-        envs.put("authorizationEnabled", "true");
-        envs.put("superUserRoles", String.format("%s,%s", SUPER_USER_ROLE, PROXY_ROLE));
-        envs.put("brokerClientAuthenticationPlugin", AuthenticationToken.class.getName());
-        envs.put("brokerClientAuthenticationParameters", String.format("token:%s", superUserAuthToken));
-        envs.put("authenticationRefreshCheckSeconds", "1");
-        envs.put("authenticateOriginalAuthData", "true");
-        envs.put("tokenPublicKey", "file://" + PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-        return envs;
-    }
-
-    private Map<String, String> getProxySettingsEnvs() {
-        Map<String, String> envs = new HashMap<>();
-        envs.put("authenticationEnabled", "true");
-        envs.put("authenticationProviders", AuthenticationProviderToken.class.getName());
-        envs.put("authorizationEnabled", "true");
-        envs.put("brokerClientAuthenticationPlugin", AuthenticationToken.class.getName());
-        envs.put("brokerClientAuthenticationParameters", String.format("token:%s", proxyAuthToken));
-        envs.put("authenticationRefreshCheckSeconds", "1");
-        envs.put("forwardAuthorizationCredentials", "true");
-        envs.put("tokenPublicKey", "file://" + PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-        return envs;
-    }
-
-    protected void createKeysAndTokens(PulsarContainer container) throws Exception {
-        container
-                .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create-key-pair",
-                        "--output-private-key", PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                        "--output-public-key", PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-
-        byte[] publicKeyBytes = DockerUtils
-                .runCommandWithRawOutput(container.getDockerClient(), container.getContainerId(),
-                        "/bin/cat", PUBLIC_KEY_PATH_INSIDE_CONTAINER)
-                .getStdout();
-
-        publicKeyFile = File.createTempFile("public-", ".key", new File("/tmp"));
-        Files.write(publicKeyBytes, publicKeyFile);
-
-        clientAuthToken = container
-                .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                        "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                        "--subject", REGULAR_USER_ROLE)
-                .getStdout().trim();
-        log.info("Created client token: {}", clientAuthToken);
-
-        superUserAuthToken = container
-                .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                        "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                        "--subject", SUPER_USER_ROLE)
-                .getStdout().trim();
-        log.info("Created super-user token: {}", superUserAuthToken);
-
-        proxyAuthToken = container
-                .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                        "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                        "--subject", PROXY_ROLE)
-                .getStdout().trim();
-        log.info("Created proxy token: {}", proxyAuthToken);
-    }
-
-    @Test
-    public void testGetTopicsOfNamespaceOpsWithConsumePermission() throws Exception {
-        @Cleanup
-        PulsarAdmin superUserAdmin = PulsarAdmin.builder()
-                .serviceHttpUrl(pulsarCluster.getHttpServiceUrl())
-                .authentication(AuthenticationFactory.token(superUserAuthToken))
-                .build();
-
-        @Cleanup
-        PulsarAdmin clientAdmin = PulsarAdmin.builder()
-                .serviceHttpUrl(pulsarCluster.getHttpServiceUrl())
-                .authentication(AuthenticationFactory.token(clientAuthToken))
-                .build();
-
-        // do some operation without grant any permissions
-        try {
-            clientAdmin.namespaces().getTopics("public/default");
-            fail("list topics operation should fail because the client hasn't permission to do");
-        } catch (PulsarAdminException e) {
-            assertEquals(e.getStatusCode(), 401);
-        }
-
-        // grant consume permission to the role
-        superUserAdmin.namespaces().grantPermissionOnNamespace("public/default",
-                REGULAR_USER_ROLE, Set.of(AuthAction.consume));
-
-        // then do some get topics operations again, it should success
-        List<String> topics = clientAdmin.namespaces().getTopics("public/default");
-        assertEquals(topics.size(), 0);
-    }
-}
diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/PackagesOpsWithAuthTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/PackagesOpsWithAuthTest.java
deleted file mode 100644
index 9c5ab69..0000000
--- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/auth/admin/PackagesOpsWithAuthTest.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/**
- * 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.tests.integration.auth.admin;
-
-import com.google.common.io.Files;
-import lombok.Cleanup;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
-import org.apache.pulsar.client.admin.PulsarAdmin;
-import org.apache.pulsar.client.admin.PulsarAdminException;
-import org.apache.pulsar.client.api.AuthenticationFactory;
-import org.apache.pulsar.client.impl.auth.AuthenticationToken;
-import org.apache.pulsar.common.policies.data.AuthAction;
-import org.apache.pulsar.tests.TestRetrySupport;
-import org.apache.pulsar.tests.integration.containers.PulsarContainer;
-import org.apache.pulsar.tests.integration.containers.ZKContainer;
-import org.apache.pulsar.tests.integration.topologies.PulsarCluster;
-import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec;
-import org.apache.pulsar.tests.integration.utils.DockerUtils;
-import org.elasticsearch.common.collect.Set;
-import org.testcontainers.containers.Network;
-import org.testcontainers.shaded.org.apache.commons.lang.RandomStringUtils;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
-
-import java.io.File;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.fail;
-
-
-/**
- * PackagesOpsWithAuthTest will test all package operations with and without the proper permission.
- */
-@Slf4j
-public class PackagesOpsWithAuthTest extends TestRetrySupport {
-
-    private static final String CLUSTER_PREFIX = "package-auth";
-    private static final String PRIVATE_KEY_PATH_INSIDE_CONTAINER = "/tmp/private.key";
-    private static final String PUBLIC_KEY_PATH_INSIDE_CONTAINER = "/tmp/public.key";
-
-    private static final String SUPER_USER_ROLE = "super-user";
-    private String superUserAuthToken;
-    private static final String PROXY_ROLE = "proxy";
-    private String proxyAuthToken;
-    private static final String REGULAR_USER_ROLE = "client";
-    private String clientAuthToken;
-    private File publicKeyFile;
-
-    private PulsarCluster pulsarCluster;
-    private PulsarContainer cmdContainer;
-
-    @Override
-    @BeforeClass(alwaysRun = true)
-    public void setup() throws Exception {
-        incrementSetupNumber();
-        // Before starting the cluster, generate the secret key and the token
-        // Use Zk container to have 1 container available before starting the cluster
-        final String clusterName = String.format("%s-%s", CLUSTER_PREFIX, RandomStringUtils.randomAlphabetic(6));
-        final String cliContainerName = String.format("%s-%s", "cli", RandomStringUtils.randomAlphabetic(6));
-        cmdContainer = new ZKContainer<>(cliContainerName);
-        cmdContainer
-            .withNetwork(Network.newNetwork())
-            .withNetworkAliases(ZKContainer.NAME)
-            .withEnv("zkServers", ZKContainer.NAME);
-        cmdContainer.start();
-
-        createKeysAndTokens(cmdContainer);
-
-        PulsarClusterSpec spec = PulsarClusterSpec.builder()
-            .numBookies(2)
-            .numBrokers(2)
-            .numProxies(1)
-            .clusterName(clusterName)
-            .brokerEnvs(getBrokerSettingsEnvs())
-            .proxyEnvs(getProxySettingsEnvs())
-            .brokerMountFiles(Collections.singletonMap(publicKeyFile.toString(), PUBLIC_KEY_PATH_INSIDE_CONTAINER))
-            .proxyMountFiles(Collections.singletonMap(publicKeyFile.toString(), PUBLIC_KEY_PATH_INSIDE_CONTAINER))
-            .build();
-
-        pulsarCluster = PulsarCluster.forSpec(spec);
-        pulsarCluster.start();
-    }
-
-    @Override
-    @AfterClass(alwaysRun = true)
-    public void cleanup() {
-        markCurrentSetupNumberCleaned();
-        if (cmdContainer != null) {
-            cmdContainer.stop();
-        }
-        if (pulsarCluster != null) {
-            pulsarCluster.stop();
-        }
-    }
-
-    private Map<String, String> getBrokerSettingsEnvs() {
-        Map<String, String> envs = new HashMap<>();
-        envs.put("enablePackagesManagement", "true");
-        envs.put("authenticationEnabled", "true");
-        envs.put("authenticationProviders", AuthenticationProviderToken.class.getName());
-        envs.put("authorizationEnabled", "true");
-        envs.put("superUserRoles", String.format("%s,%s", SUPER_USER_ROLE, PROXY_ROLE));
-        envs.put("brokerClientAuthenticationPlugin", AuthenticationToken.class.getName());
-        envs.put("brokerClientAuthenticationParameters", String.format("token:%s", superUserAuthToken));
-        envs.put("authenticationRefreshCheckSeconds", "1");
-        envs.put("authenticateOriginalAuthData", "true");
-        envs.put("tokenPublicKey", "file://" + PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-        return envs;
-    }
-
-    private Map<String, String> getProxySettingsEnvs() {
-        Map<String, String> envs = new HashMap<>();
-        envs.put("authenticationEnabled", "true");
-        envs.put("authenticationProviders", AuthenticationProviderToken.class.getName());
-        envs.put("authorizationEnabled", "true");
-        envs.put("brokerClientAuthenticationPlugin", AuthenticationToken.class.getName());
-        envs.put("brokerClientAuthenticationParameters", String.format("token:%s", proxyAuthToken));
-        envs.put("authenticationRefreshCheckSeconds", "1");
-        envs.put("forwardAuthorizationCredentials", "true");
-        envs.put("tokenPublicKey", "file://" + PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-        return envs;
-    }
-
-    protected void createKeysAndTokens(PulsarContainer container) throws Exception {
-        container
-            .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create-key-pair",
-                "--output-private-key", PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                "--output-public-key", PUBLIC_KEY_PATH_INSIDE_CONTAINER);
-
-        byte[] publicKeyBytes = DockerUtils
-            .runCommandWithRawOutput(container.getDockerClient(), container.getContainerId(),
-                "/bin/cat", PUBLIC_KEY_PATH_INSIDE_CONTAINER)
-            .getStdout();
-
-        publicKeyFile = File.createTempFile("public-", ".key", new File("/tmp"));
-        Files.write(publicKeyBytes, publicKeyFile);
-
-        clientAuthToken = container
-            .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                "--subject", REGULAR_USER_ROLE)
-            .getStdout().trim();
-        log.info("Created client token: {}", clientAuthToken);
-
-        superUserAuthToken = container
-            .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                "--subject", SUPER_USER_ROLE)
-            .getStdout().trim();
-        log.info("Created super-user token: {}", superUserAuthToken);
-
-        proxyAuthToken = container
-            .execCmd(PulsarCluster.PULSAR_COMMAND_SCRIPT, "tokens", "create",
-                "--private-key", "file://" + PRIVATE_KEY_PATH_INSIDE_CONTAINER,
-                "--subject", PROXY_ROLE)
-            .getStdout().trim();
-        log.info("Created proxy token: {}", proxyAuthToken);
-    }
-
-    @Test
-    public void testPackagesOps() throws Exception {
-        @Cleanup
-        PulsarAdmin superUserAdmin = PulsarAdmin.builder()
-            .serviceHttpUrl(pulsarCluster.getHttpServiceUrl())
-            .authentication(AuthenticationFactory.token(superUserAuthToken))
-            .build();
-
-        @Cleanup
-        PulsarAdmin clientAdmin = PulsarAdmin.builder()
-            .serviceHttpUrl(pulsarCluster.getHttpServiceUrl())
-            .authentication(AuthenticationFactory.token(clientAuthToken))
-            .build();
-
-        // do some operation without grant any permissions
-        try {
-            List<String> packagesName = clientAdmin.packages().listPackages("function", "public/default");
-            fail("list package operation should fail because the client hasn't permission to do");
-        } catch (PulsarAdminException e) {
-            assertEquals(e.getStatusCode(), 401);
-        }
-
-        // grant package permission to the role
-        superUserAdmin.namespaces().grantPermissionOnNamespace("public/default",
-            REGULAR_USER_ROLE, Set.of(AuthAction.packages));
-
-        // then do some package operations again, it should success
-        List<String> packagesName = clientAdmin.packages().listPackages("function", "public/default");
-        assertEquals(packagesName.size(), 0);
-    }
-}
diff --git a/tests/integration/src/test/resources/pulsar-auth.xml b/tests/integration/src/test/resources/pulsar-auth.xml
index d361c57..81d2c13 100644
--- a/tests/integration/src/test/resources/pulsar-auth.xml
+++ b/tests/integration/src/test/resources/pulsar-auth.xml
@@ -22,7 +22,6 @@
 <suite name="Pulsar Auth Integration Tests" verbose="2" annotations="JDK">
     <test name="pulsar-auth-test-suite" preserve-order="true" >
         <classes>
-            <class name="org.apache.pulsar.tests.integration.auth.admin.PackagesOpsWithAuthTest"/>
         </classes>
     </test>
 </suite>