Skip to content
This repository has been archived by the owner on Jan 24, 2024. It is now read-only.

Commit

Permalink
Optimize authorization by caching authorization results
Browse files Browse the repository at this point in the history
### Motivation

To follow Kafka's behavior, KoP also performs authorization for each
PRODUCE or FETCH request. If the custom authorization provider is slow
to authorize produce or consume permissions, the performance will be
impacted.

### Modifications

Introduce caches for authorization:
- PRODUCE: (topic, role) -> result
- FETCH: (topic, role, group) -> result;

Add `SlowAuthorizationTest` to verify the producer and consumer won't be
affected significantly by slow authorization.
  • Loading branch information
BewareMyPower committed Aug 23, 2023
1 parent 5193592 commit 45db3ed
Show file tree
Hide file tree
Showing 4 changed files with 294 additions and 115 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,17 @@
*/
package io.streamnative.pulsar.handlers.kop.security.auth;

import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.LoadingCache;
import io.streamnative.pulsar.handlers.kop.KafkaServiceConfiguration;
import io.streamnative.pulsar.handlers.kop.security.KafkaPrincipal;
import java.time.Duration;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.commons.lang3.tuple.Triple;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.authorization.AuthorizationService;
import org.apache.pulsar.common.naming.NamespaceName;
Expand All @@ -39,6 +44,19 @@ public class SimpleAclAuthorizer implements Authorizer {
private final AuthorizationService authorizationService;

private final boolean forceCheckGroupId;
// Cache the authorization results to avoid authorizing PRODUCE or FETCH requests each time.
// key is (topic, role)
private final LoadingCache<Pair<TopicName, String>, Boolean> produceCache = Caffeine.newBuilder()
.maximumSize(10000)
.expireAfterWrite(Duration.ofMinutes(5))
.refreshAfterWrite(Duration.ofMinutes(1))
.build(__ -> null);
// key is (topic, role, group)
private final LoadingCache<Triple<TopicName, String, String>, Boolean> fetchCache = Caffeine.newBuilder()
.maximumSize(10000)
.expireAfterWrite(Duration.ofMinutes(5))
.refreshAfterWrite(Duration.ofMinutes(1))
.build(__ -> null);

public SimpleAclAuthorizer(PulsarService pulsarService, KafkaServiceConfiguration config) {
this.pulsarService = pulsarService;
Expand Down Expand Up @@ -151,7 +169,16 @@ public CompletableFuture<Boolean> canGetTopicList(KafkaPrincipal principal, Reso
public CompletableFuture<Boolean> canProduceAsync(KafkaPrincipal principal, Resource resource) {
checkResourceType(resource, ResourceType.TOPIC);
TopicName topicName = TopicName.get(resource.getName());
return authorizationService.canProduceAsync(topicName, principal.getName(), principal.getAuthenticationData());
final Pair<TopicName, String> key = Pair.of(topicName, principal.getName());
final Boolean authorized = produceCache.get(key);
if (authorized != null) {
return CompletableFuture.completedFuture(authorized);
}
return authorizationService.canProduceAsync(topicName, principal.getName(), principal.getAuthenticationData())
.thenApply(__ -> {
produceCache.put(key, __);
return __;
});
}

@Override
Expand All @@ -161,8 +188,17 @@ public CompletableFuture<Boolean> canConsumeAsync(KafkaPrincipal principal, Reso
if (forceCheckGroupId && StringUtils.isBlank(principal.getGroupId())) {
return CompletableFuture.completedFuture(false);
}
final Triple<TopicName, String, String> key = Triple.of(topicName, principal.getName(), principal.getGroupId());
final Boolean authorized = fetchCache.get(key);
if (authorized != null) {
return CompletableFuture.completedFuture(authorized);
}
return authorizationService.canConsumeAsync(
topicName, principal.getName(), principal.getAuthenticationData(), principal.getGroupId());
topicName, principal.getName(), principal.getAuthenticationData(), principal.getGroupId())
.thenApply(__ -> {
fetchCache.put(key, __);
return __;
});
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,130 +13,25 @@
*/
package io.streamnative.pulsar.handlers.kop.security.auth;

import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;

import com.google.common.collect.Sets;
import io.jsonwebtoken.SignatureAlgorithm;
import io.streamnative.pulsar.handlers.kop.KopProtocolHandlerTestBase;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import javax.crypto.SecretKey;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.PartitionInfo;
import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.impl.auth.AuthenticationToken;
import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Test;

/**
* Unit test for Authorization with `entryFormat=pulsar`.
*/
public class KafkaAuthorizationMockTest extends KopProtocolHandlerTestBase {

protected static final String TENANT = "KafkaAuthorizationTest";
protected static final String NAMESPACE = "ns1";
private static final SecretKey secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);

protected static final String ADMIN_USER = "pass.pass";
public class KafkaAuthorizationMockTest extends KafkaAuthorizationMockTestBase {

@BeforeClass
@Override
protected void setup() throws Exception {
Properties properties = new Properties();
properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(secretKey));

String adminToken = AuthTokenUtils.createToken(secretKey, ADMIN_USER, Optional.empty());

conf.setSaslAllowedMechanisms(Sets.newHashSet("PLAIN"));
conf.setKafkaMetadataTenant("internal");
conf.setKafkaMetadataNamespace("__kafka");
conf.setKafkaTenant(TENANT);
conf.setKafkaNamespace(NAMESPACE);

conf.setClusterName(super.configClusterName);
conf.setAuthorizationEnabled(true);
conf.setAuthenticationEnabled(true);
conf.setAuthorizationAllowWildcardsMatching(true);
conf.setAuthorizationProvider(KafkaMockAuthorizationProvider.class.getName());
conf.setAuthenticationProviders(
Sets.newHashSet(AuthenticationProviderToken.class.getName()));
conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
conf.setBrokerClientAuthenticationParameters("token:" + adminToken);
conf.setProperties(properties);

super.internalSetup();
public void setup() throws Exception {
super.setup();
}

@AfterClass
@Override
protected void cleanup() throws Exception {
super.admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString())
.authentication(this.conf.getBrokerClientAuthenticationPlugin(),
this.conf.getBrokerClientAuthenticationParameters()).build());
@AfterClass(alwaysRun = true)
public void cleanup() throws Exception {
super.cleanup();
}

@Override
protected void createAdmin() throws Exception {
super.admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString())
.authentication(this.conf.getBrokerClientAuthenticationPlugin(),
this.conf.getBrokerClientAuthenticationParameters()).build());
}


@Test(timeOut = 30 * 1000)
@Test(timeOut = 30000)
public void testSuperUserProduceAndConsume() throws PulsarAdminException {
String superUserToken = AuthTokenUtils.createToken(secretKey, "pass.pass", Optional.empty());
String topic = "testSuperUserProduceAndConsumeTopic";
String fullNewTopicName = "persistent://" + TENANT + "/" + NAMESPACE + "/" + topic;
KProducer kProducer = new KProducer(topic, false, "localhost", getKafkaBrokerPort(),
TENANT + "/" + NAMESPACE, "token:" + superUserToken);
int totalMsgs = 10;
String messageStrPrefix = topic + "_message_";

for (int i = 0; i < totalMsgs; i++) {
String messageStr = messageStrPrefix + i;
kProducer.getProducer().send(new ProducerRecord<>(topic, i, messageStr));
}
KConsumer kConsumer = new KConsumer(topic, "localhost", getKafkaBrokerPort(), false,
TENANT + "/" + NAMESPACE, "token:" + superUserToken, "DemoKafkaOnPulsarConsumer");
kConsumer.getConsumer().subscribe(Collections.singleton(topic));

int i = 0;
while (i < totalMsgs) {
ConsumerRecords<Integer, String> records = kConsumer.getConsumer().poll(Duration.ofSeconds(1));
for (ConsumerRecord<Integer, String> record : records) {
Integer key = record.key();
assertEquals(messageStrPrefix + key.toString(), record.value());
i++;
}
}
assertEquals(i, totalMsgs);

// no more records
ConsumerRecords<Integer, String> records = kConsumer.getConsumer().poll(Duration.ofMillis(200));
assertTrue(records.isEmpty());

// ensure that we can list the topic
Map<String, List<PartitionInfo>> result = kConsumer.getConsumer().listTopics(Duration.ofSeconds(1));
assertEquals(result.size(), 1);
assertTrue(result.containsKey(topic),
"list of topics " + result.keySet() + " does not contains " + topic);

// Cleanup
kProducer.close();
kConsumer.close();
admin.topics().deletePartitionedTopic(fullNewTopicName);
super.testSuperUserProduceAndConsume();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/**
* Licensed 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 io.streamnative.pulsar.handlers.kop.security.auth;

import static org.mockito.Mockito.spy;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertTrue;

import com.google.common.collect.Sets;
import io.jsonwebtoken.SignatureAlgorithm;
import io.streamnative.pulsar.handlers.kop.KopProtocolHandlerTestBase;
import java.time.Duration;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import javax.crypto.SecretKey;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.PartitionInfo;
import org.apache.pulsar.broker.authentication.AuthenticationProviderToken;
import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.admin.PulsarAdminException;
import org.apache.pulsar.client.impl.auth.AuthenticationToken;

/**
* Unit test for Authorization with `entryFormat=pulsar`.
*/
public class KafkaAuthorizationMockTestBase extends KopProtocolHandlerTestBase {

protected static final String TENANT = "KafkaAuthorizationTest";
protected static final String NAMESPACE = "ns1";
protected static final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256);

protected static final String ADMIN_USER = "pass.pass";
protected String authorizationProviderClassName = KafkaMockAuthorizationProvider.class.getName();

@Override
protected void setup() throws Exception {
Properties properties = new Properties();
properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY));

String adminToken = AuthTokenUtils.createToken(SECRET_KEY, ADMIN_USER, Optional.empty());

conf.setSaslAllowedMechanisms(Sets.newHashSet("PLAIN"));
conf.setKafkaMetadataTenant("internal");
conf.setKafkaMetadataNamespace("__kafka");
conf.setKafkaTenant(TENANT);
conf.setKafkaNamespace(NAMESPACE);

conf.setClusterName(super.configClusterName);
conf.setAuthorizationEnabled(true);
conf.setAuthenticationEnabled(true);
conf.setAuthorizationAllowWildcardsMatching(true);
conf.setAuthorizationProvider(authorizationProviderClassName);
conf.setAuthenticationProviders(
Sets.newHashSet(AuthenticationProviderToken.class.getName()));
conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName());
conf.setBrokerClientAuthenticationParameters("token:" + adminToken);
conf.setProperties(properties);

