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

[Issue 905][api][pulsar-client] add serializable token class #10574

Merged
merged 1 commit into from
May 21, 2021
Merged

[Issue 905][api][pulsar-client] add serializable token class #10574

merged 1 commit into from
May 21, 2021

Conversation

atezs82
Copy link
Contributor

@atezs82 atezs82 commented May 13, 2021

Motivation

When trying to authenticate Pulsar in distributed systems (a very good example of this is the Spark connector, see here: https://pulsar.apache.org/docs/en/adaptors-spark/), the following exception is thrown (this is taken from the log of one of the Spark executors):

21/05/13 11:59:34 WARN PulsarClientImpl: [topic: persistent://tenant/namespace/topic] Could not get connection while getPartitionedTopicMetadata -- Will try again in 380 ms
21/05/13 11:59:34 INFO ConnectionPool: [[id: 0x4d13ed61, L:/1.2.3.4:43624 - R:broker.svc.cluster.local/1.2.3.4:6650]] Connected to server
21/05/13 11:59:34 WARN ClientCnx: [broker.svc.cluster.local/1.2.3.4:6650] Got exception java.lang.RuntimeException: failed to get client token
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:62)
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getCommandData(AuthenticationDataToken.java:55)
	at org.apache.pulsar.client.api.AuthenticationDataProvider.authenticate(AuthenticationDataProvider.java:133)
	at org.apache.pulsar.client.impl.ClientCnx.newConnectCommand(ClientCnx.java:218)
	at org.apache.pulsar.client.impl.ClientCnx.channelActive(ClientCnx.java:199)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelActive(AbstractChannelHandlerContext.java:209)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelActive(DefaultChannelPipeline.java:1398)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline.fireChannelActive(DefaultChannelPipeline.java:895)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:620)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:653)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:529)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:465)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
	at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NullPointerException
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:60)
	... 20 more

This happens since the token supplier function become null when it gets transferred to eg. a Spark executor. Refer to this issue (which is roughly the same): https://github.com/streamnative/pulsar/issues/905 .

Modifications

