diff --git a/pom.xml b/pom.xml
index ca7f482027226..7ae6b750e8328 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1169,7 +1169,7 @@ flexible messaging model and an intuitive client API.
**/*.hgrm
src/main/java/org/apache/bookkeeper/mledger/proto/MLDataFormats.java
src/main/java/org/apache/pulsar/broker/service/schema/proto/SchemaRegistryFormat.java
- src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
+ src/main/java/org/apache/pulsar/common/api/proto/*.java
src/main/java/org/apache/pulsar/io/kinesis/fbs/CompressionType.java
src/main/java/org/apache/pulsar/io/kinesis/fbs/EncryptionCtx.java
src/main/java/org/apache/pulsar/io/kinesis/fbs/EncryptionKey.java
@@ -1258,6 +1258,7 @@ flexible messaging model and an intuitive client API.
and are included in source tree for convenience -->
src/main/java/org/apache/bookkeeper/mledger/proto/MLDataFormats.java
src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
+ src/main/java/org/apache/pulsar/common/api/proto/Markers.java
src/main/java/org/apache/pulsar/broker/service/schema/proto/SchemaRegistryFormat.java
bin/proto/MLDataFormats_pb2.py
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
index b702a5cae6730..06f57530e3596 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java
@@ -27,6 +27,7 @@
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.pulsar.common.api.Commands;
+import org.apache.pulsar.common.api.Markers;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
@@ -72,8 +73,8 @@ public void filterEntriesForConsumer(List entries, EntryBatchSizes batchS
MessageMetadata msgMetadata = Commands.peekMessageMetadata(metadataAndPayload, subscription.toString(), -1);
try {
- if (msgMetadata == null) {
- // Message metadata was corrupted
+ if (msgMetadata == null || Markers.isServerOnlyMarker(msgMetadata)) {
+ // Message metadata was corrupted or the messages was a server-only marker
entries.set(i, null);
entry.release();
subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual,
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
index a8b2e0b24ebf5..9dccf2d7cff4c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java
@@ -46,6 +46,8 @@
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair;
import org.apache.pulsar.broker.authentication.AuthenticationDataSource;
import org.apache.pulsar.common.api.Commands;
+import org.apache.pulsar.common.api.Markers;
+import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition;
diff --git a/pulsar-common/generate_protobuf.sh b/pulsar-common/generate_protobuf.sh
index a1b224600942c..1d2bcb8886ca7 100755
--- a/pulsar-common/generate_protobuf.sh
+++ b/pulsar-common/generate_protobuf.sh
@@ -21,3 +21,4 @@
PROTOC=${PROTOC:-protoc}
${PROTOC} --java_out=src/main/java src/main/proto/PulsarApi.proto
+${PROTOC} --java_out=src/main/java src/main/proto/PulsarMarkers.proto
diff --git a/pulsar-common/generate_protobuf_docker.sh b/pulsar-common/generate_protobuf_docker.sh
index 52b9d664f7548..3a0ca81be950c 100755
--- a/pulsar-common/generate_protobuf_docker.sh
+++ b/pulsar-common/generate_protobuf_docker.sh
@@ -38,5 +38,5 @@ docker pull $IMAGE
WORKDIR=/workdir
docker run -i \
-v ${COMMON_DIR}:${WORKDIR} $IMAGE \
- bash -c "cd ${WORKDIR}; /pulsar/protobuf/src/protoc --java_out=src/main/java src/main/proto/PulsarApi.proto"
+ bash -c "cd ${WORKDIR}; PROTOC=/pulsar/protobuf/src/protoc ./generate_protobuf.sh"
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/Markers.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Markers.java
new file mode 100644
index 0000000000000..a042ea0931be7
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/Markers.java
@@ -0,0 +1,251 @@
+/**
+ * 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.api;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Optional;
+
+import lombok.SneakyThrows;
+import lombok.experimental.UtilityClass;
+
+import org.apache.pulsar.common.api.Commands.ChecksumType;
+import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.MarkerType;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate;
+import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream;
+import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream;
+
+@UtilityClass
+public class Markers {
+
+ private static ByteBuf newMessage(MarkerType markerType, Optional restrictToCluster, ByteBuf payload) {
+ MessageMetadata.Builder msgMetadataBuilder = MessageMetadata.newBuilder();
+ msgMetadataBuilder.setPublishTime(System.currentTimeMillis());
+ msgMetadataBuilder.setProducerName("pulsar.marker");
+ msgMetadataBuilder.setSequenceId(0);
+ msgMetadataBuilder.setMarkerType(markerType.getNumber());
+
+ if (restrictToCluster.isPresent()) {
+ msgMetadataBuilder.addReplicateTo(restrictToCluster.get());
+ }
+
+ MessageMetadata msgMetadata = msgMetadataBuilder.build();
+ try {
+ return Commands.serializeMetadataAndPayload(ChecksumType.Crc32c, msgMetadata, payload);
+ } finally {
+ msgMetadata.recycle();
+ msgMetadataBuilder.recycle();
+ }
+ }
+
+ public static boolean isServerOnlyMarker(MessageMetadata msgMetadata) {
+ // In future, if we add more marker types that can be also sent to clients
+ // we'll have to do finer check here.
+ return msgMetadata.hasMarkerType();
+ }
+
+ @SneakyThrows
+ public static ByteBuf newReplicatedSubscriptionsSnapshotRequest(String snapshotId, String sourceCluster) {
+ ReplicatedSubscriptionsSnapshotRequest.Builder builder = ReplicatedSubscriptionsSnapshotRequest.newBuilder();
+ builder.setSnapshotId(snapshotId);
+ builder.setSourceCluster(sourceCluster);
+
+ ReplicatedSubscriptionsSnapshotRequest req = builder.build();
+
+ int size = req.getSerializedSize();
+
+ ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(size);
+ ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(payload);
+ try {
+ req.writeTo(outStream);
+ return newMessage(MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST, Optional.empty(), payload);
+ } finally {
+ payload.release();
+ builder.recycle();
+ req.recycle();
+ outStream.recycle();
+ }
+ }
+
+ public static ReplicatedSubscriptionsSnapshotRequest parseReplicatedSubscriptionsSnapshotRequest(ByteBuf payload)
+ throws IOException {
+ ByteBufCodedInputStream inStream = ByteBufCodedInputStream.get(payload);
+ ReplicatedSubscriptionsSnapshotRequest.Builder builder = null;
+
+ try {
+ builder = ReplicatedSubscriptionsSnapshotRequest.newBuilder();
+ return builder.mergeFrom(inStream, null).build();
+ } finally {
+ builder.recycle();
+ inStream.recycle();
+ }
+ }
+
+ @SneakyThrows
+ public static ByteBuf newReplicatedSubscriptionsSnapshotResponse(String snapshotId, String replyToCluster,
+ String cluster, long ledgerId, long entryId) {
+ ReplicatedSubscriptionsSnapshotResponse.Builder builder = ReplicatedSubscriptionsSnapshotResponse.newBuilder();
+ builder.setSnapshotId(snapshotId);
+
+ MessageIdData.Builder msgIdBuilder = MessageIdData.newBuilder();
+ msgIdBuilder.setLedgerId(ledgerId);
+ msgIdBuilder.setEntryId(entryId);
+
+ ClusterMessageId.Builder clusterMessageIdBuilder = ClusterMessageId.newBuilder();
+ clusterMessageIdBuilder.setCluster(cluster);
+ clusterMessageIdBuilder.setMessageId(msgIdBuilder);
+
+ builder.setCluster(clusterMessageIdBuilder);
+ ReplicatedSubscriptionsSnapshotResponse response = builder.build();
+
+ int size = response.getSerializedSize();
+
+ ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(size);
+ ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(payload);
+ try {
+ response.writeTo(outStream);
+ return newMessage(MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT_RESPONSE, Optional.of(replyToCluster),
+ payload);
+ } finally {
+ msgIdBuilder.recycle();
+ clusterMessageIdBuilder.recycle();
+ payload.release();
+ builder.recycle();
+ response.recycle();
+ outStream.recycle();
+ }
+ }
+
+ public static ReplicatedSubscriptionsSnapshotResponse parseReplicatedSubscriptionsSnapshotResponse(ByteBuf payload)
+ throws IOException {
+ ByteBufCodedInputStream inStream = ByteBufCodedInputStream.get(payload);
+ ReplicatedSubscriptionsSnapshotResponse.Builder builder = null;
+
+ try {
+ builder = ReplicatedSubscriptionsSnapshotResponse.newBuilder();
+ return builder.mergeFrom(inStream, null).build();
+ } finally {
+ builder.recycle();
+ inStream.recycle();
+ }
+ }
+
+ @SneakyThrows
+ public static ByteBuf newReplicatedSubscriptionsSnapshot(String snapshotId, String sourceCluster, long ledgerId,
+ long entryId, Map clusterIds) {
+ ReplicatedSubscriptionsSnapshot.Builder builder = ReplicatedSubscriptionsSnapshot.newBuilder();
+ builder.setSnapshotId(snapshotId);
+
+ MessageIdData.Builder msgIdBuilder = MessageIdData.newBuilder();
+ msgIdBuilder.setLedgerId(ledgerId);
+ msgIdBuilder.setEntryId(entryId);
+ builder.setLocalMessageId(msgIdBuilder);
+
+ clusterIds.forEach((cluster, msgId) -> {
+ ClusterMessageId.Builder clusterMessageIdBuilder = ClusterMessageId.newBuilder()
+ .setCluster(cluster)
+ .setMessageId(msgId);
+ builder.addClusters(clusterMessageIdBuilder);
+ clusterMessageIdBuilder.recycle();
+ });
+
+ ReplicatedSubscriptionsSnapshot snapshot = builder.build();
+
+ int size = snapshot.getSerializedSize();
+
+ ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(size);
+ ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(payload);
+ try {
+ snapshot.writeTo(outStream);
+ return newMessage(MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT, Optional.of(sourceCluster), payload);
+ } finally {
+ payload.release();
+ builder.recycle();
+ snapshot.recycle();
+ outStream.recycle();
+ }
+ }
+
+ public static ReplicatedSubscriptionsSnapshot parseReplicatedSubscriptionsSnapshot(ByteBuf payload)
+ throws IOException {
+ ByteBufCodedInputStream inStream = ByteBufCodedInputStream.get(payload);
+ ReplicatedSubscriptionsSnapshot.Builder builder = null;
+
+ try {
+ builder = ReplicatedSubscriptionsSnapshot.newBuilder();
+ return builder.mergeFrom(inStream, null).build();
+ } finally {
+ builder.recycle();
+ inStream.recycle();
+ }
+ }
+
+ @SneakyThrows
+ public static ByteBuf newReplicatedSubscriptionsUpdate(String subscriptionName, Map clusterIds) {
+ ReplicatedSubscriptionsUpdate.Builder builder = ReplicatedSubscriptionsUpdate.newBuilder();
+ builder.setSubscriptionName(subscriptionName);
+
+ clusterIds.forEach((cluster, msgId) -> {
+ ClusterMessageId.Builder clusterMessageIdBuilder = ClusterMessageId.newBuilder()
+ .setCluster(cluster)
+ .setMessageId(msgId);
+ builder.addClusters(clusterMessageIdBuilder);
+ clusterMessageIdBuilder.recycle();
+ });
+
+ ReplicatedSubscriptionsUpdate update = builder.build();
+
+ int size = update.getSerializedSize();
+
+ ByteBuf payload = PooledByteBufAllocator.DEFAULT.buffer(size);
+ ByteBufCodedOutputStream outStream = ByteBufCodedOutputStream.get(payload);
+ try {
+ update.writeTo(outStream);
+ return newMessage(MarkerType.REPLICATED_SUBSCRIPTION_SNAPSHOT, Optional.empty(), payload);
+ } finally {
+ payload.release();
+ builder.recycle();
+ update.recycle();
+ outStream.recycle();
+ }
+ }
+
+ public static ReplicatedSubscriptionsUpdate parseReplicatedSubscriptionsUpdate(ByteBuf payload)
+ throws IOException {
+ ByteBufCodedInputStream inStream = ByteBufCodedInputStream.get(payload);
+ ReplicatedSubscriptionsUpdate.Builder builder = null;
+
+ try {
+ builder = ReplicatedSubscriptionsUpdate.newBuilder();
+ return builder.mergeFrom(inStream, null).build();
+ } finally {
+ builder.recycle();
+ inStream.recycle();
+ }
+ }
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
index d55f5f9f2730d..ffe15673cce5c 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java
@@ -3095,6 +3095,10 @@ public interface MessageMetadataOrBuilder
// optional bytes ordering_key = 18;
boolean hasOrderingKey();
org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey();
+
+ // optional int32 marker_type = 20;
+ boolean hasMarkerType();
+ int getMarkerType();
}
public static final class MessageMetadata extends
org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
@@ -3415,6 +3419,16 @@ public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingK
return orderingKey_;
}
+ // optional int32 marker_type = 20;
+ public static final int MARKER_TYPE_FIELD_NUMBER = 20;
+ private int markerType_;
+ public boolean hasMarkerType() {
+ return ((bitField0_ & 0x00004000) == 0x00004000);
+ }
+ public int getMarkerType() {
+ return markerType_;
+ }
+
private void initFields() {
producerName_ = "";
sequenceId_ = 0L;
@@ -3433,6 +3447,7 @@ private void initFields() {
schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
partitionKeyB64Encoded_ = false;
orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
+ markerType_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -3526,6 +3541,9 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr
if (((bitField0_ & 0x00002000) == 0x00002000)) {
output.writeBytes(18, orderingKey_);
}
+ if (((bitField0_ & 0x00004000) == 0x00004000)) {
+ output.writeInt32(20, markerType_);
+ }
}
private int memoizedSerializedSize = -1;
@@ -3607,6 +3625,10 @@ public int getSerializedSize() {
size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
.computeBytesSize(18, orderingKey_);
}
+ if (((bitField0_ & 0x00004000) == 0x00004000)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeInt32Size(20, markerType_);
+ }
memoizedSerializedSize = size;
return size;
}
@@ -3754,6 +3776,8 @@ public Builder clear() {
bitField0_ = (bitField0_ & ~0x00008000);
orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00010000);
+ markerType_ = 0;
+ bitField0_ = (bitField0_ & ~0x00020000);
return this;
}
@@ -3859,6 +3883,10 @@ public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildPartial
to_bitField0_ |= 0x00002000;
}
result.orderingKey_ = orderingKey_;
+ if (((from_bitField0_ & 0x00020000) == 0x00020000)) {
+ to_bitField0_ |= 0x00004000;
+ }
+ result.markerType_ = markerType_;
result.bitField0_ = to_bitField0_;
return result;
}
@@ -3937,6 +3965,9 @@ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageMet
if (other.hasOrderingKey()) {
setOrderingKey(other.getOrderingKey());
}
+ if (other.hasMarkerType()) {
+ setMarkerType(other.getMarkerType());
+ }
return this;
}
@@ -4081,6 +4112,11 @@ public Builder mergeFrom(
orderingKey_ = input.readBytes();
break;
}
+ case 160: {
+ bitField0_ |= 0x00020000;
+ markerType_ = input.readInt32();
+ break;
+ }
}
}
}
@@ -4687,6 +4723,27 @@ public Builder clearOrderingKey() {
return this;
}
+ // optional int32 marker_type = 20;
+ private int markerType_ ;
+ public boolean hasMarkerType() {
+ return ((bitField0_ & 0x00020000) == 0x00020000);
+ }
+ public int getMarkerType() {
+ return markerType_;
+ }
+ public Builder setMarkerType(int value) {
+ bitField0_ |= 0x00020000;
+ markerType_ = value;
+
+ return this;
+ }
+ public Builder clearMarkerType() {
+ bitField0_ = (bitField0_ & ~0x00020000);
+ markerType_ = 0;
+
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:pulsar.proto.MessageMetadata)
}
@@ -6537,7 +6594,7 @@ public interface CommandConnectedOrBuilder
boolean hasProtocolVersion();
int getProtocolVersion();
- // optional int32 max_message_size = 3 [default = 5242880];
+ // optional int32 max_message_size = 3;
boolean hasMaxMessageSize();
int getMaxMessageSize();
}
@@ -6618,7 +6675,7 @@ public int getProtocolVersion() {
return protocolVersion_;
}
- // optional int32 max_message_size = 3 [default = 5242880];
+ // optional int32 max_message_size = 3;
public static final int MAX_MESSAGE_SIZE_FIELD_NUMBER = 3;
private int maxMessageSize_;
public boolean hasMaxMessageSize() {
@@ -6631,7 +6688,7 @@ public int getMaxMessageSize() {
private void initFields() {
serverVersion_ = "";
protocolVersion_ = 0;
- maxMessageSize_ = 5242880;
+ maxMessageSize_ = 0;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -6800,7 +6857,7 @@ public Builder clear() {
bitField0_ = (bitField0_ & ~0x00000001);
protocolVersion_ = 0;
bitField0_ = (bitField0_ & ~0x00000002);
- maxMessageSize_ = 5242880;
+ maxMessageSize_ = 0;
bitField0_ = (bitField0_ & ~0x00000004);
return this;
}
@@ -6973,8 +7030,8 @@ public Builder clearProtocolVersion() {
return this;
}
- // optional int32 max_message_size = 3 [default = 5242880];
- private int maxMessageSize_ = 5242880;
+ // optional int32 max_message_size = 3;
+ private int maxMessageSize_ ;
public boolean hasMaxMessageSize() {
return ((bitField0_ & 0x00000004) == 0x00000004);
}
@@ -6989,7 +7046,7 @@ public Builder setMaxMessageSize(int value) {
}
public Builder clearMaxMessageSize() {
bitField0_ = (bitField0_ & ~0x00000004);
- maxMessageSize_ = 5242880;
+ maxMessageSize_ = 0;
return this;
}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarMarkers.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarMarkers.java
new file mode 100644
index 0000000000000..6af8d41bf3aac
--- /dev/null
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarMarkers.java
@@ -0,0 +1,2988 @@
+// Generated by the protocol buffer compiler. DO NOT EDIT!
+// source: src/main/proto/PulsarMarkers.proto
+
+package org.apache.pulsar.common.api.proto;
+
+public final class PulsarMarkers {
+ private PulsarMarkers() {}
+ public static void registerAllExtensions(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite registry) {
+ }
+ public enum MarkerType
+ implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite {
+ UNKNOWN_MARKER(0, 0),
+ REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST(1, 10),
+ REPLICATED_SUBSCRIPTION_SNAPSHOT_RESPONSE(2, 11),
+ REPLICATED_SUBSCRIPTION_SNAPSHOT(3, 12),
+ REPLICATED_SUBSCRIPTION_UPDATE(4, 13),
+ ;
+
+ public static final int UNKNOWN_MARKER_VALUE = 0;
+ public static final int REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST_VALUE = 10;
+ public static final int REPLICATED_SUBSCRIPTION_SNAPSHOT_RESPONSE_VALUE = 11;
+ public static final int REPLICATED_SUBSCRIPTION_SNAPSHOT_VALUE = 12;
+ public static final int REPLICATED_SUBSCRIPTION_UPDATE_VALUE = 13;
+
+
+ public final int getNumber() { return value; }
+
+ public static MarkerType valueOf(int value) {
+ switch (value) {
+ case 0: return UNKNOWN_MARKER;
+ case 10: return REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST;
+ case 11: return REPLICATED_SUBSCRIPTION_SNAPSHOT_RESPONSE;
+ case 12: return REPLICATED_SUBSCRIPTION_SNAPSHOT;
+ case 13: return REPLICATED_SUBSCRIPTION_UPDATE;
+ default: return null;
+ }
+ }
+
+ public static org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLiteMap
+ internalGetValueMap() {
+ return internalValueMap;
+ }
+ private static org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLiteMap
+ internalValueMap =
+ new org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLiteMap() {
+ public MarkerType findValueByNumber(int number) {
+ return MarkerType.valueOf(number);
+ }
+ };
+
+ private final int value;
+
+ private MarkerType(int index, int value) {
+ this.value = value;
+ }
+
+ // @@protoc_insertion_point(enum_scope:pulsar.proto.MarkerType)
+ }
+
+ public interface ReplicatedSubscriptionsSnapshotRequestOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required string snapshot_id = 1;
+ boolean hasSnapshotId();
+ String getSnapshotId();
+
+ // optional string source_cluster = 2;
+ boolean hasSourceCluster();
+ String getSourceCluster();
+ }
+ public static final class ReplicatedSubscriptionsSnapshotRequest extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements ReplicatedSubscriptionsSnapshotRequestOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use ReplicatedSubscriptionsSnapshotRequest.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private ReplicatedSubscriptionsSnapshotRequest(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected ReplicatedSubscriptionsSnapshotRequest newObject(Handle handle) {
+ return new ReplicatedSubscriptionsSnapshotRequest(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private ReplicatedSubscriptionsSnapshotRequest(boolean noInit) {}
+
+ private static final ReplicatedSubscriptionsSnapshotRequest defaultInstance;
+ public static ReplicatedSubscriptionsSnapshotRequest getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ReplicatedSubscriptionsSnapshotRequest getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required string snapshot_id = 1;
+ public static final int SNAPSHOT_ID_FIELD_NUMBER = 1;
+ private java.lang.Object snapshotId_;
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ snapshotId_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSnapshotIdBytes() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ snapshotId_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ // optional string source_cluster = 2;
+ public static final int SOURCE_CLUSTER_FIELD_NUMBER = 2;
+ private java.lang.Object sourceCluster_;
+ public boolean hasSourceCluster() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public String getSourceCluster() {
+ java.lang.Object ref = sourceCluster_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ sourceCluster_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSourceClusterBytes() {
+ java.lang.Object ref = sourceCluster_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ sourceCluster_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ private void initFields() {
+ snapshotId_ = "";
+ sourceCluster_ = "";
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasSnapshotId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBytes(2, getSourceClusterBytes());
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(2, getSourceClusterBytes());
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequestOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ snapshotId_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ sourceCluster_ = "";
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest result = org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.snapshotId_ = snapshotId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.sourceCluster_ = sourceCluster_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest.getDefaultInstance()) return this;
+ if (other.hasSnapshotId()) {
+ setSnapshotId(other.getSnapshotId());
+ }
+ if (other.hasSourceCluster()) {
+ setSourceCluster(other.getSourceCluster());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasSnapshotId()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = input.readBytes();
+ break;
+ }
+ case 18: {
+ bitField0_ |= 0x00000002;
+ sourceCluster_ = input.readBytes();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required string snapshot_id = 1;
+ private java.lang.Object snapshotId_ = "";
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ snapshotId_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSnapshotId(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ return this;
+ }
+ public Builder clearSnapshotId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ snapshotId_ = getDefaultInstance().getSnapshotId();
+
+ return this;
+ }
+ void setSnapshotId(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ }
+
+ // optional string source_cluster = 2;
+ private java.lang.Object sourceCluster_ = "";
+ public boolean hasSourceCluster() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public String getSourceCluster() {
+ java.lang.Object ref = sourceCluster_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ sourceCluster_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSourceCluster(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000002;
+ sourceCluster_ = value;
+
+ return this;
+ }
+ public Builder clearSourceCluster() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ sourceCluster_ = getDefaultInstance().getSourceCluster();
+
+ return this;
+ }
+ void setSourceCluster(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000002;
+ sourceCluster_ = value;
+
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.ReplicatedSubscriptionsSnapshotRequest)
+ }
+
+ static {
+ defaultInstance = new ReplicatedSubscriptionsSnapshotRequest(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.ReplicatedSubscriptionsSnapshotRequest)
+ }
+
+ public interface ReplicatedSubscriptionsSnapshotResponseOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required string snapshot_id = 1;
+ boolean hasSnapshotId();
+ String getSnapshotId();
+
+ // optional .pulsar.proto.ClusterMessageId cluster = 2;
+ boolean hasCluster();
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getCluster();
+ }
+ public static final class ReplicatedSubscriptionsSnapshotResponse extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements ReplicatedSubscriptionsSnapshotResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use ReplicatedSubscriptionsSnapshotResponse.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private ReplicatedSubscriptionsSnapshotResponse(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected ReplicatedSubscriptionsSnapshotResponse newObject(Handle handle) {
+ return new ReplicatedSubscriptionsSnapshotResponse(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private ReplicatedSubscriptionsSnapshotResponse(boolean noInit) {}
+
+ private static final ReplicatedSubscriptionsSnapshotResponse defaultInstance;
+ public static ReplicatedSubscriptionsSnapshotResponse getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ReplicatedSubscriptionsSnapshotResponse getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required string snapshot_id = 1;
+ public static final int SNAPSHOT_ID_FIELD_NUMBER = 1;
+ private java.lang.Object snapshotId_;
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ snapshotId_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSnapshotIdBytes() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ snapshotId_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ // optional .pulsar.proto.ClusterMessageId cluster = 2;
+ public static final int CLUSTER_FIELD_NUMBER = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId cluster_;
+ public boolean hasCluster() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getCluster() {
+ return cluster_;
+ }
+
+ private void initFields() {
+ snapshotId_ = "";
+ cluster_ = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasSnapshotId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (hasCluster()) {
+ if (!getCluster().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, cluster_);
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeMessageSize(2, cluster_);
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ snapshotId_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ cluster_ = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse result = org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.snapshotId_ = snapshotId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.cluster_ = cluster_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse.getDefaultInstance()) return this;
+ if (other.hasSnapshotId()) {
+ setSnapshotId(other.getSnapshotId());
+ }
+ if (other.hasCluster()) {
+ mergeCluster(other.getCluster());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasSnapshotId()) {
+
+ return false;
+ }
+ if (hasCluster()) {
+ if (!getCluster().isInitialized()) {
+
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = input.readBytes();
+ break;
+ }
+ case 18: {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.newBuilder();
+ if (hasCluster()) {
+ subBuilder.mergeFrom(getCluster());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setCluster(subBuilder.buildPartial());
+ subBuilder.recycle();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required string snapshot_id = 1;
+ private java.lang.Object snapshotId_ = "";
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ snapshotId_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSnapshotId(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ return this;
+ }
+ public Builder clearSnapshotId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ snapshotId_ = getDefaultInstance().getSnapshotId();
+
+ return this;
+ }
+ void setSnapshotId(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ }
+
+ // optional .pulsar.proto.ClusterMessageId cluster = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId cluster_ = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance();
+ public boolean hasCluster() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getCluster() {
+ return cluster_;
+ }
+ public Builder setCluster(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ cluster_ = value;
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder setCluster(
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ cluster_ = builderForValue.build();
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder mergeCluster(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (((bitField0_ & 0x00000002) == 0x00000002) &&
+ cluster_ != org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance()) {
+ cluster_ =
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.newBuilder(cluster_).mergeFrom(value).buildPartial();
+ } else {
+ cluster_ = value;
+ }
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder clearCluster() {
+ cluster_ = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance();
+
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.ReplicatedSubscriptionsSnapshotResponse)
+ }
+
+ static {
+ defaultInstance = new ReplicatedSubscriptionsSnapshotResponse(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.ReplicatedSubscriptionsSnapshotResponse)
+ }
+
+ public interface ReplicatedSubscriptionsSnapshotOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required string snapshot_id = 1;
+ boolean hasSnapshotId();
+ String getSnapshotId();
+
+ // optional .pulsar.proto.MessageIdData local_message_id = 2;
+ boolean hasLocalMessageId();
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getLocalMessageId();
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 3;
+ java.util.List
+ getClustersList();
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index);
+ int getClustersCount();
+ }
+ public static final class ReplicatedSubscriptionsSnapshot extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements ReplicatedSubscriptionsSnapshotOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use ReplicatedSubscriptionsSnapshot.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private ReplicatedSubscriptionsSnapshot(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected ReplicatedSubscriptionsSnapshot newObject(Handle handle) {
+ return new ReplicatedSubscriptionsSnapshot(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private ReplicatedSubscriptionsSnapshot(boolean noInit) {}
+
+ private static final ReplicatedSubscriptionsSnapshot defaultInstance;
+ public static ReplicatedSubscriptionsSnapshot getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ReplicatedSubscriptionsSnapshot getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required string snapshot_id = 1;
+ public static final int SNAPSHOT_ID_FIELD_NUMBER = 1;
+ private java.lang.Object snapshotId_;
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ snapshotId_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSnapshotIdBytes() {
+ java.lang.Object ref = snapshotId_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ snapshotId_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ // optional .pulsar.proto.MessageIdData local_message_id = 2;
+ public static final int LOCAL_MESSAGE_ID_FIELD_NUMBER = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData localMessageId_;
+ public boolean hasLocalMessageId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getLocalMessageId() {
+ return localMessageId_;
+ }
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 3;
+ public static final int CLUSTERS_FIELD_NUMBER = 3;
+ private java.util.List clusters_;
+ public java.util.List getClustersList() {
+ return clusters_;
+ }
+ public java.util.List extends org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageIdOrBuilder>
+ getClustersOrBuilderList() {
+ return clusters_;
+ }
+ public int getClustersCount() {
+ return clusters_.size();
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index) {
+ return clusters_.get(index);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageIdOrBuilder getClustersOrBuilder(
+ int index) {
+ return clusters_.get(index);
+ }
+
+ private void initFields() {
+ snapshotId_ = "";
+ localMessageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ clusters_ = java.util.Collections.emptyList();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasSnapshotId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (hasLocalMessageId()) {
+ if (!getLocalMessageId().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ for (int i = 0; i < getClustersCount(); i++) {
+ if (!getClusters(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, localMessageId_);
+ }
+ for (int i = 0; i < clusters_.size(); i++) {
+ output.writeMessage(3, clusters_.get(i));
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(1, getSnapshotIdBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeMessageSize(2, localMessageId_);
+ }
+ for (int i = 0; i < clusters_.size(); i++) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeMessageSize(3, clusters_.get(i));
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ snapshotId_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ localMessageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ clusters_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000004);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot result = org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.snapshotId_ = snapshotId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.localMessageId_ = localMessageId_;
+ if (((bitField0_ & 0x00000004) == 0x00000004)) {
+ clusters_ = java.util.Collections.unmodifiableList(clusters_);
+ bitField0_ = (bitField0_ & ~0x00000004);
+ }
+ result.clusters_ = clusters_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot.getDefaultInstance()) return this;
+ if (other.hasSnapshotId()) {
+ setSnapshotId(other.getSnapshotId());
+ }
+ if (other.hasLocalMessageId()) {
+ mergeLocalMessageId(other.getLocalMessageId());
+ }
+ if (!other.clusters_.isEmpty()) {
+ if (clusters_.isEmpty()) {
+ clusters_ = other.clusters_;
+ bitField0_ = (bitField0_ & ~0x00000004);
+ } else {
+ ensureClustersIsMutable();
+ clusters_.addAll(other.clusters_);
+ }
+
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasSnapshotId()) {
+
+ return false;
+ }
+ if (hasLocalMessageId()) {
+ if (!getLocalMessageId().isInitialized()) {
+
+ return false;
+ }
+ }
+ for (int i = 0; i < getClustersCount(); i++) {
+ if (!getClusters(i).isInitialized()) {
+
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = input.readBytes();
+ break;
+ }
+ case 18: {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.newBuilder();
+ if (hasLocalMessageId()) {
+ subBuilder.mergeFrom(getLocalMessageId());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setLocalMessageId(subBuilder.buildPartial());
+ subBuilder.recycle();
+ break;
+ }
+ case 26: {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.newBuilder();
+ input.readMessage(subBuilder, extensionRegistry);
+ addClusters(subBuilder.buildPartial());
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required string snapshot_id = 1;
+ private java.lang.Object snapshotId_ = "";
+ public boolean hasSnapshotId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSnapshotId() {
+ java.lang.Object ref = snapshotId_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ snapshotId_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSnapshotId(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ return this;
+ }
+ public Builder clearSnapshotId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ snapshotId_ = getDefaultInstance().getSnapshotId();
+
+ return this;
+ }
+ void setSnapshotId(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000001;
+ snapshotId_ = value;
+
+ }
+
+ // optional .pulsar.proto.MessageIdData local_message_id = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData localMessageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ public boolean hasLocalMessageId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getLocalMessageId() {
+ return localMessageId_;
+ }
+ public Builder setLocalMessageId(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ localMessageId_ = value;
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder setLocalMessageId(
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.Builder builderForValue) {
+ localMessageId_ = builderForValue.build();
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder mergeLocalMessageId(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData value) {
+ if (((bitField0_ & 0x00000002) == 0x00000002) &&
+ localMessageId_ != org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance()) {
+ localMessageId_ =
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.newBuilder(localMessageId_).mergeFrom(value).buildPartial();
+ } else {
+ localMessageId_ = value;
+ }
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder clearLocalMessageId() {
+ localMessageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 3;
+ private java.util.List clusters_ =
+ java.util.Collections.emptyList();
+ private void ensureClustersIsMutable() {
+ if (!((bitField0_ & 0x00000004) == 0x00000004)) {
+ clusters_ = new java.util.ArrayList(clusters_);
+ bitField0_ |= 0x00000004;
+ }
+ }
+
+ public java.util.List getClustersList() {
+ return java.util.Collections.unmodifiableList(clusters_);
+ }
+ public int getClustersCount() {
+ return clusters_.size();
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index) {
+ return clusters_.get(index);
+ }
+ public Builder setClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.set(index, value);
+
+ return this;
+ }
+ public Builder setClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.set(index, builderForValue.build());
+
+ return this;
+ }
+ public Builder addClusters(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.add(value);
+
+ return this;
+ }
+ public Builder addClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.add(index, value);
+
+ return this;
+ }
+ public Builder addClusters(
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.add(builderForValue.build());
+
+ return this;
+ }
+ public Builder addClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.add(index, builderForValue.build());
+
+ return this;
+ }
+ public Builder addAllClusters(
+ java.lang.Iterable extends org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId> values) {
+ ensureClustersIsMutable();
+ super.addAll(values, clusters_);
+
+ return this;
+ }
+ public Builder clearClusters() {
+ clusters_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000004);
+
+ return this;
+ }
+ public Builder removeClusters(int index) {
+ ensureClustersIsMutable();
+ clusters_.remove(index);
+
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.ReplicatedSubscriptionsSnapshot)
+ }
+
+ static {
+ defaultInstance = new ReplicatedSubscriptionsSnapshot(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.ReplicatedSubscriptionsSnapshot)
+ }
+
+ public interface ReplicatedSubscriptionsUpdateOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required string subscription_name = 1;
+ boolean hasSubscriptionName();
+ String getSubscriptionName();
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 2;
+ java.util.List
+ getClustersList();
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index);
+ int getClustersCount();
+ }
+ public static final class ReplicatedSubscriptionsUpdate extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements ReplicatedSubscriptionsUpdateOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use ReplicatedSubscriptionsUpdate.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private ReplicatedSubscriptionsUpdate(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected ReplicatedSubscriptionsUpdate newObject(Handle handle) {
+ return new ReplicatedSubscriptionsUpdate(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private ReplicatedSubscriptionsUpdate(boolean noInit) {}
+
+ private static final ReplicatedSubscriptionsUpdate defaultInstance;
+ public static ReplicatedSubscriptionsUpdate getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ReplicatedSubscriptionsUpdate getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required string subscription_name = 1;
+ public static final int SUBSCRIPTION_NAME_FIELD_NUMBER = 1;
+ private java.lang.Object subscriptionName_;
+ public boolean hasSubscriptionName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSubscriptionName() {
+ java.lang.Object ref = subscriptionName_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ subscriptionName_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSubscriptionNameBytes() {
+ java.lang.Object ref = subscriptionName_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ subscriptionName_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 2;
+ public static final int CLUSTERS_FIELD_NUMBER = 2;
+ private java.util.List clusters_;
+ public java.util.List getClustersList() {
+ return clusters_;
+ }
+ public java.util.List extends org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageIdOrBuilder>
+ getClustersOrBuilderList() {
+ return clusters_;
+ }
+ public int getClustersCount() {
+ return clusters_.size();
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index) {
+ return clusters_.get(index);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageIdOrBuilder getClustersOrBuilder(
+ int index) {
+ return clusters_.get(index);
+ }
+
+ private void initFields() {
+ subscriptionName_ = "";
+ clusters_ = java.util.Collections.emptyList();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasSubscriptionName()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ for (int i = 0; i < getClustersCount(); i++) {
+ if (!getClusters(i).isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getSubscriptionNameBytes());
+ }
+ for (int i = 0; i < clusters_.size(); i++) {
+ output.writeMessage(2, clusters_.get(i));
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(1, getSubscriptionNameBytes());
+ }
+ for (int i = 0; i < clusters_.size(); i++) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeMessageSize(2, clusters_.get(i));
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdateOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ subscriptionName_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ clusters_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate result = org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.subscriptionName_ = subscriptionName_;
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ clusters_ = java.util.Collections.unmodifiableList(clusters_);
+ bitField0_ = (bitField0_ & ~0x00000002);
+ }
+ result.clusters_ = clusters_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate.getDefaultInstance()) return this;
+ if (other.hasSubscriptionName()) {
+ setSubscriptionName(other.getSubscriptionName());
+ }
+ if (!other.clusters_.isEmpty()) {
+ if (clusters_.isEmpty()) {
+ clusters_ = other.clusters_;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ } else {
+ ensureClustersIsMutable();
+ clusters_.addAll(other.clusters_);
+ }
+
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasSubscriptionName()) {
+
+ return false;
+ }
+ for (int i = 0; i < getClustersCount(); i++) {
+ if (!getClusters(i).isInitialized()) {
+
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ subscriptionName_ = input.readBytes();
+ break;
+ }
+ case 18: {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.newBuilder();
+ input.readMessage(subBuilder, extensionRegistry);
+ addClusters(subBuilder.buildPartial());
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required string subscription_name = 1;
+ private java.lang.Object subscriptionName_ = "";
+ public boolean hasSubscriptionName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getSubscriptionName() {
+ java.lang.Object ref = subscriptionName_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ subscriptionName_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setSubscriptionName(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ subscriptionName_ = value;
+
+ return this;
+ }
+ public Builder clearSubscriptionName() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ subscriptionName_ = getDefaultInstance().getSubscriptionName();
+
+ return this;
+ }
+ void setSubscriptionName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000001;
+ subscriptionName_ = value;
+
+ }
+
+ // repeated .pulsar.proto.ClusterMessageId clusters = 2;
+ private java.util.List clusters_ =
+ java.util.Collections.emptyList();
+ private void ensureClustersIsMutable() {
+ if (!((bitField0_ & 0x00000002) == 0x00000002)) {
+ clusters_ = new java.util.ArrayList(clusters_);
+ bitField0_ |= 0x00000002;
+ }
+ }
+
+ public java.util.List getClustersList() {
+ return java.util.Collections.unmodifiableList(clusters_);
+ }
+ public int getClustersCount() {
+ return clusters_.size();
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getClusters(int index) {
+ return clusters_.get(index);
+ }
+ public Builder setClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.set(index, value);
+
+ return this;
+ }
+ public Builder setClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.set(index, builderForValue.build());
+
+ return this;
+ }
+ public Builder addClusters(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.add(value);
+
+ return this;
+ }
+ public Builder addClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ ensureClustersIsMutable();
+ clusters_.add(index, value);
+
+ return this;
+ }
+ public Builder addClusters(
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.add(builderForValue.build());
+
+ return this;
+ }
+ public Builder addClusters(
+ int index, org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.Builder builderForValue) {
+ ensureClustersIsMutable();
+ clusters_.add(index, builderForValue.build());
+
+ return this;
+ }
+ public Builder addAllClusters(
+ java.lang.Iterable extends org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId> values) {
+ ensureClustersIsMutable();
+ super.addAll(values, clusters_);
+
+ return this;
+ }
+ public Builder clearClusters() {
+ clusters_ = java.util.Collections.emptyList();
+ bitField0_ = (bitField0_ & ~0x00000002);
+
+ return this;
+ }
+ public Builder removeClusters(int index) {
+ ensureClustersIsMutable();
+ clusters_.remove(index);
+
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.ReplicatedSubscriptionsUpdate)
+ }
+
+ static {
+ defaultInstance = new ReplicatedSubscriptionsUpdate(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.ReplicatedSubscriptionsUpdate)
+ }
+
+ public interface ClusterMessageIdOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required string cluster = 1;
+ boolean hasCluster();
+ String getCluster();
+
+ // required .pulsar.proto.MessageIdData message_id = 2;
+ boolean hasMessageId();
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getMessageId();
+ }
+ public static final class ClusterMessageId extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements ClusterMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use ClusterMessageId.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private ClusterMessageId(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected ClusterMessageId newObject(Handle handle) {
+ return new ClusterMessageId(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private ClusterMessageId(boolean noInit) {}
+
+ private static final ClusterMessageId defaultInstance;
+ public static ClusterMessageId getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public ClusterMessageId getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required string cluster = 1;
+ public static final int CLUSTER_FIELD_NUMBER = 1;
+ private java.lang.Object cluster_;
+ public boolean hasCluster() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getCluster() {
+ java.lang.Object ref = cluster_;
+ if (ref instanceof String) {
+ return (String) ref;
+ } else {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString bs =
+ (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ String s = bs.toStringUtf8();
+ if (org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.isValidUtf8(bs)) {
+ cluster_ = s;
+ }
+ return s;
+ }
+ }
+ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getClusterBytes() {
+ java.lang.Object ref = cluster_;
+ if (ref instanceof String) {
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString b =
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.copyFromUtf8((String) ref);
+ cluster_ = b;
+ return b;
+ } else {
+ return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref;
+ }
+ }
+
+ // required .pulsar.proto.MessageIdData message_id = 2;
+ public static final int MESSAGE_ID_FIELD_NUMBER = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData messageId_;
+ public boolean hasMessageId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getMessageId() {
+ return messageId_;
+ }
+
+ private void initFields() {
+ cluster_ = "";
+ messageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasCluster()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasMessageId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!getMessageId().isInitialized()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeBytes(1, getClusterBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeMessage(2, messageId_);
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeBytesSize(1, getClusterBytes());
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeMessageSize(2, messageId_);
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ cluster_ = "";
+ bitField0_ = (bitField0_ & ~0x00000001);
+ messageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId result = org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.cluster_ = cluster_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.messageId_ = messageId_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.ClusterMessageId.getDefaultInstance()) return this;
+ if (other.hasCluster()) {
+ setCluster(other.getCluster());
+ }
+ if (other.hasMessageId()) {
+ mergeMessageId(other.getMessageId());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasCluster()) {
+
+ return false;
+ }
+ if (!hasMessageId()) {
+
+ return false;
+ }
+ if (!getMessageId().isInitialized()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ bitField0_ |= 0x00000001;
+ cluster_ = input.readBytes();
+ break;
+ }
+ case 18: {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.newBuilder();
+ if (hasMessageId()) {
+ subBuilder.mergeFrom(getMessageId());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setMessageId(subBuilder.buildPartial());
+ subBuilder.recycle();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required string cluster = 1;
+ private java.lang.Object cluster_ = "";
+ public boolean hasCluster() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public String getCluster() {
+ java.lang.Object ref = cluster_;
+ if (!(ref instanceof String)) {
+ String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8();
+ cluster_ = s;
+ return s;
+ } else {
+ return (String) ref;
+ }
+ }
+ public Builder setCluster(String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ bitField0_ |= 0x00000001;
+ cluster_ = value;
+
+ return this;
+ }
+ public Builder clearCluster() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ cluster_ = getDefaultInstance().getCluster();
+
+ return this;
+ }
+ void setCluster(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) {
+ bitField0_ |= 0x00000001;
+ cluster_ = value;
+
+ }
+
+ // required .pulsar.proto.MessageIdData message_id = 2;
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ public boolean hasMessageId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getMessageId() {
+ return messageId_;
+ }
+ public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ messageId_ = value;
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder setMessageId(
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.Builder builderForValue) {
+ messageId_ = builderForValue.build();
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData value) {
+ if (((bitField0_ & 0x00000002) == 0x00000002) &&
+ messageId_ != org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance()) {
+ messageId_ =
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial();
+ } else {
+ messageId_ = value;
+ }
+
+ bitField0_ |= 0x00000002;
+ return this;
+ }
+ public Builder clearMessageId() {
+ messageId_ = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.ClusterMessageId)
+ }
+
+ static {
+ defaultInstance = new ClusterMessageId(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.ClusterMessageId)
+ }
+
+ public interface MessageIdDataOrBuilder
+ extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder {
+
+ // required uint64 ledger_id = 1;
+ boolean hasLedgerId();
+ long getLedgerId();
+
+ // required uint64 entry_id = 2;
+ boolean hasEntryId();
+ long getEntryId();
+ }
+ public static final class MessageIdData extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite
+ implements MessageIdDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage {
+ // Use MessageIdData.newBuilder() to construct.
+ private io.netty.util.Recycler.Handle handle;
+ private MessageIdData(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ }
+
+ private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected MessageIdData newObject(Handle handle) {
+ return new MessageIdData(handle);
+ }
+ };
+
+ public void recycle() {
+ this.initFields();
+ this.memoizedIsInitialized = -1;
+ this.bitField0_ = 0;
+ this.memoizedSerializedSize = -1;
+ if (handle != null) { RECYCLER.recycle(this, handle); }
+ }
+
+ private MessageIdData(boolean noInit) {}
+
+ private static final MessageIdData defaultInstance;
+ public static MessageIdData getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public MessageIdData getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private int bitField0_;
+ // required uint64 ledger_id = 1;
+ public static final int LEDGER_ID_FIELD_NUMBER = 1;
+ private long ledgerId_;
+ public boolean hasLedgerId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getLedgerId() {
+ return ledgerId_;
+ }
+
+ // required uint64 entry_id = 2;
+ public static final int ENTRY_ID_FIELD_NUMBER = 2;
+ private long entryId_;
+ public boolean hasEntryId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public long getEntryId() {
+ return entryId_;
+ }
+
+ private void initFields() {
+ ledgerId_ = 0L;
+ entryId_ = 0L;
+ }
+ private byte memoizedIsInitialized = -1;
+ public final boolean isInitialized() {
+ byte isInitialized = memoizedIsInitialized;
+ if (isInitialized != -1) return isInitialized == 1;
+
+ if (!hasLedgerId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ if (!hasEntryId()) {
+ memoizedIsInitialized = 0;
+ return false;
+ }
+ memoizedIsInitialized = 1;
+ return true;
+ }
+
+ public void writeTo(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream output)
+ throws java.io.IOException {
+ throw new RuntimeException("Cannot use CodedOutputStream");
+ }
+
+ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(1, ledgerId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeUInt64(2, entryId_);
+ }
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeUInt64Size(1, ledgerId_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream
+ .computeUInt64Size(2, entryId_);
+ }
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ private static final long serialVersionUID = 0L;
+ @java.lang.Override
+ protected java.lang.Object writeReplace()
+ throws java.io.ObjectStreamException {
+ return super.writeReplace();
+ }
+
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ throw new RuntimeException("Disabled");
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(byte[] data)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ byte[] data,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseDelimitedFrom(
+ java.io.InputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData parseFrom(
+ org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite.Builder<
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData, Builder>
+ implements org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder {
+ // Construct using org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.newBuilder()
+ private final io.netty.util.Recycler.Handle handle;
+ private Builder(io.netty.util.Recycler.Handle handle) {
+ this.handle = handle;
+ maybeForceBuilderInitialization();
+ }
+ private final static io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() {
+ protected Builder newObject(io.netty.util.Recycler.Handle handle) {
+ return new Builder(handle);
+ }
+ };
+
+ public void recycle() {
+ clear();
+ if (handle != null) {RECYCLER.recycle(this, handle);}
+ }
+
+ private void maybeForceBuilderInitialization() {
+ }
+ private static Builder create() {
+ return RECYCLER.get();
+ }
+
+ public Builder clear() {
+ super.clear();
+ ledgerId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000001);
+ entryId_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData getDefaultInstanceForType() {
+ return org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance();
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData build() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ private org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData buildParsed()
+ throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return result;
+ }
+
+ public org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData buildPartial() {
+ org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData result = org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.RECYCLER.get();
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.ledgerId_ = ledgerId_;
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.entryId_ = entryId_;
+ result.bitField0_ = to_bitField0_;
+ return result;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData other) {
+ if (other == org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData.getDefaultInstance()) return this;
+ if (other.hasLedgerId()) {
+ setLedgerId(other.getLedgerId());
+ }
+ if (other.hasEntryId()) {
+ setEntryId(other.getEntryId());
+ }
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasLedgerId()) {
+
+ return false;
+ }
+ if (!hasEntryId()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.CodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ throw new java.io.IOException("Merge from CodedInputStream is disabled");
+ }
+ public Builder mergeFrom(
+ org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream input,
+ org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+
+ return this;
+ default: {
+ if (!input.skipField(tag)) {
+
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ bitField0_ |= 0x00000001;
+ ledgerId_ = input.readUInt64();
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ entryId_ = input.readUInt64();
+ break;
+ }
+ }
+ }
+ }
+
+ private int bitField0_;
+
+ // required uint64 ledger_id = 1;
+ private long ledgerId_ ;
+ public boolean hasLedgerId() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ public long getLedgerId() {
+ return ledgerId_;
+ }
+ public Builder setLedgerId(long value) {
+ bitField0_ |= 0x00000001;
+ ledgerId_ = value;
+
+ return this;
+ }
+ public Builder clearLedgerId() {
+ bitField0_ = (bitField0_ & ~0x00000001);
+ ledgerId_ = 0L;
+
+ return this;
+ }
+
+ // required uint64 entry_id = 2;
+ private long entryId_ ;
+ public boolean hasEntryId() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ public long getEntryId() {
+ return entryId_;
+ }
+ public Builder setEntryId(long value) {
+ bitField0_ |= 0x00000002;
+ entryId_ = value;
+
+ return this;
+ }
+ public Builder clearEntryId() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ entryId_ = 0L;
+
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:pulsar.proto.MessageIdData)
+ }
+
+ static {
+ defaultInstance = new MessageIdData(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:pulsar.proto.MessageIdData)
+ }
+
+
+ static {
+ }
+
+ // @@protoc_insertion_point(outer_class_scope)
+}
diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
index da24a6c005919..1939d4b68cc20 100644
--- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
+++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/raw/MessageParser.java
@@ -69,6 +69,11 @@ public static void parseMessage(TopicName topicName, long ledgerId, long entryId
return;
}
+ if (msgMetadata.hasMarkerType()) {
+ // Ignore marker messages as they don't contain user data
+ return;
+ }
+
if (msgMetadata.getEncryptionKeysCount() > 0) {
throw new IOException("Cannot parse encrypted message " + msgMetadata + " on topic " + topicName);
}
diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto
index 3913b660ef319..996ea21913205 100644
--- a/pulsar-common/src/main/proto/PulsarApi.proto
+++ b/pulsar-common/src/main/proto/PulsarApi.proto
@@ -114,8 +114,12 @@ message MessageMetadata {
optional bool partition_key_b64_encoded = 17 [ default = false ];
// Specific a key to overwrite the message key which used for ordering dispatch in Key_Shared mode.
optional bytes ordering_key = 18;
-}
+ // Identify whether a message is a "marker" message used for
+ // internal metadata instead of application published data.
+ // Markers will generally not be propagated back to clients
+ optional int32 marker_type = 20;
+}
message SingleMessageMetadata {
repeated KeyValue properties = 1;
diff --git a/pulsar-common/src/main/proto/PulsarMarkers.proto b/pulsar-common/src/main/proto/PulsarMarkers.proto
new file mode 100644
index 0000000000000..92433610c1f59
--- /dev/null
+++ b/pulsar-common/src/main/proto/PulsarMarkers.proto
@@ -0,0 +1,80 @@
+/**
+ * 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.
+ */
+syntax = "proto2";
+
+package pulsar.proto;
+option java_package = "org.apache.pulsar.common.api.proto";
+option optimize_for = LITE_RUNTIME;
+
+enum MarkerType {
+ UNKNOWN_MARKER = 0;
+
+ // Replicated subscription markers
+ REPLICATED_SUBSCRIPTION_SNAPSHOT_REQUEST = 10;
+ REPLICATED_SUBSCRIPTION_SNAPSHOT_RESPONSE = 11;
+ REPLICATED_SUBSCRIPTION_SNAPSHOT = 12;
+ REPLICATED_SUBSCRIPTION_UPDATE = 13;
+
+ // Next markers start at 20
+}
+
+/// --- Replicated subscriptions ---
+
+// A cluster uses this message to request the current
+// message id from all the other clusters.
+message ReplicatedSubscriptionsSnapshotRequest {
+ required string snapshot_id = 1;
+ optional string source_cluster = 2;
+}
+
+// When a cluster receives the snapshot request, it replies
+// by sending back the response (only to original asking cluster)
+message ReplicatedSubscriptionsSnapshotResponse {
+ required string snapshot_id = 1;
+ optional ClusterMessageId cluster = 2;
+}
+
+// This message is used to store the snapshot in the
+// local topic. It's not meant to be replicated to other
+// clusters
+message ReplicatedSubscriptionsSnapshot {
+ required string snapshot_id = 1;
+ optional MessageIdData local_message_id = 2;
+ repeated ClusterMessageId clusters = 3;
+}
+
+// When the replicated subscription mark-delete position
+// is updated in the source cluster, this message will be
+// sent to all clusters to updated the mirrored subscriptions
+message ReplicatedSubscriptionsUpdate {
+ required string subscription_name = 1;
+ repeated ClusterMessageId clusters = 2;
+}
+
+// Represent one cluster and an associated message id.
+// The message id is local to that particular cluster
+message ClusterMessageId {
+ required string cluster = 1;
+ required MessageIdData message_id = 2;
+}
+
+message MessageIdData {
+ required uint64 ledger_id = 1;
+ required uint64 entry_id = 2;
+}
diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/api/MarkersTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/api/MarkersTest.java
new file mode 100644
index 0000000000000..12a8a74f9ffbf
--- /dev/null
+++ b/pulsar-common/src/test/java/org/apache/pulsar/common/api/MarkersTest.java
@@ -0,0 +1,118 @@
+/**
+ * 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.api;
+
+import static org.testng.Assert.assertEquals;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.MessageIdData;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotRequest;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshotResponse;
+import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsUpdate;
+import org.testng.annotations.Test;
+
+public class MarkersTest {
+ @Test
+ public void testSnapshotRequest() throws Exception {
+ ByteBuf buf = Markers.newReplicatedSubscriptionsSnapshotRequest("sid", "us-west");
+
+ MessageMetadata msgMetadata = Commands.parseMessageMetadata(buf);
+ assertEquals(msgMetadata.getReplicateToCount(), 0);
+
+ ReplicatedSubscriptionsSnapshotRequest request = Markers.parseReplicatedSubscriptionsSnapshotRequest(buf);
+
+ assertEquals(request.getSnapshotId(), "sid");
+ assertEquals(request.getSourceCluster(), "us-west");
+ }
+
+ @Test
+ public void testSnapshotResponse() throws Exception {
+ ByteBuf buf = Markers.newReplicatedSubscriptionsSnapshotResponse("sid", "us-west", "us-east", 5, 7);
+
+ MessageMetadata msgMetadata = Commands.parseMessageMetadata(buf);
+ assertEquals(msgMetadata.getReplicateToCount(), 1);
+ assertEquals(msgMetadata.getReplicateTo(0), "us-west");
+
+ ReplicatedSubscriptionsSnapshotResponse response = Markers.parseReplicatedSubscriptionsSnapshotResponse(buf);
+
+ assertEquals(response.getSnapshotId(), "sid");
+ assertEquals(response.getCluster().getCluster(), "us-east");
+ assertEquals(response.getCluster().getMessageId().getLedgerId(), 5);
+ assertEquals(response.getCluster().getMessageId().getEntryId(), 7);
+ }
+
+ @Test
+ public void testSnapshot() throws Exception {
+ Map clusters = new TreeMap<>();
+ clusters.put("us-east", MessageIdData.newBuilder().setLedgerId(10).setEntryId(11).build());
+ clusters.put("us-cent", MessageIdData.newBuilder().setLedgerId(20).setEntryId(21).build());
+
+ ByteBuf buf = Markers.newReplicatedSubscriptionsSnapshot("sid", "us-west", 5, 7, clusters);
+
+ MessageMetadata msgMetadata = Commands.parseMessageMetadata(buf);
+ assertEquals(msgMetadata.getReplicateToCount(), 1);
+ assertEquals(msgMetadata.getReplicateTo(0), "us-west");
+
+ ReplicatedSubscriptionsSnapshot snapshot = Markers.parseReplicatedSubscriptionsSnapshot(buf);
+
+ assertEquals(snapshot.getSnapshotId(), "sid");
+
+ assertEquals(snapshot.getLocalMessageId().getLedgerId(), 5);
+ assertEquals(snapshot.getLocalMessageId().getEntryId(), 7);
+
+ assertEquals(snapshot.getClustersCount(), 2);
+ assertEquals(snapshot.getClusters(0).getCluster(), "us-cent");
+ assertEquals(snapshot.getClusters(0).getMessageId().getLedgerId(), 20);
+ assertEquals(snapshot.getClusters(0).getMessageId().getEntryId(), 21);
+ assertEquals(snapshot.getClusters(1).getCluster(), "us-east");
+ assertEquals(snapshot.getClusters(1).getMessageId().getLedgerId(), 10);
+ assertEquals(snapshot.getClusters(1).getMessageId().getEntryId(), 11);
+ }
+
+ @Test
+ public void testUpdate() throws Exception {
+ Map clusters = new TreeMap<>();
+ clusters.put("us-east", MessageIdData.newBuilder().setLedgerId(10).setEntryId(11).build());
+ clusters.put("us-cent", MessageIdData.newBuilder().setLedgerId(20).setEntryId(21).build());
+
+ ByteBuf buf = Markers.newReplicatedSubscriptionsUpdate("sub-1", clusters);
+
+ MessageMetadata msgMetadata = Commands.parseMessageMetadata(buf);
+ assertEquals(msgMetadata.getReplicateToCount(), 0);
+
+ ReplicatedSubscriptionsUpdate snapshot = Markers.parseReplicatedSubscriptionsUpdate(buf);
+
+ assertEquals(snapshot.getSubscriptionName(), "sub-1");
+
+ assertEquals(snapshot.getClustersCount(), 2);
+ assertEquals(snapshot.getClusters(0).getCluster(), "us-cent");
+ assertEquals(snapshot.getClusters(0).getMessageId().getLedgerId(), 20);
+ assertEquals(snapshot.getClusters(0).getMessageId().getEntryId(), 21);
+ assertEquals(snapshot.getClusters(1).getCluster(), "us-east");
+ assertEquals(snapshot.getClusters(1).getMessageId().getLedgerId(), 10);
+ assertEquals(snapshot.getClusters(1).getMessageId().getEntryId(), 11);
+ }
+
+}