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 (#1999)
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.

Introduce two configs to configure the cache policy so that revoke
permission can work:
- kopAuthorizationCacheRefreshMs: the refresh timeout
- kopAuthorizationCacheMaxCountPerConnection: the max cache size
  • Loading branch information
BewareMyPower authored Aug 27, 2023
1 parent 5193592 commit 9ab8411
Show file tree
Hide file tree
Showing 10 changed files with 364 additions and 115 deletions.
2 changes: 2 additions & 0 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,8 @@ This section lists configurations about the authorization.
| Name | Description | Range | Default |
|-------------------------------------------|--------------------------------------------------------------------------------------------------------|-------------|---------|
| kafkaEnableAuthorizationForceGroupIdCheck | Whether to enable authorization force group ID check. Note: It only support for OAuth2 authentication. | true, false | false |
| kopAuthorizationCacheRefreshMs | If it's configured with a positive value N, each connection will cache the authorization results of PRODUCE and FETCH requests for at least N ms.<br>It could help improve the performance when authorization is enabled, but the permission revoke will also take N ms to take effect. | 1 .. 2147483647 | 30000 |
| kopAuthorizationCacheMaxCountPerConnection | If it's configured with a positive value N, each connection will cache at most N entries for PRODUCE or FETCH requests.<br>If it's non-positive, the cache size will be the default value. | 1 .. 2147483647 | 100 |


## SSL encryption
Expand Down
6 changes: 6 additions & 0 deletions kafka-impl/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,12 @@
<artifactId>test-listener</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.awaitility</groupId>
<artifactId>awaitility</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,14 @@
*/
package io.streamnative.pulsar.handlers.kop;

import com.github.benmanes.caffeine.cache.Caffeine;
import com.google.common.collect.Sets;
import io.streamnative.pulsar.handlers.kop.coordinator.group.OffsetConfig;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.time.Duration;
import java.util.Collections;
import java.util.HashSet;
import java.util.Properties;
Expand Down Expand Up @@ -564,6 +566,24 @@ public class KafkaServiceConfiguration extends ServiceConfiguration {
)
private boolean skipMessagesWithoutIndex = false;

@FieldContext(
category = CATEGORY_KOP,
doc = "If it's configured with a positive value N, each connection will cache the authorization results "
+ "of PRODUCE and FETCH requests for at least N ms.\n"
+ "It could help improve the performance when authorization is enabled, but the permission revoke "
+ "will also take N ms to take effect.\nDefault: 30000 (30 seconds)"
)
private int kopAuthorizationCacheRefreshMs = 30000;

@FieldContext(
category = CATEGORY_KOP,
doc = "If it's configured with a positive value N, each connection will cache at most N "
+ "entries for PRODUCE or FETCH requests.\n"
+ "Default: 100\n"
+ "If it's non-positive, the cache size will be the default value."
)
private int kopAuthorizationCacheMaxCountPerConnection = 100;

private String checkAdvertisedListeners(String advertisedListeners) {
StringBuilder listenersReBuilder = new StringBuilder();
for (String listener : advertisedListeners.split(EndPoint.END_POINT_SEPARATOR)) {
Expand Down Expand Up @@ -629,4 +649,14 @@ public String getListeners() {
return kopAllowedNamespaces;
}

public Caffeine<Object, Object> getAuthorizationCacheBuilder() {
if (kopAuthorizationCacheRefreshMs <= 0) {
return Caffeine.newBuilder().maximumSize(0);
} else {
int maximumSize = (kopAuthorizationCacheMaxCountPerConnection >= 0)
? kopAuthorizationCacheMaxCountPerConnection : 100;
return Caffeine.newBuilder().maximumSize(maximumSize)
.expireAfterWrite(Duration.ofMillis(kopAuthorizationCacheRefreshMs));
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,15 @@
*/
package io.streamnative.pulsar.handlers.kop.security.auth;

import com.github.benmanes.caffeine.cache.Cache;
import io.streamnative.pulsar.handlers.kop.KafkaServiceConfiguration;
import io.streamnative.pulsar.handlers.kop.security.KafkaPrincipal;
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,11 +42,18 @@ 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 Cache<Pair<TopicName, String>, Boolean> produceCache;
// key is (topic, role, group)
private final Cache<Triple<TopicName, String, String>, Boolean> fetchCache;

public SimpleAclAuthorizer(PulsarService pulsarService, KafkaServiceConfiguration config) {
this.pulsarService = pulsarService;
this.authorizationService = pulsarService.getBrokerService().getAuthorizationService();
this.forceCheckGroupId = config.isKafkaEnableAuthorizationForceGroupIdCheck();
this.produceCache = config.getAuthorizationCacheBuilder().build();
this.fetchCache = config.getAuthorizationCacheBuilder().build();
}

protected PulsarService getPulsarService() {
Expand Down Expand Up @@ -151,7 +161,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.getIfPresent(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 +180,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.getIfPresent(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 @@ -19,8 +19,10 @@
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotNull;
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;

import com.github.benmanes.caffeine.cache.Cache;
import com.google.common.collect.Sets;
import io.streamnative.pulsar.handlers.kop.utils.ConfigurationUtils;
import java.io.File;
Expand All @@ -31,17 +33,21 @@
import java.io.PrintWriter;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.Objects;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.stream.IntStream;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.api.DigestType;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.ServiceConfiguration;
import org.apache.pulsar.broker.ServiceConfigurationUtils;
import org.apache.pulsar.broker.resources.NamespaceResources;
import org.apache.pulsar.broker.resources.PulsarResources;
import org.awaitility.Awaitility;
import org.testng.annotations.Test;

/**
Expand Down Expand Up @@ -283,4 +289,36 @@ public void testKopMigrationServiceConfiguration() {
assertTrue(configuration.isKopMigrationEnable());
assertEquals(port, configuration.getKopMigrationServicePort());
}

@Test(timeOut = 10000)
public void testKopAuthorizationCache() throws InterruptedException {
KafkaServiceConfiguration configuration = new KafkaServiceConfiguration();
configuration.setKopAuthorizationCacheRefreshMs(500);
configuration.setKopAuthorizationCacheMaxCountPerConnection(5);
Cache<Integer, Integer> cache = configuration.getAuthorizationCacheBuilder().build();
for (int i = 0; i < 5; i++) {
assertNull(cache.getIfPresent(1));
}
for (int i = 0; i < 10; i++) {
cache.put(i, i + 100);
}
Awaitility.await().atMost(Duration.ofMillis(100)).pollInterval(Duration.ofMillis(1))
.until(() -> IntStream.range(0, 10).mapToObj(cache::getIfPresent)
.filter(Objects::nonNull).count() <= 5);
IntStream.range(0, 10).mapToObj(cache::getIfPresent).filter(Objects::nonNull).map(i -> i - 100).forEach(key ->
assertEquals(cache.getIfPresent(key), Integer.valueOf(key + 100)));

Thread.sleep(600); // wait until the cache expired
for (int i = 0; i < 10; i++) {
assertNull(cache.getIfPresent(i));
}

configuration.setKopAuthorizationCacheRefreshMs(0);
Cache<Integer, Integer> cache2 = configuration.getAuthorizationCacheBuilder().build();
for (int i = 0; i < 5; i++) {
cache2.put(i, i);
}
Awaitility.await().atMost(Duration.ofMillis(10)).pollInterval(Duration.ofMillis(1))
.until(() -> IntStream.range(0, 5).mapToObj(cache2::getIfPresent).noneMatch(Objects::nonNull));
}
}
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();
}
}
Loading

0 comments on commit 9ab8411

Please sign in to comment.