Added a new token storage class, SerializableAuthenticationToken, which can be serialized properly (this is tested in its corresponding unit test as well). On the other hand, this class does not use supplier functions to fetch token data, it stores the token as a plain string instead. For use-cases requiring a token supplier, this cannot be used, but can load its configuration from file (using the file:// URL prefix), like the original AuthenticationToken class.

Corresponding unit test were added to the class. Will also add proper documentation prior to merging once the change can be accepted - if that is required.

Verifying this change

  • Make sure that the change passes the CI checks.

This change added tests and can be verified as follows:

  • added unit tests for the new class to ensure that it supports most functionality as the original AuthenticationToken
  • added an unit test to ensure that the token inside the class remains intact when going through serialization

Does this pull request potentially affect one of the following parts:

If yes was chosen, please highlight the changes

  • Dependencies (does it add or upgrade a dependency): no
  • The public API: yes (the Java API allows to use SerializableAuthenticationToken upon connecting as well, apart from the existing authentication methods)
  • The schema: no
  • The default values of configurations: no
  • The wire protocol: no, but if this feature is used, JWT tokens in plain text format will be transferred throughout the network
  • The rest endpoints: no
  • The admin cli options: no
  • Anything that affects deployment: don't know

Documentation

  • Does this pull request introduce a new feature? yes (in the Java Client API)
  • If yes, how is the feature documented? brief JavaDocs, will extend public docs as well if needed

@atezs82 atezs82 marked this pull request as draft May 13, 2021 12:59
@eolivelli
Copy link
Contributor

thank you @atezs82
We will review this patch

@lhotari PTAL

@merlimat
Copy link
Contributor

@atezs82 I'd rather not introduce a new auth provider here duplicating most of the code. Is there a way that this can be fixed/enhanced within the context of the existing AuthenticationProviderToken?

// Read token from a file
URI filePath = URI.create(encodedAuthParamString);
try {
this.token = new String(Files.readAllBytes(Paths.get(filePath)), Charsets.UTF_8).trim();
Copy link
Contributor

Choose a reason for hiding this comment

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

This has a problem that it only reads the content of the file once, at startup. When the token file gets refreshed, the client will still be operating with the old credentials, and the broker will disconnect it when the token expires

@atezs82
Copy link
Contributor Author

atezs82 commented May 14, 2021

@merlimat thanks for the review, I'll try to modify this accordingly.

@atezs82
Copy link
Contributor Author

atezs82 commented May 14, 2021

@merlimat I pushed a slightly different version without new classes (just modified AuthenticationToken). Let me know what do you think.

Note that in theory it is still possible to provide a not-serializable token supplier, which would still give back that cryptic NullPointerException above.

@@ -107,4 +101,38 @@ public void start() throws PulsarClientException {
// noop
}

public static class SerializableURITokenSupplier implements Supplier<String>, Serializable {
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
public static class SerializableURITokenSupplier implements Supplier<String>, Serializable {
private static class SerializableURITokenSupplier implements Supplier<String>, Serializable {

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

public static class SerializableURITokenSupplier implements Supplier<String>, Serializable {

private static final long serialVersionUID = 3160666668166028760L;
private URI uri;
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
private URI uri;
private final URI uri;

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

public static class SerializableTokenSupplier implements Supplier<String>, Serializable {

private static final long serialVersionUID = 5095234161799506913L;
private String token;
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
private String token;
private final String token;

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

@merlimat
Copy link
Contributor

@atezs82 Change LGTM, just left few minor suggestions.

@merlimat merlimat added this to the 2.8.0 milestone May 17, 2021
@merlimat merlimat added the type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages label May 17, 2021
@atezs82
Copy link
Contributor Author

atezs82 commented May 18, 2021

@merlimat Thanks, I'll fix these and push the changes.

@atezs82 atezs82 marked this pull request as ready for review May 18, 2021 13:36
@merlimat
Copy link
Contributor

@atezs82 There seems to be an issue reported by spotbugs tool:

Error:  Medium: The field org.apache.pulsar.client.impl.auth.AuthenticationDataTls.certStreamProvider is transient but isn't set by deserialization [org.apache.pulsar.client.impl.auth.AuthenticationDataTls] In AuthenticationDataTls.java SE_TRANSIENT_FIELD_NOT_RESTORED
2473
Error:  Medium: The field org.apache.pulsar.client.impl.auth.AuthenticationDataTls.keyStreamProvider is transient but isn't set by deserialization [org.apache.pulsar.client.impl.auth.AuthenticationDataTls] In AuthenticationDataTls.java SE_TRANSIENT_FIELD_NOT_RESTORED
2474
Error:  Medium: The field org.apache.pulsar.client.impl.auth.AuthenticationDataTls.trustStoreStreamProvider is transient but isn't set by deserialization [org.apache.pulsar.client.impl.auth.AuthenticationDataTls] In AuthenticationDataTls.java SE_TRANSIENT_FIELD_NOT_RESTORED

@atezs82
Copy link
Contributor Author

atezs82 commented May 19, 2021

@merlimat Please excuse my limited knowledge on the pulsar-client. On the other hand, it seems that you have handled a very similar Findbugs issue in the AuthenticationTls class as well (line 49):

@SuppressFBWarnings(value = "SE_BAD_FIELD", justification = "Using custom serializer which Findbugs can't detect")
private Supplier<ByteArrayInputStream> certStreamProvider, keyStreamProvider, trustStoreStreamProvider;

Based on this, I also added a similar suppress annotation to AuthenticationDataTls (I am not exactly sure why this was missing from there though).

Let me know if this approach shall be OK or shall I modify the custom deserializer logic instead. Thanks!

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

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

great work

+1

@atezs82
Copy link
Contributor Author

atezs82 commented May 19, 2021

Regarding the CI failure it seems that a unit test for BROKER_GROUP_1 was failing (org.apache.pulsar.broker.loadbalance.LoadBalancerTest.testBrokerRanking). I tried to execute the whole test suite locally by running mvn clean install && ./build/run_unit_group.sh BROKER_GROUP_1, but eventually got a lot of failing tests due to various reasons (Tests run: 1168, Failures: 818, Errors: 0, Skipped: 345, I think most of the skips were happening because of the failures).

On the other hand, I managed to execute just the problematic unit test locally by running mvn clean install && mvn -B -ntp -pl pulsar-broker -Dtest=LoadBalancerTest (just copied the command form the .sh script). That ran without any problems:

[INFO] -------------------------------------------------------
[INFO]  T E S T S
[INFO] -------------------------------------------------------
[INFO] Running org.apache.pulsar.broker.loadbalance.LoadBalancerTest
[INFO] Tests run: 7, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 87.75 s - in org.apache.pulsar.broker.loadbalance.LoadBalancerTest
[INFO]
[INFO] Results:
[INFO]
[INFO] Tests run: 7, Failures: 0, Errors: 0, Skipped: 0
[INFO]

Not exactly sure on how to proceed with this. Shall these test be executed locally, or do they need some kind of CI environment to run? Given that I could execute this locally, I would re-run the workflow on the CI but I think I lack privileges for doing so.

@codelipenghui codelipenghui merged commit 5497107 into apache:master May 21, 2021
@atezs82 atezs82 deleted the serializable_auth_token branch June 16, 2021 07:37
yangl pushed a commit to yangl/pulsar that referenced this pull request Jun 23, 2021
)

### Motivation

When trying to authenticate Pulsar in distributed systems (a very good example of this is the Spark connector, see here: https://pulsar.apache.org/docs/en/adaptors-spark/), the following exception is thrown (this is taken from the log of one of the Spark executors):
```
21/05/13 11:59:34 WARN PulsarClientImpl: [topic: persistent://tenant/namespace/topic] Could not get connection while getPartitionedTopicMetadata -- Will try again in 380 ms
21/05/13 11:59:34 INFO ConnectionPool: [[id: 0x4d13ed61, L:/1.2.3.4:43624 - R:broker.svc.cluster.local/1.2.3.4:6650]] Connected to server
21/05/13 11:59:34 WARN ClientCnx: [broker.svc.cluster.local/1.2.3.4:6650] Got exception java.lang.RuntimeException: failed to get client token
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:62)
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getCommandData(AuthenticationDataToken.java:55)
	at org.apache.pulsar.client.api.AuthenticationDataProvider.authenticate(AuthenticationDataProvider.java:133)
	at org.apache.pulsar.client.impl.ClientCnx.newConnectCommand(ClientCnx.java:218)
	at org.apache.pulsar.client.impl.ClientCnx.channelActive(ClientCnx.java:199)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelActive(AbstractChannelHandlerContext.java:209)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelActive(DefaultChannelPipeline.java:1398)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline.fireChannelActive(DefaultChannelPipeline.java:895)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:620)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:653)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:529)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:465)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
	at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NullPointerException
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:60)
	... 20 more
```
This happens since the token supplier function become null when it gets transferred to eg. a Spark executor. Refer to this issue (which is roughly the same): https://github.com/streamnative/pulsar/issues/905 .

### Modifications
Added a new token storage class, `SerializableAuthenticationToken`, which can be serialized properly (this is tested in its corresponding unit test as well). On the other hand, this class does not use supplier functions to fetch token data, it stores the token as a plain string instead. For use-cases requiring a token supplier, this cannot be used, but can load its configuration from file (using the `file://` URL prefix), like the original `AuthenticationToken` class.

Corresponding unit test were added to the class. Will also add proper documentation prior to merging once the change can be accepted - if that is required.
bharanic-dev pushed a commit to bharanic-dev/pulsar that referenced this pull request Mar 18, 2022
)

### Motivation

When trying to authenticate Pulsar in distributed systems (a very good example of this is the Spark connector, see here: https://pulsar.apache.org/docs/en/adaptors-spark/), the following exception is thrown (this is taken from the log of one of the Spark executors):
```
21/05/13 11:59:34 WARN PulsarClientImpl: [topic: persistent://tenant/namespace/topic] Could not get connection while getPartitionedTopicMetadata -- Will try again in 380 ms
21/05/13 11:59:34 INFO ConnectionPool: [[id: 0x4d13ed61, L:/1.2.3.4:43624 - R:broker.svc.cluster.local/1.2.3.4:6650]] Connected to server
21/05/13 11:59:34 WARN ClientCnx: [broker.svc.cluster.local/1.2.3.4:6650] Got exception java.lang.RuntimeException: failed to get client token
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:62)
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getCommandData(AuthenticationDataToken.java:55)
	at org.apache.pulsar.client.api.AuthenticationDataProvider.authenticate(AuthenticationDataProvider.java:133)
	at org.apache.pulsar.client.impl.ClientCnx.newConnectCommand(ClientCnx.java:218)
	at org.apache.pulsar.client.impl.ClientCnx.channelActive(ClientCnx.java:199)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.fireChannelActive(AbstractChannelHandlerContext.java:209)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline$HeadContext.channelActive(DefaultChannelPipeline.java:1398)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:230)
	at org.apache.pulsar.shade.io.netty.channel.AbstractChannelHandlerContext.invokeChannelActive(AbstractChannelHandlerContext.java:216)
	at org.apache.pulsar.shade.io.netty.channel.DefaultChannelPipeline.fireChannelActive(DefaultChannelPipeline.java:895)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:620)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:653)
	at org.apache.pulsar.shade.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:529)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:465)
	at org.apache.pulsar.shade.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
	at org.apache.pulsar.shade.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
	at org.apache.pulsar.shade.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
	at org.apache.pulsar.shade.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NullPointerException
	at org.apache.pulsar.client.impl.auth.AuthenticationDataToken.getToken(AuthenticationDataToken.java:60)
	... 20 more
```
This happens since the token supplier function become null when it gets transferred to eg. a Spark executor. Refer to this issue (which is roughly the same): https://github.com/streamnative/pulsar/issues/905 .

### Modifications
Added a new token storage class, `SerializableAuthenticationToken`, which can be serialized properly (this is tested in its corresponding unit test as well). On the other hand, this class does not use supplier functions to fetch token data, it stores the token as a plain string instead. For use-cases requiring a token supplier, this cannot be used, but can load its configuration from file (using the `file://` URL prefix), like the original `AuthenticationToken` class.

Corresponding unit test were added to the class. Will also add proper documentation prior to merging once the change can be accepted - if that is required.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

4 participants