31
31
import org .apache .pulsar .client .admin .PulsarAdminException ;
32
32
import org .apache .pulsar .client .admin .PulsarAdminException .NotFoundException ;
33
33
import org .apache .pulsar .client .api .Consumer ;
34
+ import org .apache .pulsar .client .api .ConsumerBuilder ;
34
35
import org .apache .pulsar .client .api .Message ;
35
36
import org .apache .pulsar .client .api .MessageId ;
36
37
import org .apache .pulsar .client .api .Producer ;
38
+ import org .apache .pulsar .client .api .ProducerBuilder ;
37
39
import org .apache .pulsar .client .api .PulsarClient ;
38
40
import org .apache .pulsar .client .api .PulsarClientException ;
39
41
import org .apache .pulsar .client .api .Schema ;
40
- import org .apache .pulsar .client .api .SubscriptionInitialPosition ;
41
42
import org .apache .pulsar .client .api .TypedMessageBuilder ;
42
43
import org .apache .pulsar .client .api .transaction .TransactionCoordinatorClient ;
43
44
import org .apache .pulsar .client .api .transaction .TxnID ;
49
50
import java .io .IOException ;
50
51
import java .time .Duration ;
51
52
import java .util .ArrayList ;
52
- import java .util .Arrays ;
53
53
import java .util .Collection ;
54
54
import java .util .List ;
55
- import java .util .Map ;
56
55
import java .util .Random ;
57
56
import java .util .concurrent .ConcurrentHashMap ;
58
57
import java .util .concurrent .ExecutionException ;
62
61
import static java .util .Collections .emptyList ;
63
62
import static java .util .Collections .singletonList ;
64
63
import static java .util .concurrent .TimeUnit .MILLISECONDS ;
65
- import static java .util .function .Function .identity ;
66
64
import static java .util .stream .Collectors .toList ;
67
- import static java .util .stream .Collectors .toMap ;
68
65
import static org .apache .commons .lang3 .RandomStringUtils .randomAlphanumeric ;
69
66
import static org .apache .flink .connector .base .DeliveryGuarantee .EXACTLY_ONCE ;
70
67
import static org .apache .flink .connector .pulsar .common .config .PulsarOptions .PULSAR_ADMIN_URL ;
79
76
import static org .apache .flink .connector .pulsar .source .enumerator .topic .TopicNameUtils .topicName ;
80
77
import static org .apache .flink .connector .pulsar .source .enumerator .topic .TopicNameUtils .topicNameWithPartition ;
81
78
import static org .apache .flink .util .Preconditions .checkArgument ;
79
+ import static org .apache .pulsar .client .api .SubscriptionInitialPosition .Earliest ;
82
80
import static org .apache .pulsar .client .api .SubscriptionMode .Durable ;
83
81
import static org .apache .pulsar .client .api .SubscriptionType .Exclusive ;
82
+ import static org .apache .pulsar .common .partition .PartitionedTopicMetadata .NON_PARTITIONED ;
84
83
85
84
/**
86
85
* A pulsar cluster operator used for operating pulsar instance. It's serializable for using in
@@ -178,7 +177,7 @@ public <T> void setupTopic(
178
177
*/
179
178
public void createTopic (String topic , int numberOfPartitions ) {
180
179
checkArgument (numberOfPartitions >= 0 );
181
- if (numberOfPartitions < = 0 ) {
180
+ if (numberOfPartitions = = 0 ) {
182
181
createNonPartitionedTopic (topic );
183
182
} else {
184
183
createPartitionedTopic (topic , numberOfPartitions );
@@ -196,7 +195,7 @@ public void increaseTopicPartitions(String topic, int newPartitionsNum) {
196
195
sneakyAdmin (() -> admin ().topics ().getPartitionedTopicMetadata (topic ));
197
196
checkArgument (
198
197
metadata .partitions < newPartitionsNum ,
199
- "The new partition size which should exceed previous size." );
198
+ "The new partition size which should greater than previous size." );
200
199
201
200
sneakyAdmin (() -> admin ().topics ().updatePartitionedTopic (topic , newPartitionsNum ));
202
201
}
@@ -220,9 +219,11 @@ public void deleteTopic(String topic) {
220
219
return ;
221
220
}
222
221
222
+ // Close all the available consumers and producers.
223
223
removeConsumers (topic );
224
224
removeProducers (topic );
225
- if (metadata .partitions <= 0 ) {
225
+
226
+ if (metadata .partitions == NON_PARTITIONED ) {
226
227
sneakyAdmin (() -> admin ().topics ().delete (topicName ));
227
228
} else {
228
229
sneakyAdmin (() -> admin ().topics ().deletePartitionedTopic (topicName ));
@@ -245,22 +246,6 @@ public List<TopicPartition> topicInfo(String topic) {
245
246
}
246
247
}
247
248
248
- /**
249
- * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a
250
- * mapping for topic and its partitions.
251
- */
252
- public Map <String , List <TopicPartition >> topicsInfo (String ... topics ) {
253
- return topicsInfo (Arrays .asList (topics ));
254
- }
255
-
256
- /**
257
- * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a
258
- * mapping for topic and its partitions.
259
- */
260
- public Map <String , List <TopicPartition >> topicsInfo (Collection <String > topics ) {
261
- return topics .stream ().collect (toMap (identity (), this ::topicInfo ));
262
- }
263
-
264
249
/**
265
250
* Send a single message to Pulsar, return the message id after the ack from Pulsar.
266
251
*
@@ -518,12 +503,13 @@ private <T> Producer<T> createProducer(String topic, Schema<T> schema)
518
503
topicProducers .computeIfAbsent (
519
504
index ,
520
505
i -> {
521
- try {
522
- return client ().newProducer (schema ).topic (topic ).create ();
523
- } catch (PulsarClientException e ) {
524
- sneakyThrow (e );
525
- return null ;
526
- }
506
+ ProducerBuilder <T > builder =
507
+ client ().newProducer (schema )
508
+ .topic (topic )
509
+ .enableBatching (false )
510
+ .enableMultiSchema (true );
511
+
512
+ return sneakyClient (builder ::create );
527
513
});
528
514
}
529
515
@@ -540,19 +526,15 @@ private <T> Consumer<T> createConsumer(String topic, Schema<T> schema)
540
526
topicConsumers .computeIfAbsent (
541
527
index ,
542
528
i -> {
543
- try {
544
- return client ().newConsumer (schema )
545
- .topic (topic )
546
- .subscriptionName (SUBSCRIPTION_NAME )
547
- .subscriptionMode (Durable )
548
- .subscriptionType (Exclusive )
549
- .subscriptionInitialPosition (
550
- SubscriptionInitialPosition .Earliest )
551
- .subscribe ();
552
- } catch (PulsarClientException e ) {
553
- sneakyThrow (e );
554
- return null ;
555
- }
529
+ ConsumerBuilder <T > builder =
530
+ client ().newConsumer (schema )
531
+ .topic (topic )
532
+ .subscriptionName (SUBSCRIPTION_NAME )
533
+ .subscriptionMode (Durable )
534
+ .subscriptionType (Exclusive )
535
+ .subscriptionInitialPosition (Earliest );
536
+
537
+ return sneakyClient (builder ::subscribe );
556
538
});
557
539
}
558
540
@@ -561,11 +543,7 @@ private void removeProducers(String topic) {
561
543
ConcurrentHashMap <Integer , Producer <?>> integerProducers = producers .remove (topicName );
562
544
if (integerProducers != null ) {
563
545
for (Producer <?> producer : integerProducers .values ()) {
564
- try {
565
- producer .close ();
566
- } catch (PulsarClientException e ) {
567
- sneakyThrow (e );
568
- }
546
+ sneakyClient (producer ::close );
569
547
}
570
548
}
571
549
}
@@ -575,11 +553,7 @@ private void removeConsumers(String topic) {
575
553
ConcurrentHashMap <Integer , Consumer <?>> integerConsumers = consumers .remove (topicName );
576
554
if (integerConsumers != null ) {
577
555
for (Consumer <?> consumer : integerConsumers .values ()) {
578
- try {
579
- consumer .close ();
580
- } catch (PulsarClientException e ) {
581
- sneakyThrow (e );
582
- }
556
+ sneakyClient (consumer ::close );
583
557
}
584
558
}
585
559
}
0 commit comments