super.internalSetup();
}

@Override
protected void cleanup() throws Exception {
super.admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString())
.authentication(this.conf.getBrokerClientAuthenticationPlugin(),
this.conf.getBrokerClientAuthenticationParameters()).build());
}

@Override
protected void createAdmin() throws Exception {
super.admin = spy(PulsarAdmin.builder().serviceHttpUrl(brokerUrl.toString())
.authentication(this.conf.getBrokerClientAuthenticationPlugin(),
this.conf.getBrokerClientAuthenticationParameters()).build());
}

public void testSuperUserProduceAndConsume() throws PulsarAdminException {
String superUserToken = AuthTokenUtils.createToken(SECRET_KEY, "pass.pass", Optional.empty());
String topic = "testSuperUserProduceAndConsumeTopic";
String fullNewTopicName = "persistent://" + TENANT + "/" + NAMESPACE + "/" + topic;
KProducer kProducer = new KProducer(topic, false, "localhost", getKafkaBrokerPort(),
TENANT + "/" + NAMESPACE, "token:" + superUserToken);
int totalMsgs = 10;
String messageStrPrefix = topic + "_message_";

for (int i = 0; i < totalMsgs; i++) {
String messageStr = messageStrPrefix + i;
kProducer.getProducer().send(new ProducerRecord<>(topic, i, messageStr));
}
KConsumer kConsumer = new KConsumer(topic, "localhost", getKafkaBrokerPort(), false,
TENANT + "/" + NAMESPACE, "token:" + superUserToken, "DemoKafkaOnPulsarConsumer");
kConsumer.getConsumer().subscribe(Collections.singleton(topic));

int i = 0;
while (i < totalMsgs) {
ConsumerRecords<Integer, String> records = kConsumer.getConsumer().poll(Duration.ofSeconds(1));
for (ConsumerRecord<Integer, String> record : records) {
Integer key = record.key();
assertEquals(messageStrPrefix + key.toString(), record.value());
i++;
}
}
assertEquals(i, totalMsgs);

// no more records
ConsumerRecords<Integer, String> records = kConsumer.getConsumer().poll(Duration.ofMillis(200));
assertTrue(records.isEmpty());

// ensure that we can list the topic
Map<String, List<PartitionInfo>> result = kConsumer.getConsumer().listTopics(Duration.ofSeconds(1));
assertEquals(result.size(), 1);
assertTrue(result.containsKey(topic),
"list of topics " + result.keySet() + " does not contains " + topic);

// Cleanup
kProducer.close();
kConsumer.close();
admin.topics().deletePartitionedTopic(fullNewTopicName);
}
}
Loading

0 comments on commit 45db3ed

Please sign in to comment.