/
Commands.java
1765 lines (1532 loc) · 75.5 KB
/
Commands.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.common.protocol;
import static com.scurrilous.circe.checksum.Crc32cIntChecksum.computeChecksum;
import static com.scurrilous.circe.checksum.Crc32cIntChecksum.resumeChecksum;
import static java.nio.charset.StandardCharsets.UTF_8;
import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.CompositeByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.util.concurrent.FastThreadLocal;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import lombok.experimental.UtilityClass;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Triple;
import org.apache.pulsar.PulsarVersion;
import org.apache.pulsar.client.api.KeySharedPolicy;
import org.apache.pulsar.client.api.ProducerAccessMode;
import org.apache.pulsar.client.api.Range;
import org.apache.pulsar.client.api.transaction.TxnID;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.AuthData;
import org.apache.pulsar.common.intercept.BrokerEntryMetadataInterceptor;
import org.apache.pulsar.common.api.proto.AuthMethod;
import org.apache.pulsar.common.api.proto.BaseCommand;
import org.apache.pulsar.common.api.proto.BaseCommand.Type;
import org.apache.pulsar.common.api.proto.BrokerEntryMetadata;
import org.apache.pulsar.common.api.proto.CommandAck;
import org.apache.pulsar.common.api.proto.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.CommandAck.ValidationError;
import org.apache.pulsar.common.api.proto.CommandAckResponse;
import org.apache.pulsar.common.api.proto.CommandAddPartitionToTxn;
import org.apache.pulsar.common.api.proto.CommandAddSubscriptionToTxn;
import org.apache.pulsar.common.api.proto.CommandAddSubscriptionToTxnResponse;
import org.apache.pulsar.common.api.proto.CommandAuthChallenge;
import org.apache.pulsar.common.api.proto.CommandConnect;
import org.apache.pulsar.common.api.proto.CommandConnected;
import org.apache.pulsar.common.api.proto.CommandEndTxnOnPartitionResponse;
import org.apache.pulsar.common.api.proto.CommandEndTxnOnSubscriptionResponse;
import org.apache.pulsar.common.api.proto.CommandEndTxnResponse;
import org.apache.pulsar.common.api.proto.CommandGetLastMessageIdResponse;
import org.apache.pulsar.common.api.proto.CommandGetSchema;
import org.apache.pulsar.common.api.proto.CommandGetSchemaResponse;
import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace;
import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode;
import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespaceResponse;
import org.apache.pulsar.common.api.proto.CommandLookupTopic;
import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse;
import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType;
import org.apache.pulsar.common.api.proto.CommandMessage;
import org.apache.pulsar.common.api.proto.CommandNewTxnResponse;
import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadataResponse;
import org.apache.pulsar.common.api.proto.CommandProducer;
import org.apache.pulsar.common.api.proto.CommandProducerSuccess;
import org.apache.pulsar.common.api.proto.CommandRedeliverUnacknowledgedMessages;
import org.apache.pulsar.common.api.proto.CommandSeek;
import org.apache.pulsar.common.api.proto.CommandSend;
import org.apache.pulsar.common.api.proto.CommandSubscribe;
import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.FeatureFlags;
import org.apache.pulsar.common.api.proto.IntRange;
import org.apache.pulsar.common.api.proto.KeySharedMeta;
import org.apache.pulsar.common.api.proto.KeySharedMode;
import org.apache.pulsar.common.api.proto.MessageIdData;
import org.apache.pulsar.common.api.proto.MessageMetadata;
import org.apache.pulsar.common.api.proto.ProtocolVersion;
import org.apache.pulsar.common.api.proto.Schema;
import org.apache.pulsar.common.api.proto.ServerError;
import org.apache.pulsar.common.api.proto.SingleMessageMetadata;
import org.apache.pulsar.common.api.proto.Subscription;
import org.apache.pulsar.common.api.proto.TxnAction;
import org.apache.pulsar.common.protocol.schema.SchemaVersion;
import org.apache.pulsar.common.schema.SchemaInfo;
import org.apache.pulsar.common.schema.SchemaType;
import org.apache.pulsar.common.util.collections.BitSetRecyclable;
import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable;
@UtilityClass
@Slf4j
@SuppressWarnings("checkstyle:JavadocType")
public class Commands {
// default message size for transfer
public static final int DEFAULT_MAX_MESSAGE_SIZE = 5 * 1024 * 1024;
public static final int MESSAGE_SIZE_FRAME_PADDING = 10 * 1024;
public static final int INVALID_MAX_MESSAGE_SIZE = -1;
@SuppressWarnings("checkstyle:ConstantName")
public static final short magicCrc32c = 0x0e01;
@SuppressWarnings("checkstyle:ConstantName")
public static final short magicBrokerEntryMetadata = 0x0e02;
private static final int checksumSize = 4;
private static final FastThreadLocal<BaseCommand> LOCAL_BASE_COMMAND = new FastThreadLocal<BaseCommand>() {
@Override
protected BaseCommand initialValue() throws Exception {
return new BaseCommand();
}
};
private static BaseCommand localCmd(BaseCommand.Type type) {
return LOCAL_BASE_COMMAND.get()
.clear()
.setType(type);
}
private static final FastThreadLocal<SingleMessageMetadata> LOCAL_SINGLE_MESSAGE_METADATA = //
new FastThreadLocal<SingleMessageMetadata>() {
@Override
protected SingleMessageMetadata initialValue() throws Exception {
return new SingleMessageMetadata();
}
};
private static final FastThreadLocal<MessageMetadata> LOCAL_MESSAGE_METADATA = //
new FastThreadLocal<MessageMetadata>() {
@Override
protected MessageMetadata initialValue() throws Exception {
return new MessageMetadata();
}
};
private static final FastThreadLocal<BrokerEntryMetadata> BROKER_ENTRY_METADATA = //
new FastThreadLocal<BrokerEntryMetadata>() {
@Override
protected BrokerEntryMetadata initialValue() throws Exception {
return new BrokerEntryMetadata();
}
};
public static ByteBuf newConnect(String authMethodName, String authData, String libVersion) {
return newConnect(authMethodName, authData, getCurrentProtocolVersion(), libVersion, null /* target broker */,
null /* originalPrincipal */, null /* Client Auth Data */, null /* Client Auth Method */);
}
public static ByteBuf newConnect(String authMethodName, String authData, String libVersion, String targetBroker) {
return newConnect(authMethodName, authData, getCurrentProtocolVersion(), libVersion, targetBroker, null, null,
null);
}
public static ByteBuf newConnect(String authMethodName, String authData, String libVersion, String targetBroker,
String originalPrincipal, String clientAuthData, String clientAuthMethod) {
return newConnect(authMethodName, authData, getCurrentProtocolVersion(), libVersion, targetBroker,
originalPrincipal, clientAuthData, clientAuthMethod);
}
private static void setFeatureFlags(FeatureFlags flags) {
flags.setSupportsAuthRefresh(true);
flags.setSupportsBrokerEntryMetadata(true);
}
public static ByteBuf newConnect(String authMethodName, String authData, int protocolVersion, String libVersion,
String targetBroker, String originalPrincipal, String originalAuthData,
String originalAuthMethod) {
BaseCommand cmd = localCmd(Type.CONNECT);
CommandConnect connect = cmd.setConnect()
.setClientVersion(libVersion != null ? libVersion : "Pulsar Client")
.setAuthMethodName(authMethodName);
if ("ycav1".equals(authMethodName)) {
// Handle the case of a client that gets updated before the broker and starts sending the string auth method
// name. An example would be in broker-to-broker replication. We need to make sure the clients are still
// passing both the enum and the string until all brokers are upgraded.
connect.setAuthMethod(AuthMethod.AuthMethodYcaV1);
}
if (targetBroker != null) {
// When connecting through a proxy, we need to specify which broker do we want to be proxied through
connect.setProxyToBrokerUrl(targetBroker);
}
if (authData != null) {
connect.setAuthData(authData.getBytes(UTF_8));
}
if (originalPrincipal != null) {
connect.setOriginalPrincipal(originalPrincipal);
}
if (originalAuthData != null) {
connect.setOriginalAuthData(originalAuthData);
}
if (originalAuthMethod != null) {
connect.setOriginalAuthMethod(originalAuthMethod);
}
connect.setProtocolVersion(protocolVersion);
setFeatureFlags(connect.setFeatureFlags());
return serializeWithSize(cmd);
}
public static ByteBuf newConnect(String authMethodName, AuthData authData, int protocolVersion, String libVersion,
String targetBroker, String originalPrincipal, AuthData originalAuthData,
String originalAuthMethod) {
BaseCommand cmd = localCmd(Type.CONNECT);
CommandConnect connect = cmd.setConnect()
.setClientVersion(libVersion != null ? libVersion : "Pulsar Client")
.setAuthMethodName(authMethodName);
if (targetBroker != null) {
// When connecting through a proxy, we need to specify which broker do we want to be proxied through
connect.setProxyToBrokerUrl(targetBroker);
}
if (authData != null) {
connect.setAuthData(authData.getBytes());
}
if (originalPrincipal != null) {
connect.setOriginalPrincipal(originalPrincipal);
}
if (originalAuthData != null) {
connect.setOriginalAuthData(new String(originalAuthData.getBytes(), UTF_8));
}
if (originalAuthMethod != null) {
connect.setOriginalAuthMethod(originalAuthMethod);
}
connect.setProtocolVersion(protocolVersion);
setFeatureFlags(connect.setFeatureFlags());
return serializeWithSize(cmd);
}
public static ByteBuf newConnected(int clientProtocoVersion) {
return newConnected(clientProtocoVersion, INVALID_MAX_MESSAGE_SIZE);
}
public static BaseCommand newConnectedCommand(int clientProtocolVersion, int maxMessageSize) {
BaseCommand cmd = localCmd(Type.CONNECTED);
CommandConnected connected = cmd.setConnected()
.setServerVersion("Pulsar Server" + PulsarVersion.getVersion());
if (INVALID_MAX_MESSAGE_SIZE != maxMessageSize) {
connected.setMaxMessageSize(maxMessageSize);
}
// If the broker supports a newer version of the protocol, it will anyway advertise the max version that the
// client supports, to avoid confusing the client.
int currentProtocolVersion = getCurrentProtocolVersion();
int versionToAdvertise = Math.min(currentProtocolVersion, clientProtocolVersion);
connected.setProtocolVersion(versionToAdvertise);
return cmd;
}
public static ByteBuf newConnected(int clientProtocolVersion, int maxMessageSize) {
return serializeWithSize(newConnectedCommand(clientProtocolVersion, maxMessageSize));
}
public static ByteBuf newAuthChallenge(String authMethod, AuthData brokerData, int clientProtocolVersion) {
BaseCommand cmd = localCmd(Type.AUTH_CHALLENGE);
CommandAuthChallenge challenge = cmd.setAuthChallenge();
// If the broker supports a newer version of the protocol, it will anyway advertise the max version that the
// client supports, to avoid confusing the client.
int currentProtocolVersion = getCurrentProtocolVersion();
int versionToAdvertise = Math.min(currentProtocolVersion, clientProtocolVersion);
challenge.setProtocolVersion(versionToAdvertise)
.setChallenge()
.setAuthData(brokerData != null ? brokerData.getBytes() : new byte[0])
.setAuthMethodName(authMethod);
return serializeWithSize(cmd);
}
public static ByteBuf newAuthResponse(String authMethod,
AuthData clientData,
int clientProtocolVersion,
String clientVersion) {
BaseCommand cmd = localCmd(Type.AUTH_RESPONSE);
cmd.setAuthResponse()
.setClientVersion(clientVersion != null ? clientVersion : "Pulsar Client")
.setProtocolVersion(clientProtocolVersion)
.setResponse()
.setAuthData(clientData.getBytes())
.setAuthMethodName(authMethod);
return serializeWithSize(cmd);
}
public static BaseCommand newSuccessCommand(long requestId) {
BaseCommand cmd = localCmd(Type.SUCCESS);
cmd.setSuccess()
.setRequestId(requestId);
return cmd;
}
public static ByteBuf newSuccess(long requestId) {
return serializeWithSize(newSuccessCommand(requestId));
}
public static BaseCommand newProducerSuccessCommand(long requestId, String producerName,
SchemaVersion schemaVersion) {
return newProducerSuccessCommand(requestId, producerName, -1, schemaVersion, Optional.empty(), true);
}
public static ByteBuf newProducerSuccess(long requestId, String producerName, SchemaVersion schemaVersion) {
return newProducerSuccess(requestId, producerName, -1, schemaVersion, Optional.empty(), true);
}
public static BaseCommand newProducerSuccessCommand(long requestId, String producerName, long lastSequenceId,
SchemaVersion schemaVersion, Optional<Long> topicEpoch, boolean isProducerReady) {
BaseCommand cmd = localCmd(Type.PRODUCER_SUCCESS);
CommandProducerSuccess ps = cmd.setProducerSuccess()
.setRequestId(requestId)
.setProducerName(producerName)
.setLastSequenceId(lastSequenceId)
.setSchemaVersion(schemaVersion.bytes())
.setProducerReady(isProducerReady);
topicEpoch.ifPresent(ps::setTopicEpoch);
return cmd;
}
public static ByteBuf newProducerSuccess(long requestId, String producerName, long lastSequenceId,
SchemaVersion schemaVersion, Optional<Long> topicEpoch, boolean isProducerReady) {
return serializeWithSize(newProducerSuccessCommand(requestId, producerName, lastSequenceId, schemaVersion,
topicEpoch, isProducerReady));
}
public static BaseCommand newErrorCommand(long requestId, ServerError serverError, String message) {
BaseCommand cmd = localCmd(Type.ERROR);
cmd.setError()
.setRequestId(requestId)
.setError(serverError)
.setMessage(message);
return cmd;
}
public static ByteBuf newError(long requestId, ServerError serverError, String message) {
return serializeWithSize(newErrorCommand(requestId, serverError, message));
}
public static BaseCommand newSendReceiptCommand(long producerId, long sequenceId, long highestId, long ledgerId,
long entryId) {
BaseCommand cmd = localCmd(Type.SEND_RECEIPT);
cmd.setSendReceipt()
.setProducerId(producerId)
.setSequenceId(sequenceId)
.setHighestSequenceId(highestId)
.setMessageId()
.setLedgerId(ledgerId)
.setEntryId(entryId);
return cmd;
}
public static ByteBuf newSendReceipt(long producerId, long sequenceId, long highestId, long ledgerId,
long entryId) {
return serializeWithSize(newSendReceiptCommand(producerId, sequenceId, highestId, ledgerId, entryId));
}
public static BaseCommand newSendErrorCommand(long producerId, long sequenceId, ServerError error,
String errorMsg) {
BaseCommand cmd = localCmd(Type.SEND_ERROR);
cmd.setSendError()
.setProducerId(producerId)
.setSequenceId(sequenceId)
.setError(error)
.setMessage(errorMsg);
return cmd;
}
public static ByteBuf newSendError(long producerId, long sequenceId, ServerError error, String errorMsg) {
return serializeWithSize(newSendErrorCommand(producerId, sequenceId, error, errorMsg));
}
public static boolean hasChecksum(ByteBuf buffer) {
return buffer.getShort(buffer.readerIndex()) == magicCrc32c;
}
/**
* Read the checksum and advance the reader index in the buffer.
*
* <p>Note: This method assume the checksum presence was already verified before.
*/
public static int readChecksum(ByteBuf buffer) {
buffer.skipBytes(2); //skip magic bytes
return buffer.readInt();
}
public static void skipChecksumIfPresent(ByteBuf buffer) {
if (hasChecksum(buffer)) {
readChecksum(buffer);
}
}
public static MessageMetadata parseMessageMetadata(ByteBuf buffer) {
MessageMetadata md = LOCAL_MESSAGE_METADATA.get();
parseMessageMetadata(buffer, md);
return md;
}
public static void parseMessageMetadata(ByteBuf buffer, MessageMetadata msgMetadata) {
// initially reader-index may point to start_of_checksum : increment reader-index to start_of_metadata
// to parse metadata
skipChecksumIfPresent(buffer);
int metadataSize = (int) buffer.readUnsignedInt();
msgMetadata.parseFrom(buffer, metadataSize);
}
public static void skipMessageMetadata(ByteBuf buffer) {
// initially reader-index may point to start_of_checksum : increment reader-index to start_of_metadata to parse
// metadata
skipBrokerEntryMetadataIfExist(buffer);
skipChecksumIfPresent(buffer);
int metadataSize = (int) buffer.readUnsignedInt();
buffer.skipBytes(metadataSize);
}
public static BaseCommand newMessageCommand(long consumerId, long ledgerId, long entryId, int partition,
int redeliveryCount, long[] ackSet) {
BaseCommand cmd = localCmd(Type.MESSAGE);
CommandMessage msg = cmd.setMessage()
.setConsumerId(consumerId);
msg.setMessageId()
.setLedgerId(ledgerId)
.setEntryId(entryId)
.setPartition(partition);
if (redeliveryCount > 0) {
msg.setRedeliveryCount(redeliveryCount);
}
if (ackSet != null) {
for (int i = 0; i < ackSet.length; i++) {
msg.addAckSet(ackSet[i]);
}
}
return cmd;
}
public static ByteBufPair newMessage(long consumerId, long ledgerId, long entryId, int partition,
int redeliveryCount, ByteBuf metadataAndPayload, long[] ackSet) {
return serializeCommandMessageWithSize(
newMessageCommand(consumerId, ledgerId, entryId, partition, redeliveryCount, ackSet),
metadataAndPayload);
}
public static ByteBufPair newSend(long producerId, long sequenceId, int numMessaegs, ChecksumType checksumType,
MessageMetadata messageMetadata, ByteBuf payload) {
return newSend(producerId, sequenceId, -1 /* highestSequenceId */, numMessaegs,
messageMetadata.hasTxnidLeastBits() ? messageMetadata.getTxnidLeastBits() : -1,
messageMetadata.hasTxnidMostBits() ? messageMetadata.getTxnidMostBits() : -1,
checksumType, messageMetadata, payload);
}
public static ByteBufPair newSend(long producerId, long lowestSequenceId, long highestSequenceId, int numMessaegs,
ChecksumType checksumType, MessageMetadata messageMetadata, ByteBuf payload) {
return newSend(producerId, lowestSequenceId, highestSequenceId, numMessaegs,
messageMetadata.hasTxnidLeastBits() ? messageMetadata.getTxnidLeastBits() : -1,
messageMetadata.hasTxnidMostBits() ? messageMetadata.getTxnidMostBits() : -1,
checksumType, messageMetadata, payload);
}
public static ByteBufPair newSend(long producerId, long sequenceId, long highestSequenceId, int numMessages,
long txnIdLeastBits, long txnIdMostBits, ChecksumType checksumType,
MessageMetadata messageData, ByteBuf payload) {
BaseCommand cmd = localCmd(Type.SEND);
CommandSend send = cmd.setSend()
.setProducerId(producerId)
.setSequenceId(sequenceId);
if (highestSequenceId >= 0) {
send.setHighestSequenceId(highestSequenceId);
}
if (numMessages > 1) {
send.setNumMessages(numMessages);
}
if (txnIdLeastBits >= 0) {
send.setTxnidLeastBits(txnIdLeastBits);
}
if (txnIdMostBits >= 0) {
send.setTxnidMostBits(txnIdMostBits);
}
if (messageData.hasTotalChunkMsgSize() && messageData.getTotalChunkMsgSize() > 1) {
send.setIsChunk(true);
}
return serializeCommandSendWithSize(cmd, checksumType, messageData, payload);
}
public static ByteBuf newSubscribe(String topic, String subscription, long consumerId, long requestId,
SubType subType, int priorityLevel, String consumerName, long resetStartMessageBackInSeconds) {
return newSubscribe(topic, subscription, consumerId, requestId, subType, priorityLevel, consumerName,
true /* isDurable */, null /* startMessageId */, Collections.emptyMap(), false,
false /* isReplicated */, InitialPosition.Earliest, resetStartMessageBackInSeconds, null,
true /* createTopicIfDoesNotExist */);
}
public static ByteBuf newSubscribe(String topic, String subscription, long consumerId, long requestId,
SubType subType, int priorityLevel, String consumerName, boolean isDurable, MessageIdData startMessageId,
Map<String, String> metadata, boolean readCompacted, boolean isReplicated,
InitialPosition subscriptionInitialPosition, long startMessageRollbackDurationInSec, SchemaInfo schemaInfo,
boolean createTopicIfDoesNotExist) {
return newSubscribe(topic, subscription, consumerId, requestId, subType, priorityLevel, consumerName,
isDurable, startMessageId, metadata, readCompacted, isReplicated, subscriptionInitialPosition,
startMessageRollbackDurationInSec, schemaInfo, createTopicIfDoesNotExist, null);
}
public static ByteBuf newSubscribe(String topic, String subscription, long consumerId, long requestId,
SubType subType, int priorityLevel, String consumerName, boolean isDurable, MessageIdData startMessageId,
Map<String, String> metadata, boolean readCompacted, boolean isReplicated,
InitialPosition subscriptionInitialPosition, long startMessageRollbackDurationInSec,
SchemaInfo schemaInfo, boolean createTopicIfDoesNotExist, KeySharedPolicy keySharedPolicy) {
BaseCommand cmd = localCmd(Type.SUBSCRIBE);
CommandSubscribe subscribe = cmd.setSubscribe()
.setTopic(topic)
.setSubscription(subscription)
.setSubType(subType)
.setConsumerId(consumerId)
.setConsumerName(consumerName)
.setRequestId(requestId)
.setPriorityLevel(priorityLevel)
.setDurable(isDurable)
.setReadCompacted(readCompacted)
.setInitialPosition(subscriptionInitialPosition)
.setReplicateSubscriptionState(isReplicated)
.setForceTopicCreation(createTopicIfDoesNotExist);
if (keySharedPolicy != null) {
KeySharedMeta keySharedMeta = subscribe.setKeySharedMeta();
keySharedMeta.setAllowOutOfOrderDelivery(keySharedPolicy.isAllowOutOfOrderDelivery());
keySharedMeta.setKeySharedMode(convertKeySharedMode(keySharedPolicy.getKeySharedMode()));
if (keySharedPolicy instanceof KeySharedPolicy.KeySharedPolicySticky) {
List<Range> ranges = ((KeySharedPolicy.KeySharedPolicySticky) keySharedPolicy)
.getRanges();
for (Range range : ranges) {
IntRange r = keySharedMeta.addHashRange();
r.setStart(range.getStart());
r.setEnd(range.getEnd());
}
}
}
if (startMessageId != null) {
subscribe.setStartMessageId().copyFrom(startMessageId);
}
if (startMessageRollbackDurationInSec > 0) {
subscribe.setStartMessageRollbackDurationSec(startMessageRollbackDurationInSec);
}
if (!metadata.isEmpty()) {
metadata.entrySet().forEach(e -> subscribe.addMetadata()
.setKey(e.getKey())
.setValue(e.getValue()));
}
if (schemaInfo != null) {
if (subscribe.hasSchema()) {
throw new IllegalStateException();
}
if (subscribe.setSchema().getPropertiesCount() > 0) {
throw new IllegalStateException();
}
convertSchema(schemaInfo, subscribe.setSchema());
}
return serializeWithSize(cmd);
}
private static KeySharedMode convertKeySharedMode(org.apache.pulsar.client.api.KeySharedMode mode) {
switch (mode) {
case AUTO_SPLIT: return KeySharedMode.AUTO_SPLIT;
case STICKY: return KeySharedMode.STICKY;
default:
throw new IllegalArgumentException("Unexpected key shared mode: " + mode);
}
}
public static ByteBuf newUnsubscribe(long consumerId, long requestId) {
BaseCommand cmd = localCmd(Type.UNSUBSCRIBE);
cmd.setUnsubscribe()
.setConsumerId(consumerId)
.setRequestId(requestId);
return serializeWithSize(cmd);
}
public static ByteBuf newActiveConsumerChange(long consumerId, boolean isActive) {
BaseCommand cmd = localCmd(Type.ACTIVE_CONSUMER_CHANGE);
cmd.setActiveConsumerChange()
.setConsumerId(consumerId)
.setIsActive(isActive);
return serializeWithSize(cmd);
}
public static ByteBuf newSeek(long consumerId, long requestId,
long ledgerId, long entryId, long[] ackSet) {
BaseCommand cmd = localCmd(Type.SEEK);
CommandSeek seek = cmd.setSeek()
.setConsumerId(consumerId)
.setRequestId(requestId);
MessageIdData messageId = seek.setMessageId()
.setLedgerId(ledgerId)
.setEntryId(entryId);
for (int i = 0; i < ackSet.length; i++) {
messageId.addAckSet(ackSet[i]);
}
return serializeWithSize(cmd);
}
public static ByteBuf newSeek(long consumerId, long requestId, long timestamp) {
BaseCommand cmd = localCmd(Type.SEEK);
cmd.setSeek()
.setConsumerId(consumerId)
.setRequestId(requestId)
.setMessagePublishTime(timestamp);
return serializeWithSize(cmd);
}
public static ByteBuf newCloseConsumer(long consumerId, long requestId) {
BaseCommand cmd = localCmd(Type.CLOSE_CONSUMER);
cmd.setCloseConsumer()
.setConsumerId(consumerId)
.setRequestId(requestId);
return serializeWithSize(cmd);
}
public static ByteBuf newReachedEndOfTopic(long consumerId) {
BaseCommand cmd = localCmd(Type.REACHED_END_OF_TOPIC);
cmd.setReachedEndOfTopic()
.setConsumerId(consumerId);
return serializeWithSize(cmd);
}
public static ByteBuf newCloseProducer(long producerId, long requestId) {
BaseCommand cmd = localCmd(Type.CLOSE_PRODUCER);
cmd.setCloseProducer()
.setProducerId(producerId)
.setRequestId(requestId);
return serializeWithSize(cmd);
}
@VisibleForTesting
public static ByteBuf newProducer(String topic, long producerId, long requestId, String producerName,
Map<String, String> metadata) {
return newProducer(topic, producerId, requestId, producerName, false, metadata);
}
public static ByteBuf newProducer(String topic, long producerId, long requestId, String producerName,
boolean encrypted, Map<String, String> metadata) {
return newProducer(topic, producerId, requestId, producerName, encrypted, metadata, null, 0, false,
ProducerAccessMode.Shared, Optional.empty());
}
private static Schema.Type getSchemaType(SchemaType type) {
if (type.getValue() < 0) {
return Schema.Type.None;
} else {
return Schema.Type.valueOf(type.getValue());
}
}
public static SchemaType getSchemaType(Schema.Type type) {
if (type.getValue() < 0) {
// this is unexpected
return SchemaType.NONE;
} else {
return SchemaType.valueOf(type.getValue());
}
}
private static void convertSchema(SchemaInfo schemaInfo, Schema schema) {
schema.setName(schemaInfo.getName())
.setSchemaData(schemaInfo.getSchema())
.setType(getSchemaType(schemaInfo.getType()));
schemaInfo.getProperties().entrySet().stream().forEach(entry -> {
if (entry.getKey() != null && entry.getValue() != null) {
schema.addProperty()
.setKey(entry.getKey())
.setValue(entry.getValue());
}
});
}
public static ByteBuf newProducer(String topic, long producerId, long requestId, String producerName,
boolean encrypted, Map<String, String> metadata, SchemaInfo schemaInfo,
long epoch, boolean userProvidedProducerName,
ProducerAccessMode accessMode, Optional<Long> topicEpoch) {
BaseCommand cmd = localCmd(Type.PRODUCER);
CommandProducer producer = cmd.setProducer()
.setTopic(topic)
.setProducerId(producerId)
.setRequestId(requestId)
.setEpoch(epoch)
.setUserProvidedProducerName(userProvidedProducerName)
.setEncrypted(encrypted)
.setProducerAccessMode(convertProducerAccessMode(accessMode));
if (producerName != null) {
producer.setProducerName(producerName);
}
if (!metadata.isEmpty()) {
metadata.forEach((k, v) -> producer.addMetadata()
.setKey(k)
.setValue(v));
}
if (null != schemaInfo) {
convertSchema(schemaInfo, producer.setSchema());
}
topicEpoch.ifPresent(producer::setTopicEpoch);
return serializeWithSize(cmd);
}
public static BaseCommand newPartitionMetadataResponseCommand(ServerError error, String errorMsg, long requestId) {
BaseCommand cmd = localCmd(Type.PARTITIONED_METADATA_RESPONSE);
CommandPartitionedTopicMetadataResponse response = cmd.setPartitionMetadataResponse()
.setRequestId(requestId)
.setError(error)
.setResponse(CommandPartitionedTopicMetadataResponse.LookupType.Failed);
if (errorMsg != null) {
response.setMessage(errorMsg);
}
return cmd;
}
public static ByteBuf newPartitionMetadataResponse(ServerError error, String errorMsg, long requestId) {
return serializeWithSize(newPartitionMetadataResponseCommand(error, errorMsg, requestId));
}
public static ByteBuf newPartitionMetadataRequest(String topic, long requestId) {
BaseCommand cmd = localCmd(Type.PARTITIONED_METADATA);
cmd.setPartitionMetadata()
.setTopic(topic)
.setRequestId(requestId);
return serializeWithSize(cmd);
}
public static BaseCommand newPartitionMetadataResponseCommand(int partitions, long requestId) {
BaseCommand cmd = localCmd(Type.PARTITIONED_METADATA_RESPONSE);
cmd.setPartitionMetadataResponse()
.setPartitions(partitions)
.setResponse(CommandPartitionedTopicMetadataResponse.LookupType.Success)
.setRequestId(requestId);
return cmd;
}
public static ByteBuf newPartitionMetadataResponse(int partitions, long requestId) {
return serializeWithSize(newPartitionMetadataResponseCommand(partitions, requestId));
}
public static ByteBuf newLookup(String topic, boolean authoritative, long requestId) {
return newLookup(topic, null, authoritative, requestId);
}
public static ByteBuf newLookup(String topic, String listenerName, boolean authoritative, long requestId) {
BaseCommand cmd = localCmd(Type.LOOKUP);
CommandLookupTopic lookup = cmd.setLookupTopic()
.setTopic(topic)
.setRequestId(requestId)
.setAuthoritative(authoritative);
if (StringUtils.isNotBlank(listenerName)) {
lookup.setAdvertisedListenerName(listenerName);
}
return serializeWithSize(cmd);
}
public static BaseCommand newLookupResponseCommand(String brokerServiceUrl, String brokerServiceUrlTls,
boolean authoritative, LookupType lookupType, long requestId, boolean proxyThroughServiceUrl) {
BaseCommand cmd = localCmd(Type.LOOKUP_RESPONSE);
CommandLookupTopicResponse response = cmd.setLookupTopicResponse()
.setBrokerServiceUrl(brokerServiceUrl)
.setResponse(lookupType)
.setRequestId(requestId)
.setAuthoritative(authoritative)
.setProxyThroughServiceUrl(proxyThroughServiceUrl);
if (brokerServiceUrlTls != null) {
response.setBrokerServiceUrlTls(brokerServiceUrlTls);
}
return cmd;
}
public static ByteBuf newLookupResponse(String brokerServiceUrl, String brokerServiceUrlTls, boolean authoritative,
LookupType lookupType, long requestId, boolean proxyThroughServiceUrl) {
return serializeWithSize(newLookupResponseCommand(brokerServiceUrl, brokerServiceUrlTls, authoritative,
lookupType, requestId, proxyThroughServiceUrl));
}
public static BaseCommand newLookupErrorResponseCommand(ServerError error, String errorMsg, long requestId) {
BaseCommand cmd = localCmd(Type.LOOKUP_RESPONSE);
CommandLookupTopicResponse response = cmd.setLookupTopicResponse()
.setRequestId(requestId)
.setError(error)
.setResponse(LookupType.Failed);
if (errorMsg != null) {
response.setMessage(errorMsg);
}
return cmd;
}
public static ByteBuf newLookupErrorResponse(ServerError error, String errorMsg, long requestId) {
return serializeWithSize(newLookupErrorResponseCommand(error, errorMsg, requestId));
}
public static ByteBuf newMultiTransactionMessageAck(long consumerId, TxnID txnID,
List<Triple<Long, Long, ConcurrentBitSetRecyclable>> entries) {
BaseCommand cmd = newMultiMessageAckCommon(entries);
cmd.getAck()
.setConsumerId(consumerId)
.setAckType(AckType.Individual)
.setTxnidLeastBits(txnID.getLeastSigBits())
.setTxnidMostBits(txnID.getMostSigBits());
return serializeWithSize(cmd);
}
private static BaseCommand newMultiMessageAckCommon(List<Triple<Long, Long, ConcurrentBitSetRecyclable>> entries) {
BaseCommand cmd = localCmd(Type.ACK);
CommandAck ack = cmd.setAck();
int entriesCount = entries.size();
for (int i = 0; i < entriesCount; i++) {
long ledgerId = entries.get(i).getLeft();
long entryId = entries.get(i).getMiddle();
ConcurrentBitSetRecyclable bitSet = entries.get(i).getRight();
MessageIdData msgId = ack.addMessageId()
.setLedgerId(ledgerId)
.setEntryId(entryId);
if (bitSet != null) {
long[] ackSet = bitSet.toLongArray();
for (int j = 0; j < ackSet.length; j++) {
msgId.addAckSet(ackSet[j]);
}
bitSet.recycle();
}
}
return cmd;
}
public static ByteBuf newMultiMessageAck(long consumerId,
List<Triple<Long, Long, ConcurrentBitSetRecyclable>> entries,
long requestId) {
BaseCommand cmd = newMultiMessageAckCommon(entries);
cmd.getAck()
.setConsumerId(consumerId)
.setAckType(AckType.Individual);
if (requestId >= 0) {
cmd.getAck().setRequestId(requestId);
}
return serializeWithSize(cmd);
}
public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType,
ValidationError validationError, Map<String, Long> properties, long requestId) {
return newAck(consumerId, ledgerId, entryId, ackSet, ackType, validationError,
properties, -1L, -1L, requestId, -1);
}
public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType,
ValidationError validationError, Map<String, Long> properties, long txnIdLeastBits,
long txnIdMostBits, long requestId, int batchSize) {
BaseCommand cmd = localCmd(Type.ACK);
CommandAck ack = cmd.setAck()
.setConsumerId(consumerId)
.setAckType(ackType);
MessageIdData messageIdData = ack.addMessageId()
.setLedgerId(ledgerId)
.setEntryId(entryId);
if (ackSet != null) {
long[] as = ackSet.toLongArray();
for (int i = 0; i < as.length; i++) {
messageIdData.addAckSet(as[i]);
}
}
if (batchSize >= 0) {
messageIdData.setBatchSize(batchSize);
}
if (validationError != null) {
ack.setValidationError(validationError);
}
if (txnIdMostBits >= 0) {
ack.setTxnidMostBits(txnIdMostBits);
}
if (txnIdLeastBits >= 0) {
ack.setTxnidLeastBits(txnIdLeastBits);
}
if (requestId >= 0) {
ack.setRequestId(requestId);
}
if (!properties.isEmpty()) {
properties.forEach((k, v) -> {
ack.addProperty().setKey(k).setValue(v);
});
}
return serializeWithSize(cmd);
}
public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType,
ValidationError validationError, Map<String, Long> properties, long txnIdLeastBits,
long txnIdMostBits, long requestId) {
return newAck(consumerId, ledgerId, entryId, ackSet, ackType, validationError,
properties, txnIdLeastBits, txnIdMostBits, requestId, -1);
}
public static ByteBuf newAckResponse(long requestId, ServerError error, String errorMsg, long consumerId) {
BaseCommand cmd = localCmd(Type.ACK_RESPONSE);
CommandAckResponse response = cmd.setAckResponse()
.setConsumerId(consumerId)
.setRequestId(requestId);
if (error != null) {
response.setError(error);
}
if (errorMsg != null) {
response.setMessage(errorMsg);
}
return serializeWithSize(cmd);
}
public static ByteBuf newFlow(long consumerId, int messagePermits) {
BaseCommand cmd = localCmd(Type.FLOW);
cmd.setFlow()
.setConsumerId(consumerId)
.setMessagePermits(messagePermits);
return serializeWithSize(cmd);
}
public static ByteBuf newRedeliverUnacknowledgedMessages(long consumerId) {
BaseCommand cmd = localCmd(Type.REDELIVER_UNACKNOWLEDGED_MESSAGES);
cmd.setRedeliverUnacknowledgedMessages()
.setConsumerId(consumerId);
return serializeWithSize(cmd);
}
public static ByteBuf newRedeliverUnacknowledgedMessages(long consumerId, List<MessageIdData> messageIds) {
BaseCommand cmd = localCmd(Type.REDELIVER_UNACKNOWLEDGED_MESSAGES);
CommandRedeliverUnacknowledgedMessages req = cmd.setRedeliverUnacknowledgedMessages()
.setConsumerId(consumerId);
messageIds.forEach(msgId -> {
MessageIdData m = req.addMessageId()
.setLedgerId(msgId.getLedgerId())
.setEntryId(msgId.getEntryId());
if (msgId.hasBatchIndex()) {
m.setBatchIndex(msgId.getBatchIndex());
}
});
return serializeWithSize(cmd);
}
public static ByteBuf newConsumerStatsResponse(ServerError serverError, String errMsg, long requestId) {
return serializeWithSize(newConsumerStatsResponseCommand(serverError, errMsg, requestId));
}
public static BaseCommand newConsumerStatsResponseCommand(ServerError serverError, String errMsg, long requestId) {
BaseCommand cmd = localCmd(Type.CONSUMER_STATS_RESPONSE);
cmd.setConsumerStatsResponse()
.setRequestId(requestId)
.setErrorCode(serverError);
if (errMsg != null) {
cmd.getConsumerStatsResponse()
.setErrorMessage(errMsg);
}
return cmd;
}
public static ByteBuf newGetTopicsOfNamespaceRequest(String namespace, long requestId, Mode mode) {
BaseCommand cmd = localCmd(Type.GET_TOPICS_OF_NAMESPACE);
CommandGetTopicsOfNamespace topics = cmd.setGetTopicsOfNamespace();
topics.setNamespace(namespace);
topics.setRequestId(requestId);
topics.setMode(mode);
return serializeWithSize(cmd);