2222import static com .google .common .base .Preconditions .checkNotNull ;
2323
2424import java .io .Serializable ;
25+ import java .util .HashMap ;
26+ import java .util .Map ;
2527
26- import org .apache .pulsar .client .api .Authentication ;
27- import org .apache .pulsar .client .api .Consumer ;
28- import org .apache .pulsar .client .api .MessageListener ;
29- import org .apache .pulsar .client .api .PulsarClient ;
30- import org .apache .pulsar .client .api .PulsarClientException ;
28+ import org .apache .pulsar .client .api .*;
3129import org .apache .pulsar .client .impl .PulsarClientImpl ;
30+ import org .apache .pulsar .client .impl .conf .ClientConfigurationData ;
3231import org .apache .pulsar .client .impl .conf .ConsumerConfigurationData ;
3332import org .apache .spark .storage .StorageLevel ;
3433import org .apache .spark .streaming .receiver .Receiver ;
@@ -43,34 +42,52 @@ public class SparkStreamingPulsarReceiver extends Receiver<byte[]> {
4342 private static final Logger LOG = LoggerFactory .getLogger (SparkStreamingPulsarReceiver .class );
4443
4544 private String serviceUrl ;
46- private ConsumerConfigurationData <byte []> conf ;
45+ private Map <String ,Object > clientConfig ;
46+ private ConsumerConfigurationData <byte []> consumerConfig ;
4747 private Authentication authentication ;
4848 private PulsarClient pulsarClient ;
4949 private Consumer <byte []> consumer ;
5050
5151 public SparkStreamingPulsarReceiver (
5252 String serviceUrl ,
53- ConsumerConfigurationData <byte []> conf ,
53+ ConsumerConfigurationData <byte []> consumerConfig ,
5454 Authentication authentication ) {
55- this (StorageLevel .MEMORY_AND_DISK_2 (), serviceUrl , conf , authentication );
55+ this (StorageLevel .MEMORY_AND_DISK_2 (), serviceUrl , new HashMap <>(), consumerConfig , authentication );
56+ }
57+
58+ public SparkStreamingPulsarReceiver (
59+ String serviceUrl ,
60+ Map <String ,Object > clientConfig ,
61+ ConsumerConfigurationData <byte []> consumerConfig ,
62+ Authentication authentication ) {
63+ this (StorageLevel .MEMORY_AND_DISK_2 (), serviceUrl , clientConfig , consumerConfig , authentication );
64+ }
65+
66+ public SparkStreamingPulsarReceiver (StorageLevel storageLevel ,
67+ String serviceUrl ,
68+ ConsumerConfigurationData <byte []> consumerConf ,
69+ Authentication authentication ) {
70+ this (StorageLevel .MEMORY_AND_DISK_2 (), serviceUrl , new HashMap <>(), consumerConf , authentication );
5671 }
5772
5873 public SparkStreamingPulsarReceiver (StorageLevel storageLevel ,
5974 String serviceUrl ,
60- ConsumerConfigurationData <byte []> conf ,
75+ Map <String ,Object > clientConfig ,
76+ ConsumerConfigurationData <byte []> consumerConfig ,
6177 Authentication authentication ) {
6278 super (storageLevel );
6379
6480 checkNotNull (serviceUrl , "serviceUrl must not be null" );
65- checkNotNull (conf , "ConsumerConfigurationData must not be null" );
66- checkArgument (conf .getTopicNames ().size () > 0 , "TopicNames must be set a value." );
67- checkNotNull (conf .getSubscriptionName (), "SubscriptionName must not be null" );
81+ checkNotNull (consumerConfig , "ConsumerConfigurationData must not be null" );
82+ checkNotNull (clientConfig , "Client configuration map must not be null" );
83+ checkArgument (consumerConfig .getTopicNames ().size () > 0 , "TopicNames must be set a value." );
84+ checkNotNull (consumerConfig .getSubscriptionName (), "SubscriptionName must not be null" );
6885
6986 this .serviceUrl = serviceUrl ;
7087 this .authentication = authentication ;
7188
72- if (conf .getMessageListener () == null ) {
73- conf .setMessageListener ((MessageListener <byte []> & Serializable ) (consumer , msg ) -> {
89+ if (consumerConfig .getMessageListener () == null ) {
90+ consumerConfig .setMessageListener ((MessageListener <byte []> & Serializable ) (consumer , msg ) -> {
7491 try {
7592 store (msg .getData ());
7693 consumer .acknowledgeAsync (msg );
@@ -80,13 +97,18 @@ public SparkStreamingPulsarReceiver(StorageLevel storageLevel,
8097 }
8198 });
8299 }
83- this .conf = conf ;
100+ this .clientConfig = clientConfig ;
101+ this .consumerConfig = consumerConfig ;
84102 }
85103
86104 public void onStart () {
87105 try {
88- pulsarClient = PulsarClient .builder ().serviceUrl (serviceUrl ).authentication (authentication ).build ();
89- consumer = ((PulsarClientImpl ) pulsarClient ).subscribeAsync (conf ).join ();
106+ ClientBuilder builder = PulsarClient .builder ().serviceUrl (serviceUrl ).authentication (authentication );
107+ if (!clientConfig .isEmpty ()) {
108+ builder .loadConf (clientConfig );
109+ }
110+ pulsarClient = builder .build ();
111+ consumer = ((PulsarClientImpl ) pulsarClient ).subscribeAsync (consumerConfig ).join ();
90112 } catch (Exception e ) {
91113 LOG .error ("Failed to start subscription : {}" , e .getMessage ());
92114 restart ("Restart a consumer" );
0 commit comments