Skip to content

Commit

Permalink
Check whether the property exists at the beginning of `validateAdminA…
Browse files Browse the repository at this point in the history
…ccessOnProperty` (apache#912)

* Check whether the property exists at the beginning of `validateAdminAccessOnProperty`

* Fix C++ tests
  • Loading branch information
yush1ga authored and nkurihar committed Nov 28, 2017
1 parent 1366951 commit f0a7dd9
Show file tree
Hide file tree
Showing 13 changed files with 69 additions and 19 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,16 @@ protected void validateAdminAccessOnProperty(String property) {
}

protected static void validateAdminAccessOnProperty(PulsarService pulsar, String clientAppId, String property) throws RestException, Exception{
PropertyAdmin propertyAdmin;

try {
propertyAdmin = pulsar.getConfigurationCache().propertiesCache().get(path(POLICIES, property))
.orElseThrow(() -> new RestException(Status.NOT_FOUND, "Property does not exist"));
} catch (KeeperException.NoNodeException e) {
log.warn("Failed to get property admin data for non existing property {}", property);
throw new RestException(Status.NOT_FOUND, "Property does not exist");
}

if (pulsar.getConfiguration().isAuthenticationEnabled() && pulsar.getConfiguration().isAuthorizationEnabled()) {
log.debug("check admin access on property: {} - Authenticated: {} -- role: {}", property,
(isClientAuthenticated(clientAppId)), clientAppId);
Expand All @@ -186,15 +196,6 @@ protected static void validateAdminAccessOnProperty(PulsarService pulsar, String
// Super-user has access to configure all the policies
log.debug("granting access to super-user {} on property {}", clientAppId, property);
} else {
PropertyAdmin propertyAdmin;

try {
propertyAdmin = pulsar.getConfigurationCache().propertiesCache().get(path(POLICIES, property))
.orElseThrow(() -> new RestException(Status.UNAUTHORIZED, "Property does not exist"));
} catch (KeeperException.NoNodeException e) {
log.warn("Failed to get property admin data for non existing property {}", property);
throw new RestException(Status.UNAUTHORIZED, "Property does not exist");
}

if (!propertyAdmin.getAdminRoles().contains(clientAppId)) {
throw new RestException(Status.UNAUTHORIZED,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,7 @@ public void setup() throws Exception {
doReturn(Sets.newTreeSet(Lists.newArrayList("use", "usw", "usc"))).when(persistentTopics).clusters();
doNothing().when(persistentTopics).validateAdminAccessOnProperty("my-property");
doNothing().when(persistentTopics).validateAdminAccessOnProperty("other-property");
doNothing().when(persistentTopics).validateAdminAccessOnProperty("prop-xyz");

resourceQuotas = spy(new ResourceQuotas());
resourceQuotas.setServletContext(new MockServletContext());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@
import org.apache.pulsar.client.api.SubscriptionType;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.ConsumerBase;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.annotations.AfterMethod;
Expand Down Expand Up @@ -412,6 +413,7 @@ public void testExclusiveSingleAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 10;
final int numberOfPartitions = 4;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);
// Special step to create partitioned topic

Expand Down Expand Up @@ -466,6 +468,7 @@ public void testSharedSingleAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 10;
final int numberOfPartitions = 3;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);
Random rn = new Random();
// Special step to create partitioned topic
Expand Down Expand Up @@ -563,6 +566,7 @@ public void testFailoverSingleAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 10;
final int numberOfPartitions = 3;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);
Random rn = new Random();
// Special step to create partitioned topic
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.PartitionedConsumerImpl;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.annotations.AfterMethod;
Expand Down Expand Up @@ -359,6 +360,7 @@ public void testSharedAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 15;
final int numberOfPartitions = 3;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);

// 1. producer connect
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.client.impl.ConsumerImpl;
import org.apache.pulsar.client.impl.MessageIdImpl;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testng.Assert;
Expand Down Expand Up @@ -169,6 +170,7 @@ public void testSharedSingleAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 20;
final int numberOfPartitions = 3;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);
// Special step to create partitioned topic

Expand Down Expand Up @@ -258,6 +260,7 @@ public void testFailoverSingleAckedPartitionedTopic() throws Exception {
final String messagePredicate = "my-message-" + key + "-";
final int totalMessages = 10;
final int numberOfPartitions = 3;
admin.properties().createProperty("prop", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topicName, numberOfPartitions);
// Special step to create partitioned topic

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public void testRiderectUrlWithServerStarted() throws Exception {
**/

assertEquals(hitBrokerService(HttpMethod.POST, postRequestUrl, Lists.newArrayList("use")),
"Cannot set replication on a non-global namespace");
"Property does not exist");
assertEquals(hitBrokerService(HttpMethod.PUT, putRequestUrl, new BundlesData(1)), "Property does not exist");
assertEquals(hitBrokerService(HttpMethod.GET, getRequestUrl, null), "Property does not exist");

Expand Down
6 changes: 5 additions & 1 deletion pulsar-client-cpp/run-unit-tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,11 @@ PULSAR_STANDALONE_CONF=$PWD/test-conf/standalone-ssl.conf pulsar-dist/bin/pulsar
auth_pid=$!;
sleep 10

PULSAR_CLIENT_CONF=$PWD/test-conf/client.conf pulsar-dist/bin/pulsar-admin clusters create \
# create property for test
PULSAR_CLIENT_CONF=$PWD/test-conf/client.conf pulsar-dist/bin/pulsar-admin properties create prop -r "" -c "unit"
PULSAR_CLIENT_CONF=$PWD/test-conf/client.conf pulsar-dist/bin/pulsar-admin properties create property -r "" -c "cluster"

PULSAR_CLIENT_CONF=$PWD/test-conf/client-ssl.conf pulsar-dist/bin/pulsar-admin clusters create \
--url http://localhost:9765/ --url-secure https://localhost:9766/ \
--broker-url pulsar://localhost:9885/ --broker-url-secure pulsar+ssl://localhost:9886/ \
cluster
Expand Down
27 changes: 27 additions & 0 deletions pulsar-client-cpp/test-conf/client-ssl.conf
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
#
# 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.
#

# Pulsar Client configuration
webServiceUrl=https://localhost:9766/
brokerServiceUrl=pulsar+ssl://localhost:9886/
useTls=true
tlsAllowInsecureConnection=true
tlsTrustCertsFilePath=/pulsar/pulsar-broker/src/test/resources/authentication/tls/cacert.pem
authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationTls
authParams=tlsCertFile:/pulsar/pulsar-broker/src/test/resources/authentication/tls/client-cert.pem,tlsKeyFile:/pulsar/pulsar-broker/src/test/resources/authentication/tls/client-key.pem
14 changes: 7 additions & 7 deletions pulsar-client-cpp/test-conf/client.conf
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,10 @@
#

# Pulsar Client configuration
webServiceUrl=https://localhost:9766/
brokerServiceUrl=pulsar+ssl://localhost:9886/
useTls=true
tlsAllowInsecureConnection=true
tlsTrustCertsFilePath=/pulsar/pulsar-broker/src/test/resources/authentication/tls/cacert.pem
authPlugin=org.apache.pulsar.client.impl.auth.AuthenticationTls
authParams=tlsCertFile:/pulsar/pulsar-broker/src/test/resources/authentication/tls/client-cert.pem,tlsKeyFile:/pulsar/pulsar-broker/src/test/resources/authentication/tls/client-key.pem
webServiceUrl=http://localhost:8765/
brokerServiceUrl=pulsar://localhost:8885/
#authPlugin=
#authParams=
#useTls=
#tlsAllowInsecureConnection
#tlsTrustCertsFilePath
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.ProducerConfiguration;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;
Expand Down Expand Up @@ -190,6 +191,7 @@ public void testPartitions() throws Exception {
String topic = "persistent://sample/standalone/ns/testPartitions";

// Create 8 partitions in topic
admin.properties().createProperty("sample", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic(topic, 8);

Properties props = new Properties();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.cli.PulsarClientTool;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.testng.Assert;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand All @@ -51,11 +52,12 @@ public void cleanup() throws Exception {
}

@Test(timeOut = 10000)
public void testInitialzation() throws MalformedURLException, InterruptedException, ExecutionException {
public void testInitialzation() throws MalformedURLException, InterruptedException, ExecutionException, PulsarAdminException {
Properties properties = new Properties();
properties.setProperty("serviceUrl", brokerUrl.toString());
properties.setProperty("useTls", "false");

admin.properties().createProperty("property", new PropertyAdmin());
String topicName = "persistent://property/ns/topic-scale-ns-0/topic";

int numberOfMessages = 10;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.pulsar.client.api.ProducerConfiguration;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.mockito.Mockito;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -101,6 +102,7 @@ public void testProducerConsumer() throws Exception {

@Test
public void testPartitions() throws Exception {
admin.properties().createProperty("sample", new PropertyAdmin());
PulsarClient client = PulsarClient.create("pulsar://localhost:" + proxyConfig.getServicePort());
admin.persistentTopics().createPartitionedTopic("persistent://sample/test/local/partitioned-topic", 2);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.pulsar.client.api.ProducerConfiguration;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.client.api.ProducerConfiguration.MessageRoutingMode;
import org.apache.pulsar.common.policies.data.PropertyAdmin;
import org.mockito.Mockito;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
Expand Down Expand Up @@ -98,6 +99,7 @@ public void testPartitions() throws Exception {
conf.setTlsTrustCertsFilePath(TLS_TRUST_CERT_FILE_PATH);

PulsarClient client = PulsarClient.create("pulsar://localhost:" + proxyConfig.getServicePortTls(), conf);
admin.properties().createProperty("sample", new PropertyAdmin());
admin.persistentTopics().createPartitionedTopic("persistent://sample/test/local/partitioned-topic", 2);

ProducerConfiguration producerConf = new ProducerConfiguration();
Expand Down

0 comments on commit f0a7dd9

Please sign in to comment.