Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve][authentication] Pass the authorization when user lookup transactionCoordinator topic #22744

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
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)) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should write this code in the org.apache.pulsar.broker.authentication.AuthenticationService.

That looks like an important change, could you discuss this in the mailing list?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why should we implement it in AuthenticationService?

Copy link
Contributor

@congbobo184 congbobo184 Jul 17, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how do we control user can use transaction?
It seem like a good way to control user to use transaction, if user have the produce role of TC system topic, then can send transaction message and can do lookup for tc topic. this only a idea. WDYT? @thetumbled @TakaHiR07

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to restrict user to use transaction feature by authorizing the lookup permission of TC system topic?
It is weird as there is no official document pointing out this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is just a thought, as we currently have no means to restrict users from using transactions.

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