-
Notifications
You must be signed in to change notification settings - Fork 31
[pulsar-spark] added option for configuring Pulsar client #14
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
Open
atezs82
wants to merge
1
commit into
apache:master
Choose a base branch
from
atezs82:pulsar_spark_add_client_config
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,12 +22,15 @@ | |
| import static com.google.common.base.Preconditions.checkNotNull; | ||
|
|
||
| import java.io.Serializable; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
|
|
||
| import org.apache.pulsar.client.api.Authentication; | ||
| import org.apache.pulsar.client.api.Consumer; | ||
| import org.apache.pulsar.client.api.MessageListener; | ||
| import org.apache.pulsar.client.api.PulsarClient; | ||
| import org.apache.pulsar.client.api.PulsarClientException; | ||
| import org.apache.pulsar.client.api.ClientBuilder; | ||
| import org.apache.pulsar.client.impl.PulsarClientImpl; | ||
| import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; | ||
| import org.apache.spark.storage.StorageLevel; | ||
|
|
@@ -43,34 +46,52 @@ public class SparkStreamingPulsarReceiver extends Receiver<byte[]> { | |
| private static final Logger LOG = LoggerFactory.getLogger(SparkStreamingPulsarReceiver.class); | ||
|
|
||
| private String serviceUrl; | ||
| private ConsumerConfigurationData<byte[]> conf; | ||
| private Map<String,Object> clientConfig; | ||
| private ConsumerConfigurationData<byte[]> consumerConfig; | ||
| private Authentication authentication; | ||
| private PulsarClient pulsarClient; | ||
| private Consumer<byte[]> consumer; | ||
|
|
||
| public SparkStreamingPulsarReceiver( | ||
| String serviceUrl, | ||
| ConsumerConfigurationData<byte[]> conf, | ||
| ConsumerConfigurationData<byte[]> consumerConfig, | ||
| Authentication authentication) { | ||
| this(StorageLevel.MEMORY_AND_DISK_2(), serviceUrl, conf, authentication); | ||
| this(StorageLevel.MEMORY_AND_DISK_2(), serviceUrl, new HashMap<>(), consumerConfig, authentication); | ||
| } | ||
|
|
||
| public SparkStreamingPulsarReceiver( | ||
| String serviceUrl, | ||
| Map<String,Object> clientConfig, | ||
| ConsumerConfigurationData<byte[]> consumerConfig, | ||
| Authentication authentication) { | ||
| this(StorageLevel.MEMORY_AND_DISK_2(), serviceUrl, clientConfig, consumerConfig, authentication); | ||
| } | ||
|
|
||
| public SparkStreamingPulsarReceiver(StorageLevel storageLevel, | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We are introducing a new parameter
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for the finding, corrected this in the upcoming patchset. |
||
| String serviceUrl, | ||
| ConsumerConfigurationData<byte[]> consumerConf, | ||
| Authentication authentication) { | ||
| this(storageLevel, serviceUrl, new HashMap<>(), consumerConf, authentication); | ||
| } | ||
|
|
||
| public SparkStreamingPulsarReceiver(StorageLevel storageLevel, | ||
| String serviceUrl, | ||
| ConsumerConfigurationData<byte[]> conf, | ||
| Map<String,Object> clientConfig, | ||
| ConsumerConfigurationData<byte[]> consumerConfig, | ||
| Authentication authentication) { | ||
| super(storageLevel); | ||
|
|
||
| checkNotNull(serviceUrl, "serviceUrl must not be null"); | ||
| checkNotNull(conf, "ConsumerConfigurationData must not be null"); | ||
| checkArgument(conf.getTopicNames().size() > 0, "TopicNames must be set a value."); | ||
| checkNotNull(conf.getSubscriptionName(), "SubscriptionName must not be null"); | ||
| checkNotNull(consumerConfig, "ConsumerConfigurationData must not be null"); | ||
| checkNotNull(clientConfig, "Client configuration map must not be null"); | ||
| checkArgument(consumerConfig.getTopicNames().size() > 0, "TopicNames must be set a value."); | ||
| checkNotNull(consumerConfig.getSubscriptionName(), "SubscriptionName must not be null"); | ||
|
|
||
| this.serviceUrl = serviceUrl; | ||
| this.authentication = authentication; | ||
|
|
||
| if (conf.getMessageListener() == null) { | ||
| conf.setMessageListener((MessageListener<byte[]> & Serializable) (consumer, msg) -> { | ||
| if (consumerConfig.getMessageListener() == null) { | ||
| consumerConfig.setMessageListener((MessageListener<byte[]> & Serializable) (consumer, msg) -> { | ||
| try { | ||
| store(msg.getData()); | ||
| consumer.acknowledgeAsync(msg); | ||
|
|
@@ -80,13 +101,18 @@ public SparkStreamingPulsarReceiver(StorageLevel storageLevel, | |
| } | ||
| }); | ||
| } | ||
| this.conf = conf; | ||
| this.clientConfig = clientConfig; | ||
| this.consumerConfig = consumerConfig; | ||
| } | ||
|
|
||
| public void onStart() { | ||
| try { | ||
| pulsarClient = PulsarClient.builder().serviceUrl(serviceUrl).authentication(authentication).build(); | ||
| consumer = ((PulsarClientImpl) pulsarClient).subscribeAsync(conf).join(); | ||
| ClientBuilder builder = PulsarClient.builder().serviceUrl(serviceUrl).authentication(authentication); | ||
| if (!clientConfig.isEmpty()) { | ||
| builder.loadConf(clientConfig); | ||
| } | ||
| pulsarClient = builder.build(); | ||
| consumer = ((PulsarClientImpl) pulsarClient).subscribeAsync(consumerConfig).join(); | ||
| } catch (Exception e) { | ||
| LOG.error("Failed to start subscription : {}", e.getMessage()); | ||
| restart("Restart a consumer"); | ||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why we are going to introduce so many new builders?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The idea behind adding 2 new constructors to the 2 existing ones is I wanted to allow users of the interface to be able to configure Pulsar clients regardless whether they want to specify a storage level or leave it to default. Also, I wanted to keep previous constructors as well to keep backward compatibility.
Please let me know what do you think about this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would be happy to remove eg. the one having a default storage level:
In that way we might have broader functionality (we just enforce the user to set the storage level whenever client configuration is provided).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@sijie What do you think about this proposal? Shall I change the code like this?