Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.broker.resources.PulsarResources;
import org.apache.pulsar.common.naming.NamespaceName;
import org.apache.pulsar.common.naming.SystemTopicNames;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.AuthAction;
import org.apache.pulsar.common.policies.data.AuthPolicies;
Expand Down Expand Up @@ -155,6 +156,9 @@ public CompletableFuture<Boolean> canConsumeAsync(TopicName topicName, String ro
@Override
public CompletableFuture<Boolean> canLookupAsync(TopicName topicName, String role,
AuthenticationDataSource authenticationData) {
if (SystemTopicNames.isTransactionCoordinatorAssign(topicName)) {
return CompletableFuture.completedFuture(true);
}
return canProduceAsync(topicName, role, authenticationData)
.thenCompose(canProduce -> {
if (canProduce) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -317,6 +317,62 @@ public void testNoAuth() throws Exception {
}
}

@Test
public void testTcLookupAuth() throws Exception {
String role = "test";
String tcSystemTopic = "persistent://pulsar/system/transaction_coordinator_assign";
admin.topics().grantPermission(TOPIC, role, Sets.newHashSet(AuthAction.produce, AuthAction.consume));

@Cleanup
final PulsarClient pulsarClient = PulsarClient.builder()
.serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl())
.authentication(AuthenticationFactory.token(generateToken(kp, "test")))
.enableTransaction(true)
.build();

try {
@Cleanup final Producer<String> producer = pulsarClient
.newProducer(Schema.STRING)
.sendTimeout(60, TimeUnit.SECONDS)
.topic(TOPIC)
.create();

Transaction transaction = pulsarClient.newTransaction()
.withTransactionTimeout(60, TimeUnit.SECONDS).build().get();
producer.newMessage(transaction).value("message").send();
final Throwable ex = syncGetException((
(PulsarClientImpl) pulsarClient).getTcClient().commitAsync(transaction.getTxnID())
);
Assert.assertNull(ex);
} catch (Exception e) {
Assert.fail();
}

try {
@Cleanup final Producer<String> producer = pulsarClient
.newProducer(Schema.STRING)
.sendTimeout(60, TimeUnit.SECONDS)
.topic(tcSystemTopic)
.create();
Assert.fail();
} catch (Exception e) {
Assert.assertTrue(e instanceof PulsarClientException.AuthorizationException);
Assert.assertTrue(e.getMessage().contains("Client is not authorized to Produce"));
}

try {
@Cleanup final Consumer<String> consumer = pulsarClient
.newConsumer(Schema.STRING)
.subscriptionName("test")
.topic(tcSystemTopic)
.subscribe();
Assert.fail();
} catch (Exception e) {
Assert.assertTrue(e instanceof PulsarClientException.AuthorizationException);
Assert.assertTrue(e.getMessage().contains("Client is not authorized to subscribe"));
}
}

private static Throwable syncGetException(CompletableFuture<?> future) {
try {
future.get();
Expand Down