From f69fc22983033c72540bb30f45790e1bb1468381 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Tue, 14 Jan 2020 21:21:35 +0800 Subject: [PATCH 01/21] Add support for batch local index ack --- conf/broker.conf | 3 + .../bookkeeper/mledger/ManagedCursor.java | 21 + .../mledger/ManagedLedgerConfig.java | 8 + .../mledger/impl/ManagedCursorImpl.java | 150 +++ .../src/main/proto/MLDataFormats.proto | 18 +- .../impl/ManagedCursorContainerTest.java | 10 + .../mledger/impl/ManagedCursorTest.java | 107 +++ .../pulsar/broker/ServiceConfiguration.java | 3 + .../service/AbstractBaseDispatcher.java | 19 +- .../pulsar/broker/service/Consumer.java | 36 +- .../broker/service/EntryBatchIndexesAcks.java | 71 ++ .../pulsar/broker/service/Subscription.java | 3 +- ...PersistentDispatcherMultipleConsumers.java | 4 +- ...sistentDispatcherSingleActiveConsumer.java | 4 +- ...tStickyKeyDispatcherMultipleConsumers.java | 4 +- .../NonPersistentSubscription.java | 3 +- .../persistent/CompactorSubscription.java | 4 +- ...PersistentDispatcherMultipleConsumers.java | 8 +- ...sistentDispatcherSingleActiveConsumer.java | 6 +- ...tStickyKeyDispatcherMultipleConsumers.java | 8 +- .../persistent/PersistentSubscription.java | 33 +- .../service/persistent/PersistentTopic.java | 7 +- .../ReplicatedSubscriptionsController.java | 2 +- .../pulsar/client/impl/RawReaderImpl.java | 3 +- .../broker/service/PersistentTopicTest.java | 2 +- .../PersistentSubscriptionTest.java | 2 +- .../impl/BatchMessageIndexAckDisableTest.java | 148 +++ .../client/impl/BatchMessageIndexAckTest.java | 183 ++++ .../impl/CompactedOutBatchMessageTest.java | 2 +- .../impl/AcknowledgmentsGroupingTracker.java | 2 + .../client/impl/BatchMessageIdImpl.java | 8 +- .../apache/pulsar/client/impl/ClientCnx.java | 2 +- .../pulsar/client/impl/ConsumerImpl.java | 29 +- ...rsistentAcknowledgmentGroupingTracker.java | 5 + ...sistentAcknowledgmentsGroupingTracker.java | 74 +- .../client/impl/ZeroQueueConsumerImpl.java | 3 + .../pulsar/client/impl/KeySharedConsumer.java | 35 + .../client/impl/KeySharedConsumer1.java | 35 + .../client/impl/KeySharedConsumer2.java | 35 + .../pulsar/client/impl/KeySharedProducer.java | 33 + .../apache/pulsar/client/util/HashTest.java | 22 + .../pulsar/common/api/proto/PulsarApi.java | 905 ++++++++++++++++++ .../pulsar/common/protocol/Commands.java | 107 ++- pulsar-common/src/main/proto/PulsarApi.proto | 9 + 44 files changed, 2125 insertions(+), 51 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 8aebf1038716f..dddd8dded0939 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -325,6 +325,9 @@ delayedDeliveryEnabled=true # Default is 1 second. delayedDeliveryTickTimeMillis=1000 +# Whether to enable acknowledge of batch local index. +batchIndexAcknowledgeEnable = true + # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 226b5a05702c8..e02eebd0443df 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; /** * A ManangedCursor is a persisted cursor inside a ManagedLedger. @@ -320,6 +321,21 @@ void markDelete(Position position, Map properties) */ void asyncDelete(Iterable position, DeleteCallback callback, Object ctx); + /** + * Delete a group of batch indexes in a batch message asynchronously + * + *

+ * Mark a group of batch indexed for deletion. When all indexes of a batch message are all deleted, then will mark the batch message + * for deletion + * + * @param position the position of the message to deleted + * @param batchSize batch size of the batch message + * @param deleteIndexRanges delete index ranges for a batch message + * @param callback callback object + * @param ctx opaque context + */ + void asyncDelete(Position position, int batchSize, List deleteIndexRanges, DeleteCallback callback, Object ctx); + /** * Get the read position. This points to the next message to be read from the cursor. * @@ -609,4 +625,9 @@ Set asyncReplayEntries( * Trim delete entries for the given entries */ void trimDeletedEntries(List entries); + + /** + * Get deleted batch indexes list for a batch message. + */ + List getDeletedBatchIndexes(Position position); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index bf23fe719a547..56dd9064d7c5f 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -42,6 +42,7 @@ public class ManagedLedgerConfig { private boolean createIfMissing = true; private int maxUnackedRangesToPersist = 10000; + private int maxBatchDeletedIndexToPersist = 10000; private int maxUnackedRangesToPersistInZk = 1000; private int maxEntriesPerLedger = 50000; private int maxSizePerLedgerMb = 100; @@ -430,6 +431,13 @@ public int getMaxUnackedRangesToPersist() { return maxUnackedRangesToPersist; } + /** + * @return max batch deleted index that will be persisted and recoverd. + */ + public int getMaxBatchDeletedIndexToPersist() { + return maxBatchDeletedIndexToPersist; + } + /** * @param maxUnackedRangesToPersist * max unacked message ranges that will be persisted and receverd. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index e47c000d9d23c..c431d997f5331 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -43,11 +43,13 @@ import java.time.Clock; import java.util.ArrayDeque; +import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -90,6 +92,8 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; +import org.apache.pulsar.common.util.collections.ConcurrentBitSet; import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; @@ -156,6 +160,10 @@ public class ManagedCursorImpl implements ManagedCursor { return position; }; private final LongPairRangeSet individualDeletedMessages; + + // Maintain the indexes deleted status of batch messages that not deleted completely + // (ledgerId, entryId, batchSize) -> deleted indexes + private final ConcurrentHashMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -227,6 +235,7 @@ public interface VoidCallback { this.individualDeletedMessages = config.isUnackedRangesOpenCacheSetEnabled() ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); + this.batchDeletedIndexes = new ConcurrentHashMap<>(); this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType()); STATE_UPDATER.set(this, State.Uninitialized); PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0); @@ -374,6 +383,9 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } + if (positionInfo.getBatchDeletedIndexesCount() > 0) { + recoverBatchDeletedIndexes(positionInfo.getBatchDeletedIndexesList()); + } recoveredCursor(position, recoveredProperties, lh); callback.operationComplete(); }, null); @@ -393,6 +405,24 @@ private void recoverIndividualDeletedMessages(List i } } + private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList) { + lock.writeLock().lock(); + try { + this.batchDeletedIndexes.clear(); + batchDeletedIndexInfoList.forEach(batchDeletedIndexInfo -> { + if (batchDeletedIndexInfo.getBatchDeletedIndexesCount() > 0) { + ConcurrentBitSet value = new ConcurrentBitSet(0); + batchDeletedIndexInfo.getBatchDeletedIndexesList().forEach(deletedIndexRange -> + value.set(deletedIndexRange.getLowerIndex(), deletedIndexRange.getUpperIndex())); + this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(), + batchDeletedIndexInfo.getPosition().getEntryId()), value); + } + }); + } finally { + lock.writeLock().unlock(); + } + } + private void recoveredCursor(PositionImpl position, Map properties, LedgerHandle recoveredFromCursorLedger) { // if the position was at a ledger that didn't exist (since it will be deleted if it was previously empty), @@ -902,6 +932,7 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, Collections.emptyMap(), null, null); individualDeletedMessages.clear(); + batchDeletedIndexes.clear(); PositionImpl oldReadPosition = readPosition; if (oldReadPosition.compareTo(newPosition) >= 0) { @@ -1582,6 +1613,9 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); + batchDeletedIndexes.entrySet().removeIf(entry -> + entry.getKey().compareTo( + PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId())) <= 0); } finally { lock.writeLock().unlock(); } @@ -1723,6 +1757,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb continue; } + batchDeletedIndexes.remove(position); + // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make // the RangeSet recognize the "continuity" between adjacent Positions PositionImpl previousPosition = ledger.getPreviousPosition(position); @@ -1808,6 +1844,58 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { } } + @Override + public void asyncDelete(Position position, int batchSize, List deleteIndexRanges, AsyncCallbacks.DeleteCallback callback, Object ctx) { + if (isClosed()) { + callback.deleteFailed(new ManagedLedgerException("Cursor was already closed"), null); + return; + } + lock.writeLock().lock(); + try { + + ConcurrentBitSet bitSet; + PositionImpl pos = (PositionImpl) checkNotNull(position); + if (markDeletePosition.compareTo(pos) >= 0 || individualDeletedMessages.contains(pos.ledgerId, pos.entryId)) { + batchDeletedIndexes.remove(pos); + callback.deleteComplete(ctx); + return; + } + + if (!batchDeletedIndexes.containsKey(pos)) { + bitSet = new ConcurrentBitSet(batchSize); + batchDeletedIndexes.put(pos, bitSet); + } else { + bitSet = batchDeletedIndexes.get(pos); + } + + for (IntRange deleteIndexRange : deleteIndexRanges) { + bitSet.set(deleteIndexRange.getStart(), deleteIndexRange.getEnd() + 1); + } + + if (bitSet.nextClearBit(0) == batchSize) { + asyncDelete(pos, new AsyncCallbacks.DeleteCallback() { + @Override + public void deleteComplete(Object ctx) { + callback.deleteComplete(ctx); + } + + @Override + public void deleteFailed(ManagedLedgerException exception, Object ctx) { + callback.deleteFailed(getManagedLedgerException(exception), ctx); + } + }, ctx); + } else { + callback.deleteComplete(ctx); + } + } catch (Exception e) { + log.warn("[{}] [{}] Error while updating batchDeletedMessages [{}]", ledger.getName(), name, + e.getMessage(), e); + callback.deleteFailed(getManagedLedgerException(e), ctx); + } finally { + lock.writeLock().unlock(); + } + } + /** * Given a list of entries, filter out the entries that have already been individually deleted. * @@ -2044,6 +2132,7 @@ private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl positio info.addAllProperties(buildPropertiesMap(properties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); + info.addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()); } if (log.isDebugEnabled()) { @@ -2288,11 +2377,49 @@ private List buildIndividualDeletedMessageRanges() { } } + private List buildBatchDeletedIndexInfoList() { + lock.readLock().lock(); + try { + if (batchDeletedIndexes.isEmpty()) { + return Collections.emptyList(); + } + MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo + .newBuilder(); + MLDataFormats.BatchDeletedIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchDeletedIndexInfo + .newBuilder(); + MLDataFormats.BatchDeletedIndexRange.Builder batchDeletedIndexRangeBuilder = MLDataFormats.BatchDeletedIndexRange + .newBuilder(); + List result = Lists.newArrayList(); + for (Map.Entry entry : batchDeletedIndexes.entrySet()) { + nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); + nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); + batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); + BitSet bitSet = entry.getValue(); + int nextSetBit = bitSet.nextSetBit(0); + while (nextSetBit != -1) { + int nextClearBit = bitSet.nextClearBit(nextSetBit); + batchDeletedIndexRangeBuilder.setLowerIndex(nextSetBit); + batchDeletedIndexRangeBuilder.setUpperIndex(nextClearBit); + batchDeletedIndexInfoBuilder.addBatchDeletedIndexes(batchDeletedIndexRangeBuilder.build()); + nextSetBit = bitSet.nextSetBit(nextClearBit); + } + result.add(batchDeletedIndexInfoBuilder.build()); + if (result.size() >= config.getMaxBatchDeletedIndexToPersist()) { + break; + } + } + return result; + } finally { + lock.readLock().unlock(); + } + } + void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { PositionImpl position = mdEntry.newPosition; PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) + .addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()) .addAllProperties(buildPropertiesMap(mdEntry.properties)).build(); @@ -2671,8 +2798,31 @@ public void trimDeletedEntries(List entries) { || individualDeletedMessages.contains(entry.getLedgerId(), entry.getEntryId())); } +<<<<<<< HEAD private ManagedCursorImpl cursorImpl() { return this; +======= + @Override + public List getDeletedBatchIndexes(Position position) { + if (batchDeletedIndexes.isEmpty()) { + return null; + } + PositionImpl pos = (PositionImpl) checkNotNull(position); + if (batchDeletedIndexes.containsKey(pos)) { + List result = Lists.newArrayList(); + BitSet bitSet = batchDeletedIndexes.get(pos); + int nextSetBit = bitSet.nextSetBit(0); + IntRange.Builder builder = IntRange.newBuilder(); + while (nextSetBit != -1) { + int nextClearBit = bitSet.nextClearBit(nextSetBit); + result.add(builder.setStart(nextSetBit).setEnd(nextClearBit - 1).build()); + nextSetBit = bitSet.nextSetBit(nextClearBit); + } + return result; + } else { + return null; + } +>>>>>>> Add support for batch local index ack } private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 4dbd23154de71..2c12a950b6004 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -66,6 +66,9 @@ message PositionInfo { // Additional custom properties associated with // the current cursor position repeated LongProperty properties = 4; + + // Store which index in the batch message has been deleted + repeated BatchDeletedIndexInfo batchDeletedIndexes = 5; } message NestedPositionInfo { @@ -78,6 +81,16 @@ message MessageRange { required NestedPositionInfo upperEndpoint = 2; } +message BatchDeletedIndexRange { + required uint32 lowerIndex = 1; + required uint32 upperIndex = 2; +} + +message BatchDeletedIndexInfo { + required NestedPositionInfo position = 1; + repeated BatchDeletedIndexRange batchDeletedIndexes = 2; +} + // Generic string and long tuple message LongProperty { required string name = 1; @@ -98,5 +111,8 @@ message ManagedCursorInfo { // the current cursor position repeated LongProperty properties = 5; - optional int64 lastActive = 6; + optional int64 lastActive = 6; + + // Store which index in the batch message has been deleted + repeated BatchDeletedIndexInfo batchDeletedIndexes = 7; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 99e7aec740bed..d781e1c2440e8 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -44,6 +44,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.testng.annotations.Test; @Test @@ -179,6 +180,10 @@ public void delete(Iterable positions) throws InterruptedException, Ma public void asyncDelete(Iterable position, DeleteCallback callback, Object ctx) { } + @Override + public void asyncDelete(Position position, int batchSize, List deleteIndexRanges, DeleteCallback callback, Object ctx) { + } + @Override public void clearBacklog() throws InterruptedException, ManagedLedgerException { } @@ -326,6 +331,11 @@ public void trimDeletedEntries(List entries) { } + @Override + public List getDeletedBatchIndexes(Position position) { + return null; + } + } @Test diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 7884533bc764c..9eac3bc848946 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -81,11 +81,13 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.metadata.api.Stat; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.zookeeper.KeeperException.Code; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -3050,5 +3052,110 @@ public void deleteMessagesCheckhMarkDelete() throws Exception { assertEquals(c1.getReadPosition(), positions[markDelete + 1]); } + @Test + public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedException { + ManagedLedger ledger = factory.open("test_batch_index_delete"); + ManagedCursor cursor = ledger.openCursor("c1"); + + final int totalEntries = 100; + final Position[] positions = new Position[totalEntries]; + for (int i = 0; i < totalEntries; i++) { + // add entry + positions[i] = ledger.addEntry(("entry-" + i).getBytes(Encoding)); + } + assertEquals(cursor.getNumberOfEntries(), totalEntries); + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(2).setEnd(4).build())); + List deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + Assert.assertEquals(1, deletedIndexes.size()); + Assert.assertEquals(2, deletedIndexes.get(0).getStart()); + Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); + + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(3).setEnd(8).build())); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + Assert.assertEquals(1, deletedIndexes.size()); + Assert.assertEquals(2, deletedIndexes.get(0).getStart()); + Assert.assertEquals(8, deletedIndexes.get(0).getEnd()); + + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(0).build())); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + Assert.assertEquals(2, deletedIndexes.size()); + Assert.assertEquals(0, deletedIndexes.get(0).getStart()); + Assert.assertEquals(0, deletedIndexes.get(0).getEnd()); + Assert.assertEquals(2, deletedIndexes.get(1).getStart()); + Assert.assertEquals(8, deletedIndexes.get(1).getEnd()); + + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(1).setEnd(1).build())); + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(9).setEnd(9).build())); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + Assert.assertNull(deletedIndexes); + Assert.assertEquals(positions[0], cursor.getMarkDeletedPosition()); + + deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); + cursor.delete(positions[1]); + deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(6).setEnd(8).build())); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[1]); + Assert.assertNull(deletedIndexes); + + deleteBatchIndex(cursor, positions[2], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); + cursor.markDelete(positions[3]); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[2]); + Assert.assertNull(deletedIndexes); + + deleteBatchIndex(cursor, positions[3], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); + cursor.resetCursor(positions[0]); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[3]); + Assert.assertNull(deletedIndexes); + } + + @Test + public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerException, InterruptedException { + ManagedLedger ledger = factory.open("test_batch_indexes_deletion_persistent"); + ManagedCursor cursor = ledger.openCursor("c1"); + + final int totalEntries = 100; + final Position[] positions = new Position[totalEntries]; + for (int i = 0; i < totalEntries; i++) { + // add entry + positions[i] = ledger.addEntry(("entry-" + i).getBytes(Encoding)); + } + assertEquals(cursor.getNumberOfEntries(), totalEntries); + deleteBatchIndex(cursor, positions[5], 10, Lists.newArrayList(IntRange.newBuilder().setStart(3).setEnd(6).build())); + deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + deleteBatchIndex(cursor, positions[2], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + deleteBatchIndex(cursor, positions[3], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + deleteBatchIndex(cursor, positions[4], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + + ledger = factory.open("test_batch_indexes_deletion_persistent"); + cursor = ledger.openCursor("c1"); + List deletedIndexes = cursor.getDeletedBatchIndexes(positions[5]); + Assert.assertEquals(deletedIndexes.size(), 1); + Assert.assertEquals(deletedIndexes.get(0).getStart(), 3); + Assert.assertEquals(deletedIndexes.get(0).getEnd(), 6); + Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[4]); + deleteBatchIndex(cursor, positions[5], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); + deletedIndexes = cursor.getDeletedBatchIndexes(positions[5]); + Assert.assertNull(deletedIndexes); + Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[5]); + } + + private void deleteBatchIndex(ManagedCursor cursor, Position position, int batchSize, + List deleteIndexes) throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + cursor.asyncDelete(position, batchSize, deleteIndexes, + new DeleteCallback() { + @Override + public void deleteComplete(Object ctx) { + latch.countDown(); + } + + @Override + public void deleteFailed(ManagedLedgerException exception, Object ctx) { + latch.countDown(); + } + }, null); + latch.await(); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 030e83893ef9e..124a8f368e4c7 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -173,6 +173,9 @@ public class ServiceConfiguration implements PulsarConfiguration { + " affecting the accuracy of the delivery time compared to the scheduled time. Default is 1 second.") private long delayedDeliveryTickTimeMillis = 1000; + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the acknowledge of batch local index") + private boolean batchIndexAcknowledgeEnable = true; + @FieldContext( category = CATEGORY_WEBSOCKET, doc = "Enable the WebSocket API service in broker" 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 c8da4209033f5..fa1500008594d 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,9 +27,12 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.PulsarApi; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; @@ -66,7 +69,7 @@ protected AbstractBaseDispatcher(Subscription subscription) { * an object where the total size in messages and bytes will be returned back to the caller */ public void filterEntriesForConsumer(List entries, EntryBatchSizes batchSizes, - SendMessageInfo sendMessageInfo) { + SendMessageInfo sendMessageInfo, EntryBatchIndexesAcks indexesAcks, ManagedCursor cursor) { int totalMessages = 0; long totalBytes = 0; @@ -87,7 +90,7 @@ public void filterEntriesForConsumer(List entries, EntryBatchSizes batchS entries.set(i, null); entry.release(); - subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, + subscription.acknowledgeMessage(Collections.singletonList(pos), null, AckType.Individual, Collections.emptyMap()); continue; } else if (msgMetadata.hasDeliverAtTime() @@ -102,6 +105,18 @@ && trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { totalMessages += batchSize; totalBytes += metadataAndPayload.readableBytes(); batchSizes.setBatchSize(i, batchSize); + if (indexesAcks != null && cursor != null) { + List ranges = cursor.getDeletedBatchIndexes(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); + if (ranges != null) { + int indexesAckedCount = 0; + for (IntRange range : ranges) { + indexesAckedCount += range.getEnd() - range.getStart() + 1; + } + indexesAcks.setIndexesAcks(Pair.of(indexesAckedCount, ranges)); + } else { + indexesAcks.setIndexesAcks(null); + } + } } finally { msgMetadata.recycle(); } 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 786084bae0340..3bf55a760f773 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 @@ -49,6 +49,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; import org.apache.pulsar.common.api.proto.PulsarApi.ProtocolVersion; import org.apache.pulsar.common.naming.TopicName; @@ -189,8 +190,9 @@ public boolean readCompacted() { * * @return a SendMessageInfo object that contains the detail of what was sent to consumer */ - public ChannelPromise sendMessages(final List entries, EntryBatchSizes batchSizes, int totalMessages, - long totalBytes, RedeliveryTracker redeliveryTracker) { + + public ChannelPromise sendMessages(final List entries, EntryBatchSizes batchSizes, EntryBatchIndexesAcks batchIndexesAcks, + int totalMessages, long totalBytes, RedeliveryTracker redeliveryTracker) { this.lastConsumedTimestamp = System.currentTimeMillis(); final ChannelHandlerContext ctx = cnx.ctx(); final ChannelPromise writePromise = ctx.newPromise(); @@ -202,6 +204,9 @@ public ChannelPromise sendMessages(final List entries, EntryBatchSizes ba } writePromise.setSuccess(); batchSizes.recyle(); + if (batchIndexesAcks != null) { + batchIndexesAcks.recycle(); + } return writePromise; } @@ -219,7 +224,8 @@ public ChannelPromise sendMessages(final List entries, EntryBatchSizes ba } // reduce permit and increment unackedMsg count with total number of messages in batch-msgs - MESSAGE_PERMITS_UPDATER.addAndGet(this, -totalMessages); + int ackedCount = batchIndexesAcks == null ? 0 : batchIndexesAcks.getTotalAckedIndexCount(); + MESSAGE_PERMITS_UPDATER.addAndGet(this, ackedCount - totalMessages); incrementUnackedMessages(totalMessages); msgOut.recordMultipleEvents(totalMessages, totalBytes); @@ -267,7 +273,12 @@ public ChannelPromise sendMessages(final List entries, EntryBatchSizes ba if (redeliveryTracker.contains(position)) { redeliveryCount = redeliveryTracker.incrementAndGetRedeliveryCount(position); } - ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload), ctx.voidPromise()); + + List ackedIndexRanges = null; + if (batchIndexesAcks != null && batchIndexesAcks.getIndexesAcks(i) != null) { + ackedIndexRanges = batchIndexesAcks.getIndexesAcks(i).getRight(); + } + ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload, ackedIndexRanges), ctx.voidPromise()); messageId.recycle(); messageIdBuilder.recycle(); entry.release(); @@ -276,6 +287,9 @@ public ChannelPromise sendMessages(final List entries, EntryBatchSizes ba // Use an empty write here so that we can just tie the flush with the write promise for last entry ctx.writeAndFlush(Unpooled.EMPTY_BUFFER, writePromise); batchSizes.recyle(); + if (batchIndexesAcks != null) { + batchIndexesAcks.recycle(); + } }); return writePromise; @@ -350,7 +364,7 @@ void messageAcked(CommandAck ack) { } if (ack.getAckType() == AckType.Cumulative) { - if (ack.getMessageIdCount() != 1) { + if (ack.getMessageIdCount() != 1 && ack.getBatchMessageAckIndexesCount() != 1) { log.warn("[{}] [{}] Received multi-message ack", subscription, consumerId); return; } @@ -359,10 +373,12 @@ void messageAcked(CommandAck ack) { log.warn("[{}] [{}] Received cumulative ack on shared subscription, ignoring", subscription, consumerId); return; } - - MessageIdData msgId = ack.getMessageId(0); - PositionImpl position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); - subscription.acknowledgeMessage(Collections.singletonList(position), AckType.Cumulative, properties); + PositionImpl position = PositionImpl.earliest; + if (ack.getMessageIdCount() == 1) { + MessageIdData msgId = ack.getMessageId(0); + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + } + subscription.acknowledgeMessage(Collections.singletonList(position), ack.getBatchMessageAckIndexesList(), AckType.Cumulative, properties); } else { // Individual ack List positionsAcked = new ArrayList<>(); @@ -380,7 +396,7 @@ void messageAcked(CommandAck ack) { consumerId, position, ack.getValidationError()); } } - subscription.acknowledgeMessage(positionsAcked, AckType.Individual, properties); + subscription.acknowledgeMessage(positionsAcked, ack.getBatchMessageAckIndexesList(), AckType.Individual, properties); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java new file mode 100644 index 0000000000000..f0ca64ed9ba4c --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java @@ -0,0 +1,71 @@ +/** + * 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.broker.service; + + +import io.netty.util.Recycler; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; + +import java.util.ArrayList; +import java.util.List; + +public class EntryBatchIndexesAcks { + + List>> indexesAcks = new ArrayList<>(); + + public void setIndexesAcks(Pair> indexesAcks) { + this.indexesAcks.add(indexesAcks); + } + + public Pair> getIndexesAcks(int entryIdx) { + return this.indexesAcks.get(entryIdx); + } + + public int getTotalAckedIndexCount() { + int count = 0; + for (Pair> pair : indexesAcks) { + if (pair != null) { + count += pair.getLeft(); + } + } + return count; + } + + public void recycle() { + this.indexesAcks.clear(); + handle.recycle(this); + } + + public static EntryBatchIndexesAcks get() { + return RECYCLER.get(); + } + + private EntryBatchIndexesAcks(Recycler.Handle handle) { + this.handle = handle; + } + + private final Recycler.Handle handle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected EntryBatchIndexesAcks newObject(Handle handle) { + return new EntryBatchIndexesAcks(handle); + } + }; +} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 85b54152ad9db..1c915852f6aca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -25,6 +25,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot; @@ -45,7 +46,7 @@ default void removeConsumer(Consumer consumer) throws BrokerServiceException { void consumerFlow(Consumer consumer, int additionalNumberOfMessages); - void acknowledgeMessage(List positions, AckType ackType, Map properties); + void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties); String getTopicName(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java index 08f5f4c54f0a4..d97e0db28a6b7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherMultipleConsumers.java @@ -201,8 +201,8 @@ public void sendMessages(List entries) { if (consumer != null) { SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(entries.size()); - filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); - consumer.sendMessages(entries, batchSizes, sendMessageInfo.getTotalMessages(), + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo, null, null); + consumer.sendMessages(entries, batchSizes, null, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), getRedeliveryTracker()); TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java index e1c2e58d4aa81..7db50cc8698e6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentDispatcherSingleActiveConsumer.java @@ -62,8 +62,8 @@ public void sendMessages(List entries) { if (currentConsumer != null && currentConsumer.getAvailablePermits() > 0 && currentConsumer.isWritable()) { SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(entries.size()); - filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); - currentConsumer.sendMessages(entries, batchSizes, sendMessageInfo.getTotalMessages(), + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo, null, null); + currentConsumer.sendMessages(entries, batchSizes, null, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), getRedeliveryTracker()); } else { entries.forEach(entry -> { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java index 3fdad35a49c60..57072b9425dc5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentStickyKeyDispatcherMultipleConsumers.java @@ -79,8 +79,8 @@ public void sendMessages(List entries) { if (consumer != null) { SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesWithSameKey.getValue().size()); - filterEntriesForConsumer(entriesWithSameKey.getValue(), batchSizes, sendMessageInfo); - consumer.sendMessages(entriesWithSameKey.getValue(), batchSizes, sendMessageInfo.getTotalMessages(), + filterEntriesForConsumer(entriesWithSameKey.getValue(), batchSizes, sendMessageInfo, null, null); + consumer.sendMessages(entriesWithSameKey.getValue(), batchSizes, null, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), getRedeliveryTracker()); TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages()); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index 73152c32370ed..e1fd8f1777b23 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -38,6 +38,7 @@ import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; @@ -185,7 +186,7 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { } @Override - public void acknowledgeMessage(List position, AckType ackType, Map properties) { + public void acknowledgeMessage(List position, List batchMessageAckIndexes, AckType ackType, Map properties) { // No-op } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java index 5d39c20a2441d..b0d64ca7c423d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java @@ -28,8 +28,8 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.compaction.CompactedTopic; import org.apache.pulsar.compaction.Compactor; @@ -57,7 +57,7 @@ public CompactorSubscription(PersistentTopic topic, CompactedTopic compactedTopi } @Override - public void acknowledgeMessage(List positions, AckType ackType, Map properties) { + public void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties) { checkArgument(ackType == AckType.Cumulative); checkArgument(positions.size() == 1); checkArgument(properties.containsKey(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java index 74d131f6aff3c..7e3e85b60edd8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java @@ -51,6 +51,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.InMemoryRedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTracker; @@ -506,15 +507,16 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { List entriesForThisConsumer = entries.subList(start, start + messagesForC); EntryBatchSizes batchSizes = EntryBatchSizes.get(entriesForThisConsumer.size()); - filterEntriesForConsumer(entriesForThisConsumer, batchSizes, sendMessageInfo); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(); + filterEntriesForConsumer(entriesForThisConsumer, batchSizes, sendMessageInfo, batchIndexesAcks, cursor); - c.sendMessages(entriesForThisConsumer, batchSizes, sendMessageInfo.getTotalMessages(), + c.sendMessages(entriesForThisConsumer, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), redeliveryTracker); int msgSent = sendMessageInfo.getTotalMessages(); start += messagesForC; entriesToDispatch -= messagesForC; - TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -msgSent); + TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -(msgSent - batchIndexesAcks.getTotalAckedIndexCount())); totalMessagesSent += sendMessageInfo.getTotalMessages(); totalBytesSent += sendMessageInfo.getTotalBytes(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 4ad0f5ab28217..9b0699c8425f9 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -42,6 +42,7 @@ import org.apache.pulsar.broker.service.AbstractDispatcherSingleActiveConsumer; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.RedeliveryTracker; import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled; @@ -237,13 +238,14 @@ public synchronized void internalReadEntriesComplete(final List entries, } else { EntryBatchSizes batchSizes = EntryBatchSizes.get(entries.size()); SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); - filterEntriesForConsumer(entries, batchSizes, sendMessageInfo); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(); + filterEntriesForConsumer(entries, batchSizes, sendMessageInfo, batchIndexesAcks, cursor); int totalMessages = sendMessageInfo.getTotalMessages(); long totalBytes = sendMessageInfo.getTotalBytes(); currentConsumer - .sendMessages(entries, batchSizes, sendMessageInfo.getTotalMessages(), + .sendMessages(entries, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), redeliveryTracker) .addListener(future -> { if (future.isSuccess()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java index c651782920ef2..fc938fa24d6c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumers.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.service.EntryBatchIndexesAcks; import org.apache.pulsar.broker.service.EntryBatchSizes; import org.apache.pulsar.broker.service.SendMessageInfo; import org.apache.pulsar.broker.service.StickyKeyConsumerSelector; @@ -111,9 +112,10 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal(); EntryBatchSizes batchSizes = EntryBatchSizes.get(subList.size()); - filterEntriesForConsumer(subList, batchSizes, sendMessageInfo); + EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(); + filterEntriesForConsumer(subList, batchSizes, sendMessageInfo, batchIndexesAcks, cursor); - consumer.sendMessages(subList, batchSizes, sendMessageInfo.getTotalMessages(), + consumer.sendMessages(subList, batchSizes, batchIndexesAcks, sendMessageInfo.getTotalMessages(), sendMessageInfo.getTotalBytes(), getRedeliveryTracker()).addListener(future -> { if (future.isSuccess() && keyNumbers.decrementAndGet() == 0) { readMoreEntries(); @@ -121,7 +123,7 @@ protected void sendMessagesToConsumers(ReadType readType, List entries) { }); entriesWithSameKey.getValue().removeAll(subList); - TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, -sendMessageInfo.getTotalMessages()); + TOTAL_AVAILABLE_PERMITS_UPDATER.getAndAdd(this, -(sendMessageInfo.getTotalMessages() - batchIndexesAcks.getTotalAckedIndexCount())); totalMessagesSent += sendMessageInfo.getTotalMessages(); totalBytesSent += sendMessageInfo.getTotalBytes(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index c5b7e4c78e2db..2544a013930f6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -48,6 +48,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap; +import org.apache.commons.collections.CollectionUtils; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; @@ -59,6 +60,7 @@ import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot; @@ -106,6 +108,8 @@ public class PersistentSubscription implements Subscription { // This parameter only keep the the largest Position it cumulative ack,as any Position smaller will also be covered. private volatile Position pendingCumulativeAckMessage; + private boolean enableBatchLocalIndexAck = false; + private static final AtomicReferenceFieldUpdater POSITION_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentSubscription.class, Position.class, "pendingCumulativeAckMessage"); @@ -140,11 +144,17 @@ static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) { } public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - boolean replicated) { + boolean replicated) { + this(topic, subscriptionName, cursor, replicated, false); + } + + public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, + boolean replicated, boolean enableBatchLocalIndexAck) { this.topic = topic; this.cursor = cursor; this.topicName = topic.getName(); this.subName = subscriptionName; + this.enableBatchLocalIndexAck = enableBatchLocalIndexAck; this.fullName = MoreObjects.toStringHelper(this).add("topic", topicName).add("name", subName).toString();; this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, subscriptionName, cursor); this.setReplicated(replicated); @@ -321,7 +331,7 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { } @Override - public void acknowledgeMessage(List positions, AckType ackType, Map properties) { + public void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties) { Position previousMarkDeletePosition = cursor.getMarkDeletedPosition(); if (ackType == AckType.Cumulative) { @@ -342,6 +352,16 @@ public void acknowledgeMessage(List positions, AckType ackType, Map positions, AckType ackType, Map cursor.asyncDelete( + PositionImpl.get(ackedIndex.getMessageId().getLedgerId(), ackedIndex.getMessageId().getEntryId()), + ackedIndex.getBatchSize(), + ackedIndex.getAckIndexesList(), + deleteCallback, + null)); + } } if (!cursor.getMarkDeletedPosition().equals(previousMarkDeletePosition)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 808620923d7e2..67c911ff49f28 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -312,7 +312,8 @@ private PersistentSubscription createPersistentSubscription(String subscriptionN if (subscriptionName.equals(Compactor.COMPACTION_SUBSCRIPTION)) { return new CompactorSubscription(this, compactedTopic, subscriptionName, cursor); } else { - return new PersistentSubscription(this, subscriptionName, cursor, replicated); + return new PersistentSubscription(this, subscriptionName, cursor, replicated, + brokerService.getPulsar().getConfiguration().isBatchIndexAcknowledgeEnable()); } } @@ -718,8 +719,8 @@ private CompletableFuture getNonDurableSubscription(Stri } catch (ManagedLedgerException e) { subscriptionFuture.completeExceptionally(e); } - - return new PersistentSubscription(this, subscriptionName, cursor, false); + return new PersistentSubscription(this, subscriptionName, cursor, false, + brokerService.getPulsar().getConfiguration().isBatchIndexAcknowledgeEnable()); }); if (!subscriptionFuture.isDone()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index 5a8875770451a..c2c13d4be935b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -174,7 +174,7 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { PersistentSubscription sub = topic.getSubscription(update.getSubscriptionName()); if (sub != null) { - sub.acknowledgeMessage(Collections.singletonList(pos), AckType.Cumulative, Collections.emptyMap()); + sub.acknowledgeMessage(Collections.singletonList(pos), null, AckType.Cumulative, Collections.emptyMap()); } else { // Subscription doesn't exist. We need to force the creation of the subscription in this cluster, because log.info("[{}][{}] Creating subscription at {}:{} after receiving update from replicated subcription", diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java index 3ca072dda51aa..ec622c828ff7a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java @@ -35,6 +35,7 @@ import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.protocol.Commands; @@ -203,7 +204,7 @@ public CompletableFuture closeAsync() { } @Override - void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf headersAndPayload, ClientCnx cnx) { + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received raw message: {}/{}/{}", topic, subscription, messageId.getEntryId(), messageId.getLedgerId(), messageId.getPartition()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index 37cdec6371c66..e040a5721d4e8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -1417,7 +1417,7 @@ public void testCompactorSubscription() throws Exception { cursorMock); PositionImpl position = new PositionImpl(1, 1); long ledgerId = 0xc0bfefeL; - sub.acknowledgeMessage(Collections.singletonList(position), AckType.Cumulative, + sub.acknowledgeMessage(Collections.singletonList(position), null, AckType.Cumulative, ImmutableMap.of(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY, ledgerId)); verify(compactedTopic, Mockito.times(1)).newCompactedLedger(position, ledgerId); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java index 038b22df7dad5..34bbb5b769b79 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java @@ -273,7 +273,7 @@ public void testCanAcknowledgeAndAbortForTransaction() throws TransactionConflic positions.add(new PositionImpl(3, 5)); // Acknowledge from normal consumer will succeed ignoring message acked by ongoing transaction. - persistentSubscription.acknowledgeMessage(positions, AckType.Individual, Collections.emptyMap()); + persistentSubscription.acknowledgeMessage(positions, null, AckType.Individual, Collections.emptyMap()); //Abort txn. persistentSubscription.abortTxn(txnID1, consumerMock); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java new file mode 100644 index 0000000000000..29fb24e5c59c3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java @@ -0,0 +1,148 @@ +/** + * 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.client.impl; + +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.FutureUtil; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class BatchMessageIndexAckDisableTest extends ProducerConsumerBase { + + @BeforeMethod + @Override + protected void setup() throws Exception { + conf.setBatchIndexAcknowledgeEnable(false); + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterMethod + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testBatchMessageIndexAckForSharedSubscription() throws PulsarClientException, ExecutionException, InterruptedException { + final String topic = "testBatchMessageIndexAckForSharedSubscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .batchingMaxPublishDelay(50, TimeUnit.MILLISECONDS) + .create(); + + final int messages = 100; + List> futures = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < messages; i++) { + if (i % 2 == 0) { + consumer.acknowledge(consumer.receive()); + } + } + + List> received = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + received.add(consumer.receive()); + } + + Assert.assertEquals(received.size(), 100); + } + + @Test + public void testBatchMessageIndexAckForExclusiveSubscription() throws PulsarClientException, ExecutionException, InterruptedException { + final String topic = "testBatchMessageIndexAckForExclusiveSubscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .batchingMaxPublishDelay(50, TimeUnit.MILLISECONDS) + .create(); + + final int messages = 100; + List> futures = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < messages; i++) { + if (i == 49) { + consumer.acknowledgeCumulative(consumer.receive()); + } else { + consumer.receive(); + } + } + + //Wait ack send. + Thread.sleep(1000); + consumer.close(); + consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscribe(); + + List> received = new ArrayList<>(100); + for (int i = 0; i < messages; i++) { + received.add(consumer.receive()); + } + + Assert.assertEquals(received.size(), 100); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java new file mode 100644 index 0000000000000..a6053a8e49f48 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java @@ -0,0 +1,183 @@ +/** + * 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.client.impl; + +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.FutureUtil; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +@Slf4j +public class BatchMessageIndexAckTest extends ProducerConsumerBase { + + @BeforeMethod + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterMethod + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testBatchMessageIndexAckForSharedSubscription() throws PulsarClientException, ExecutionException, InterruptedException { + final String topic = "testBatchMessageIndexAckForSharedSubscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscriptionType(SubscriptionType.Shared) + .negativeAckRedeliveryDelay(2, TimeUnit.SECONDS) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .batchingMaxPublishDelay(50, TimeUnit.MILLISECONDS) + .create(); + + final int messages = 100; + List> futures = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < messages; i++) { + if (i % 2 == 0) { + consumer.acknowledge(consumer.receive()); + } else { + consumer.negativeAcknowledge(consumer.receive()); + } + } + + List> received = new ArrayList<>(50); + for (int i = 0; i < 50; i++) { + received.add(consumer.receive()); + } + + Assert.assertEquals(received.size(), 50); + + Message moreMessage = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNull(moreMessage); + + futures.clear(); + for (int i = 0; i < 50; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < 50; i++) { + received.add(consumer.receive()); + } + + // Ensure the flow permit is work well since the client skip the acked batch index, + // broker also need to handle the available permits. + Assert.assertEquals(received.size(), 100); + } + + @Test + public void testBatchMessageIndexAckForExclusiveSubscription() throws PulsarClientException, ExecutionException, InterruptedException { + final String topic = "testBatchMessageIndexAckForExclusiveSubscription"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .batchingMaxPublishDelay(50, TimeUnit.MILLISECONDS) + .create(); + + final int messages = 100; + List> futures = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < messages; i++) { + if (i == 49) { + consumer.acknowledgeCumulative(consumer.receive()); + } else { + consumer.receive(); + } + } + + //Wait ack send. + Thread.sleep(1000); + consumer.close(); + consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .receiverQueueSize(100) + .subscribe(); + + List> received = new ArrayList<>(50); + for (int i = 0; i < 50; i++) { + received.add(consumer.receive()); + } + + Assert.assertEquals(received.size(), 50); + + Message moreMessage = consumer.receive(1, TimeUnit.SECONDS); + Assert.assertNull(moreMessage); + + futures.clear(); + for (int i = 0; i < 50; i++) { + futures.add(producer.sendAsync(i)); + } + FutureUtil.waitForAll(futures).get(); + + for (int i = 0; i < 50; i++) { + received.add(consumer.receive()); + } + + // Ensure the flow permit is work well since the client skip the acked batch index, + // broker also need to handle the available permits. + Assert.assertEquals(received.size(), 100); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/CompactedOutBatchMessageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/CompactedOutBatchMessageTest.java index f7020378d5655..76b84dfca30c2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/CompactedOutBatchMessageTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/CompactedOutBatchMessageTest.java @@ -73,7 +73,7 @@ public void testCompactedOutMessages() throws Exception { = (ConsumerImpl) pulsarClient.newConsumer().topic(topic1) .subscriptionName("my-subscriber-name").subscribe()) { // shove it in the sideways - consumer.receiveIndividualMessagesFromBatch(metadata, 0, batchBuffer, + consumer.receiveIndividualMessagesFromBatch(metadata, 0, null, batchBuffer, MessageIdData.newBuilder().setLedgerId(1234) .setEntryId(567).build(), consumer.cnx()); Message m = consumer.receive(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java index 93600ec176fca..1517f124bc599 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AcknowledgmentsGroupingTracker.java @@ -31,6 +31,8 @@ public interface AcknowledgmentsGroupingTracker extends AutoCloseable { void addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map properties); + void addBatchIndexAcknowledgment(BatchMessageIdImpl msgId, int batchIndex, int batchSize, AckType ackType, Map properties); + void flush(); @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java index f311936b90c95..f97467ff4ec05 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BatchMessageIdImpl.java @@ -26,6 +26,7 @@ public class BatchMessageIdImpl extends MessageIdImpl { private final static int NO_BATCH = -1; private final int batchIndex; + private final int batchSize; private final transient BatchMessageAcker acker; @@ -36,12 +37,13 @@ private BatchMessageIdImpl() { } public BatchMessageIdImpl(long ledgerId, long entryId, int partitionIndex, int batchIndex) { - this(ledgerId, entryId, partitionIndex, batchIndex, BatchMessageAckerDisabled.INSTANCE); + this(ledgerId, entryId, partitionIndex, batchIndex, 0, BatchMessageAckerDisabled.INSTANCE); } - public BatchMessageIdImpl(long ledgerId, long entryId, int partitionIndex, int batchIndex, BatchMessageAcker acker) { + public BatchMessageIdImpl(long ledgerId, long entryId, int partitionIndex, int batchIndex, int batchSize, BatchMessageAcker acker) { super(ledgerId, entryId, partitionIndex); this.batchIndex = batchIndex; + this.batchSize = batchSize; this.acker = acker; } @@ -50,9 +52,11 @@ public BatchMessageIdImpl(MessageIdImpl other) { if (other instanceof BatchMessageIdImpl) { BatchMessageIdImpl otherId = (BatchMessageIdImpl) other; this.batchIndex = otherId.batchIndex; + this.batchSize = otherId.batchSize; this.acker = otherId.acker; } else { this.batchIndex = NO_BATCH; + this.batchSize = 0; this.acker = BatchMessageAckerDisabled.INSTANCE; } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index b08a97c06effb..8218fefba83d1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -384,7 +384,7 @@ protected void handleMessage(CommandMessage cmdMessage, ByteBuf headersAndPayloa } ConsumerImpl consumer = consumers.get(cmdMessage.getConsumerId()); if (consumer != null) { - consumer.messageReceived(cmdMessage.getMessageId(), cmdMessage.getRedeliveryCount(), headersAndPayload, this); + consumer.messageReceived(cmdMessage.getMessageId(), cmdMessage.getRedeliveryCount(), cmdMessage.getAckedIndexesList(), headersAndPayload, this); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 04983d9055406..fefb646a119b4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -76,6 +76,7 @@ import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.client.util.RetryMessageUtil; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; @@ -520,6 +521,9 @@ protected CompletableFuture doAcknowledge(MessageId messageId, AckType ack ackType); } } else { + BatchMessageIdImpl batchMessageId = (BatchMessageIdImpl) messageId; + acknowledgmentsGroupingTracker.addBatchIndexAcknowledgment(batchMessageId, batchMessageId.getBatchIndex(), + batchMessageId.getBatchSize(), ackType, properties); // other messages in batch are still pending ack. return CompletableFuture.completedFuture(null); } @@ -985,7 +989,7 @@ void activeConsumerChanged(boolean isActive) { }); } - void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf headersAndPayload, ClientCnx cnx) { + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received message: {}/{}", topic, subscription, messageId.getLedgerId(), messageId.getEntryId()); @@ -1077,7 +1081,7 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, ByteBuf heade } } else { // handle batch message enqueuing; uncompressed payload has all messages in batch - receiveIndividualMessagesFromBatch(msgMetadata, redeliveryCount, uncompressedPayload, messageId, cnx); + receiveIndividualMessagesFromBatch(msgMetadata, redeliveryCount, ackedBatchIndexRanges, uncompressedPayload, messageId, cnx); uncompressedPayload.release(); msgMetadata.recycle(); @@ -1168,7 +1172,7 @@ private void interceptAndComplete(final Message message, final CompletableFut listenerExecutor.execute(() -> receivedFuture.complete(interceptMessage)); } - void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, ByteBuf uncompressedPayload, + void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf uncompressedPayload, MessageIdData messageId, ClientCnx cnx) { int batchSize = msgMetadata.getNumMessagesInBatch(); @@ -1214,8 +1218,25 @@ void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliv continue; } + boolean isAcked = false; + if (ackedBatchIndexRanges != null) { + for (IntRange range : ackedBatchIndexRanges) { + if (i >= range.getStart() && i <= range.getEnd()) { + isAcked = true; + break; + } + } + } + + if (isAcked) { + singleMessagePayload.release(); + singleMessageMetadataBuilder.recycle(); + ++skippedMessages; + continue; + } + BatchMessageIdImpl batchMessageIdImpl = new BatchMessageIdImpl(messageId.getLedgerId(), - messageId.getEntryId(), getPartitionIndex(), i, acker); + messageId.getEntryId(), getPartitionIndex(), i, batchSize, acker); final MessageImpl message = new MessageImpl<>(topicName.toString(), batchMessageIdImpl, msgMetadata, singleMessageMetadataBuilder.build(), singleMessagePayload, createEncryptionContext(msgMetadata), cnx, schema, redeliveryCount); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java index 3c987d95cd74c..9061ac30bd61a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/NonPersistentAcknowledgmentGroupingTracker.java @@ -45,6 +45,11 @@ public void addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map properties) { + // no-op + } + @Override public void flush() { // no-op diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 95ca4cf91e0d3..48e84d82e8410 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -18,13 +18,17 @@ */ package org.apache.pulsar.client.impl; +import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoopGroup; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; @@ -35,8 +39,11 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; +import org.apache.pulsar.common.protocol.Commands.SingleBatchMessageIndexesAck; +import org.apache.pulsar.common.util.collections.ConcurrentBitSet; /** * Group the acknowledgements for a certain time and then sends them out in a single protobuf command. @@ -67,6 +74,7 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * broker. */ private final ConcurrentSkipListSet pendingIndividualAcks; + private final ConcurrentHashMap> pendingIndividualBatchIndexAcks; private final ScheduledFuture scheduledTask; @@ -74,6 +82,7 @@ public PersistentAcknowledgmentsGroupingTracker(ConsumerImpl consumer, Consum EventLoopGroup eventLoopGroup) { this.consumer = consumer; this.pendingIndividualAcks = new ConcurrentSkipListSet<>(); + this.pendingIndividualBatchIndexAcks = new ConcurrentHashMap<>(); this.acknowledgementGroupTimeMicros = conf.getAcknowledgementsGroupTimeMicros(); if (acknowledgementGroupTimeMicros > 0) { @@ -118,6 +127,21 @@ public void addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map properties) { + if (acknowledgementGroupTimeMicros == 0 || !properties.isEmpty() || ackType == AckType.Cumulative) { + doImmediateBatchIndexAck(msgId, batchIndex, batchSize, ackType, properties); + } else if (ackType == AckType.Individual) { + ConcurrentBitSet bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( + new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), msgId.getPartitionIndex()), (v) -> + Pair.of(batchSize, new ConcurrentBitSet(msgId.getBatchSize()))).getRight(); + bitSet.set(batchIndex); + if (pendingIndividualBatchIndexAcks.size() >= MAX_ACK_GROUP_SIZE) { + flush(); + } + } + } + private void doCumulativeAck(MessageIdImpl msgId) { // Handle concurrent updates from different threads while (true) { @@ -149,6 +173,26 @@ private boolean doImmediateAck(MessageIdImpl msgId, AckType ackType, Map properties) { + ClientCnx cnx = consumer.getClientCnx(); + + if (cnx == null) { + return false; + } + IntRange.Builder rangeBuilder = IntRange.newBuilder(); + final ByteBuf cmd = Commands.newBatchIndexAck(consumer.consumerId, + Collections.singletonList( + new SingleBatchMessageIndexesAck( + msgId.getLedgerId(), + msgId.getEntryId(), + batchSize, + Collections.singletonList(rangeBuilder.setStart(0).setEnd(batchIndex).build()) + )), ackType, null, properties); + rangeBuilder.recycle(); + cnx.ctx().writeAndFlush(cmd, cnx.ctx().voidPromise()); + return true; + } + /** * Flush all the pending acks and send them to the broker */ @@ -203,10 +247,36 @@ public void flush() { } } + if (!pendingIndividualBatchIndexAcks.isEmpty()) { + Iterator>> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); + List batchIndexesAcks = new ArrayList<>(pendingIndividualBatchIndexAcks.size()); + IntRange.Builder rangeBuilder = IntRange.newBuilder(); + while (iterator.hasNext()) { + Map.Entry> entry = iterator.next(); + SingleBatchMessageIndexesAck ack = new SingleBatchMessageIndexesAck(); + ack.setLedgerId(entry.getKey().getLedgerId()); + ack.setEntryId(entry.getKey().getEntryId()); + ack.setBatchSize(entry.getValue().getLeft()); + ack.setIndexRangesToAck(Lists.newArrayList()); + BitSet bitSet = entry.getValue().getRight(); + int nextSetBit = bitSet.nextSetBit(0); + while (nextSetBit != -1) { + int nextClearBit = bitSet.nextClearBit(nextSetBit); + ack.getIndexRangesToAck().add(rangeBuilder.setStart(nextSetBit).setEnd(nextClearBit - 1).build()); + nextSetBit = bitSet.nextSetBit(nextClearBit); + } + batchIndexesAcks.add(ack); + iterator.remove(); + } + cnx.ctx().write(Commands.newBatchIndexAck(consumer.consumerId, batchIndexesAcks, AckType.Individual, null, + null)); + shouldFlush = true; + } + if (shouldFlush) { if (log.isDebugEnabled()) { - log.debug("[{}] Flushing pending acks to broker: last-cumulative-ack: {} -- individual-acks: {}", - consumer, lastCumulativeAck, pendingIndividualAcks); + log.debug("[{}] Flushing pending acks to broker: last-cumulative-ack: {} -- individual-acks: {} -- individual-batch-index-acks: {}", + consumer, lastCumulativeAck, pendingIndividualAcks, pendingIndividualBatchIndexAcks); } cnx.ctx().flush(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java index a0de0704c2d9a..7ddb3639a1ef2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java @@ -23,6 +23,7 @@ import io.netty.buffer.ByteBuf; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.locks.Lock; @@ -36,6 +37,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; import org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata; @@ -177,6 +179,7 @@ protected void triggerListener(int numMessages) { @Override void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, + List ackedBatchIndexRanges, ByteBuf uncompressedPayload, MessageIdData messageId, ClientCnx cnx) { log.warn( "Closing consumer [{}]-[{}] due to unsupported received batch-message with zero receiver queue size", diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java new file mode 100644 index 0000000000000..5129f71db8b77 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java @@ -0,0 +1,35 @@ +package org.apache.pulsar.client.impl; + +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.concurrent.TimeUnit; + +public class KeySharedConsumer { + + public static void main(String[] args) throws PulsarClientException { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); + + for (int i = 0; i < 3000; i++) { + new Thread(() -> { + try { + Consumer consumer = client.newConsumer(Schema.STRING) + .topic("key_shared_latency-1") + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(1000) + .subscriptionName("test") + .subscribe(); + while (true) { + consumer.acknowledge(consumer.receive()); + } + } catch (PulsarClientException e) { + e.printStackTrace(); + } + }).start(); + } + + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java new file mode 100644 index 0000000000000..f965a971fa7d6 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java @@ -0,0 +1,35 @@ +package org.apache.pulsar.client.impl; + +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.concurrent.TimeUnit; + +public class KeySharedConsumer1 { + + public static void main(String[] args) throws PulsarClientException { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); + + for (int i = 0; i < 2000; i++) { + new Thread(() -> { + try { + Consumer consumer = client.newConsumer(Schema.STRING) + .topic("key_shared_latency-1") + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(1000) + .subscriptionName("test") + .subscribe(); + while (true) { + consumer.acknowledge(consumer.receive()); + } + } catch (PulsarClientException e) { + e.printStackTrace(); + } + }).start(); + } + + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java new file mode 100644 index 0000000000000..edb6c63c812d8 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java @@ -0,0 +1,35 @@ +package org.apache.pulsar.client.impl; + +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.concurrent.TimeUnit; + +public class KeySharedConsumer2 { + + public static void main(String[] args) throws PulsarClientException { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); + + for (int i = 0; i < 2000; i++) { + new Thread(() -> { + try { + Consumer consumer = client.newConsumer(Schema.STRING) + .topic("key_shared_latency-1") + .subscriptionType(SubscriptionType.Key_Shared) + .receiverQueueSize(1000) + .subscriptionName("test") + .subscribe(); + while (true) { + consumer.acknowledge(consumer.receive()); + } + } catch (PulsarClientException e) { + e.printStackTrace(); + } + }).start(); + } + + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java new file mode 100644 index 0000000000000..7f737f9391f46 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java @@ -0,0 +1,33 @@ +package org.apache.pulsar.client.impl; + +import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +public class KeySharedProducer { + + public static void main(String[] args) throws PulsarClientException, InterruptedException { + PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); + Producer producer = client.newProducer(Schema.STRING) + .topic("key_shared_latency-1") + .enableBatching(false) + .batcherBuilder(BatcherBuilder.KEY_BASED) + .maxPendingMessages(5000) + .create(); + + int i = 0; + while (true) { + producer.newMessage().key(UUID.randomUUID().toString()).value("test").sendAsync(); + if (++i % 20 == 0) { +// Thread.sleep(1); + } + } + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java new file mode 100644 index 0000000000000..c64ca31a63e79 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java @@ -0,0 +1,22 @@ +package org.apache.pulsar.client.util; + +import org.apache.pulsar.client.impl.Murmur3_32Hash; + +public class HashTest { + + public static void main(String[] args) { + + for (int i = 0; i < 3; i++) { + System.out.println(Murmur3_32Hash.getInstance().makeHash("consumer-name-" + i)); + } + + + System.out.println("-------------------------------------"); + + for (int i = 0; i < 3; i++) { + System.out.println(Murmur3_32Hash.getInstance().makeHash("ff" + i)); + } + + + } +} 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 e282ddc1b2fe7..ef259d7c698cb 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 @@ -1622,6 +1622,593 @@ public Builder clearBatchIndex() { // @@protoc_insertion_point(class_scope:pulsar.proto.MessageIdData) } + public interface BatchMessageIndexesAckDataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required .pulsar.proto.MessageIdData message_id = 1; + boolean hasMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); + + // required int32 batchSize = 2; + boolean hasBatchSize(); + int getBatchSize(); + + // repeated .pulsar.proto.IntRange ack_indexes = 3; + java.util.List + getAckIndexesList(); + org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index); + int getAckIndexesCount(); + } + public static final class BatchMessageIndexesAckData extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements BatchMessageIndexesAckDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use BatchMessageIndexesAckData.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private BatchMessageIndexesAckData(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected BatchMessageIndexesAckData newObject(Handle handle) { + return new BatchMessageIndexesAckData(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private BatchMessageIndexesAckData(boolean noInit) {} + + private static final BatchMessageIndexesAckData defaultInstance; + public static BatchMessageIndexesAckData getDefaultInstance() { + return defaultInstance; + } + + public BatchMessageIndexesAckData getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required .pulsar.proto.MessageIdData message_id = 1; + public static final int MESSAGE_ID_FIELD_NUMBER = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; + public boolean hasMessageId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + + // required int32 batchSize = 2; + public static final int BATCHSIZE_FIELD_NUMBER = 2; + private int batchSize_; + public boolean hasBatchSize() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getBatchSize() { + return batchSize_; + } + + // repeated .pulsar.proto.IntRange ack_indexes = 3; + public static final int ACK_INDEXES_FIELD_NUMBER = 3; + private java.util.List ackIndexes_; + public java.util.List getAckIndexesList() { + return ackIndexes_; + } + public java.util.List + getAckIndexesOrBuilderList() { + return ackIndexes_; + } + public int getAckIndexesCount() { + return ackIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index) { + return ackIndexes_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getAckIndexesOrBuilder( + int index) { + return ackIndexes_.get(index); + } + + private void initFields() { + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + batchSize_ = 0; + ackIndexes_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessageId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasBatchSize()) { + memoizedIsInitialized = 0; + return false; + } + if (!getMessageId().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getAckIndexesCount(); i++) { + if (!getAckIndexes(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.writeMessage(1, messageId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, batchSize_); + } + for (int i = 0; i < ackIndexes_.size(); i++) { + output.writeMessage(3, ackIndexes_.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 + .computeMessageSize(1, messageId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(2, batchSize_); + } + for (int i = 0; i < ackIndexes_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, ackIndexes_.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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.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(); + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000001); + batchSize_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + ackIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData build() { + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.messageId_ = messageId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.batchSize_ = batchSize_; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + ackIndexes_ = java.util.Collections.unmodifiableList(ackIndexes_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.ackIndexes_ = ackIndexes_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.getDefaultInstance()) return this; + if (other.hasMessageId()) { + mergeMessageId(other.getMessageId()); + } + if (other.hasBatchSize()) { + setBatchSize(other.getBatchSize()); + } + if (!other.ackIndexes_.isEmpty()) { + if (ackIndexes_.isEmpty()) { + ackIndexes_ = other.ackIndexes_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureAckIndexesIsMutable(); + ackIndexes_.addAll(other.ackIndexes_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasMessageId()) { + + return false; + } + if (!hasBatchSize()) { + + return false; + } + if (!getMessageId().isInitialized()) { + + return false; + } + for (int i = 0; i < getAckIndexesCount(); i++) { + if (!getAckIndexes(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: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasMessageId()) { + subBuilder.mergeFrom(getMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + batchSize_ = input.readInt32(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addAckIndexes(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .pulsar.proto.MessageIdData message_id = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasMessageId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + messageId_ = value; + + bitField0_ |= 0x00000001; + return this; + } + public Builder setMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + messageId_ = builderForValue.build(); + + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + messageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); + } else { + messageId_ = value; + } + + bitField0_ |= 0x00000001; + return this; + } + public Builder clearMessageId() { + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + // required int32 batchSize = 2; + private int batchSize_ ; + public boolean hasBatchSize() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getBatchSize() { + return batchSize_; + } + public Builder setBatchSize(int value) { + bitField0_ |= 0x00000002; + batchSize_ = value; + + return this; + } + public Builder clearBatchSize() { + bitField0_ = (bitField0_ & ~0x00000002); + batchSize_ = 0; + + return this; + } + + // repeated .pulsar.proto.IntRange ack_indexes = 3; + private java.util.List ackIndexes_ = + java.util.Collections.emptyList(); + private void ensureAckIndexesIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + ackIndexes_ = new java.util.ArrayList(ackIndexes_); + bitField0_ |= 0x00000004; + } + } + + public java.util.List getAckIndexesList() { + return java.util.Collections.unmodifiableList(ackIndexes_); + } + public int getAckIndexesCount() { + return ackIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index) { + return ackIndexes_.get(index); + } + public Builder setAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckIndexesIsMutable(); + ackIndexes_.set(index, value); + + return this; + } + public Builder setAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckIndexesIsMutable(); + ackIndexes_.set(index, builderForValue.build()); + + return this; + } + public Builder addAckIndexes(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckIndexesIsMutable(); + ackIndexes_.add(value); + + return this; + } + public Builder addAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckIndexesIsMutable(); + ackIndexes_.add(index, value); + + return this; + } + public Builder addAckIndexes( + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckIndexesIsMutable(); + ackIndexes_.add(builderForValue.build()); + + return this; + } + public Builder addAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckIndexesIsMutable(); + ackIndexes_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllAckIndexes( + java.lang.Iterable values) { + ensureAckIndexesIsMutable(); + super.addAll(values, ackIndexes_); + + return this; + } + public Builder clearAckIndexes() { + ackIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + + return this; + } + public Builder removeAckIndexes(int index) { + ensureAckIndexesIsMutable(); + ackIndexes_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.BatchMessageIndexesAckData) + } + + static { + defaultInstance = new BatchMessageIndexesAckData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.BatchMessageIndexesAckData) + } + public interface KeyValueOrBuilder extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { @@ -17634,6 +18221,12 @@ public interface CommandMessageOrBuilder // optional uint32 redelivery_count = 3 [default = 0]; boolean hasRedeliveryCount(); int getRedeliveryCount(); + + // repeated .pulsar.proto.IntRange acked_indexes = 4; + java.util.List + getAckedIndexesList(); + org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index); + int getAckedIndexesCount(); } public static final class CommandMessage extends org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite @@ -17700,10 +18293,32 @@ public int getRedeliveryCount() { return redeliveryCount_; } + // repeated .pulsar.proto.IntRange acked_indexes = 4; + public static final int ACKED_INDEXES_FIELD_NUMBER = 4; + private java.util.List ackedIndexes_; + public java.util.List getAckedIndexesList() { + return ackedIndexes_; + } + public java.util.List + getAckedIndexesOrBuilderList() { + return ackedIndexes_; + } + public int getAckedIndexesCount() { + return ackedIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index) { + return ackedIndexes_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getAckedIndexesOrBuilder( + int index) { + return ackedIndexes_.get(index); + } + private void initFields() { consumerId_ = 0L; messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); redeliveryCount_ = 0; + ackedIndexes_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -17722,6 +18337,12 @@ public final boolean isInitialized() { memoizedIsInitialized = 0; return false; } + for (int i = 0; i < getAckedIndexesCount(); i++) { + if (!getAckedIndexes(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -17743,6 +18364,9 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr if (((bitField0_ & 0x00000004) == 0x00000004)) { output.writeUInt32(3, redeliveryCount_); } + for (int i = 0; i < ackedIndexes_.size(); i++) { + output.writeMessage(4, ackedIndexes_.get(i)); + } } private int memoizedSerializedSize = -1; @@ -17763,6 +18387,10 @@ public int getSerializedSize() { size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream .computeUInt32Size(3, redeliveryCount_); } + for (int i = 0; i < ackedIndexes_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, ackedIndexes_.get(i)); + } memoizedSerializedSize = size; return size; } @@ -17882,6 +18510,8 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000002); redeliveryCount_ = 0; bitField0_ = (bitField0_ & ~0x00000004); + ackedIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -17927,6 +18557,11 @@ public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage buildPartial( to_bitField0_ |= 0x00000004; } result.redeliveryCount_ = redeliveryCount_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + ackedIndexes_ = java.util.Collections.unmodifiableList(ackedIndexes_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.ackedIndexes_ = ackedIndexes_; result.bitField0_ = to_bitField0_; return result; } @@ -17942,6 +18577,16 @@ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandMes if (other.hasRedeliveryCount()) { setRedeliveryCount(other.getRedeliveryCount()); } + if (!other.ackedIndexes_.isEmpty()) { + if (ackedIndexes_.isEmpty()) { + ackedIndexes_ = other.ackedIndexes_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureAckedIndexesIsMutable(); + ackedIndexes_.addAll(other.ackedIndexes_); + } + + } return this; } @@ -17958,6 +18603,12 @@ public final boolean isInitialized() { return false; } + for (int i = 0; i < getAckedIndexesCount(); i++) { + if (!getAckedIndexes(i).isInitialized()) { + + return false; + } + } return true; } @@ -18003,6 +18654,12 @@ public Builder mergeFrom( redeliveryCount_ = input.readUInt32(); break; } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addAckedIndexes(subBuilder.buildPartial()); + break; + } } } } @@ -18094,6 +18751,95 @@ public Builder clearRedeliveryCount() { return this; } + // repeated .pulsar.proto.IntRange acked_indexes = 4; + private java.util.List ackedIndexes_ = + java.util.Collections.emptyList(); + private void ensureAckedIndexesIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + ackedIndexes_ = new java.util.ArrayList(ackedIndexes_); + bitField0_ |= 0x00000008; + } + } + + public java.util.List getAckedIndexesList() { + return java.util.Collections.unmodifiableList(ackedIndexes_); + } + public int getAckedIndexesCount() { + return ackedIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index) { + return ackedIndexes_.get(index); + } + public Builder setAckedIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckedIndexesIsMutable(); + ackedIndexes_.set(index, value); + + return this; + } + public Builder setAckedIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckedIndexesIsMutable(); + ackedIndexes_.set(index, builderForValue.build()); + + return this; + } + public Builder addAckedIndexes(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckedIndexesIsMutable(); + ackedIndexes_.add(value); + + return this; + } + public Builder addAckedIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureAckedIndexesIsMutable(); + ackedIndexes_.add(index, value); + + return this; + } + public Builder addAckedIndexes( + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckedIndexesIsMutable(); + ackedIndexes_.add(builderForValue.build()); + + return this; + } + public Builder addAckedIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureAckedIndexesIsMutable(); + ackedIndexes_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllAckedIndexes( + java.lang.Iterable values) { + ensureAckedIndexesIsMutable(); + super.addAll(values, ackedIndexes_); + + return this; + } + public Builder clearAckedIndexes() { + ackedIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + public Builder removeAckedIndexes(int index) { + ensureAckedIndexesIsMutable(); + ackedIndexes_.remove(index); + + return this; + } + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandMessage) } @@ -18139,6 +18885,12 @@ public interface CommandAckOrBuilder // optional uint64 txnid_most_bits = 7 [default = 0]; boolean hasTxnidMostBits(); long getTxnidMostBits(); + + // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; + java.util.List + getBatchMessageAckIndexesList(); + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index); + int getBatchMessageAckIndexesCount(); } public static final class CommandAck extends org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite @@ -18358,6 +19110,27 @@ public long getTxnidMostBits() { return txnidMostBits_; } + // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; + public static final int BATCH_MESSAGE_ACK_INDEXES_FIELD_NUMBER = 8; + private java.util.List batchMessageAckIndexes_; + public java.util.List getBatchMessageAckIndexesList() { + return batchMessageAckIndexes_; + } + public java.util.List + getBatchMessageAckIndexesOrBuilderList() { + return batchMessageAckIndexes_; + } + public int getBatchMessageAckIndexesCount() { + return batchMessageAckIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index) { + return batchMessageAckIndexes_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckDataOrBuilder getBatchMessageAckIndexesOrBuilder( + int index) { + return batchMessageAckIndexes_.get(index); + } + private void initFields() { consumerId_ = 0L; ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; @@ -18366,6 +19139,7 @@ private void initFields() { properties_ = java.util.Collections.emptyList(); txnidLeastBits_ = 0L; txnidMostBits_ = 0L; + batchMessageAckIndexes_ = java.util.Collections.emptyList(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -18392,6 +19166,12 @@ public final boolean isInitialized() { return false; } } + for (int i = 0; i < getBatchMessageAckIndexesCount(); i++) { + if (!getBatchMessageAckIndexes(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -18425,6 +19205,9 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr if (((bitField0_ & 0x00000010) == 0x00000010)) { output.writeUInt64(7, txnidMostBits_); } + for (int i = 0; i < batchMessageAckIndexes_.size(); i++) { + output.writeMessage(8, batchMessageAckIndexes_.get(i)); + } } private int memoizedSerializedSize = -1; @@ -18461,6 +19244,10 @@ public int getSerializedSize() { size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream .computeUInt64Size(7, txnidMostBits_); } + for (int i = 0; i < batchMessageAckIndexes_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(8, batchMessageAckIndexes_.get(i)); + } memoizedSerializedSize = size; return size; } @@ -18588,6 +19375,8 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000020); txnidMostBits_ = 0L; bitField0_ = (bitField0_ & ~0x00000040); + batchMessageAckIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); return this; } @@ -18651,6 +19440,11 @@ public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck buildPartial() { to_bitField0_ |= 0x00000010; } result.txnidMostBits_ = txnidMostBits_; + if (((bitField0_ & 0x00000080) == 0x00000080)) { + batchMessageAckIndexes_ = java.util.Collections.unmodifiableList(batchMessageAckIndexes_); + bitField0_ = (bitField0_ & ~0x00000080); + } + result.batchMessageAckIndexes_ = batchMessageAckIndexes_; result.bitField0_ = to_bitField0_; return result; } @@ -18692,6 +19486,16 @@ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck if (other.hasTxnidMostBits()) { setTxnidMostBits(other.getTxnidMostBits()); } + if (!other.batchMessageAckIndexes_.isEmpty()) { + if (batchMessageAckIndexes_.isEmpty()) { + batchMessageAckIndexes_ = other.batchMessageAckIndexes_; + bitField0_ = (bitField0_ & ~0x00000080); + } else { + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.addAll(other.batchMessageAckIndexes_); + } + + } return this; } @@ -18716,6 +19520,12 @@ public final boolean isInitialized() { return false; } } + for (int i = 0; i < getBatchMessageAckIndexesCount(); i++) { + if (!getBatchMessageAckIndexes(i).isInitialized()) { + + return false; + } + } return true; } @@ -18786,6 +19596,12 @@ public Builder mergeFrom( txnidMostBits_ = input.readUInt64(); break; } + case 66: { + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addBatchMessageAckIndexes(subBuilder.buildPartial()); + break; + } } } } @@ -19081,6 +19897,95 @@ public Builder clearTxnidMostBits() { return this; } + // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; + private java.util.List batchMessageAckIndexes_ = + java.util.Collections.emptyList(); + private void ensureBatchMessageAckIndexesIsMutable() { + if (!((bitField0_ & 0x00000080) == 0x00000080)) { + batchMessageAckIndexes_ = new java.util.ArrayList(batchMessageAckIndexes_); + bitField0_ |= 0x00000080; + } + } + + public java.util.List getBatchMessageAckIndexesList() { + return java.util.Collections.unmodifiableList(batchMessageAckIndexes_); + } + public int getBatchMessageAckIndexesCount() { + return batchMessageAckIndexes_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index) { + return batchMessageAckIndexes_.get(index); + } + public Builder setBatchMessageAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.set(index, value); + + return this; + } + public Builder setBatchMessageAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.set(index, builderForValue.build()); + + return this; + } + public Builder addBatchMessageAckIndexes(org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.add(value); + + return this; + } + public Builder addBatchMessageAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.add(index, value); + + return this; + } + public Builder addBatchMessageAckIndexes( + org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.add(builderForValue.build()); + + return this; + } + public Builder addBatchMessageAckIndexes( + int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllBatchMessageAckIndexes( + java.lang.Iterable values) { + ensureBatchMessageAckIndexesIsMutable(); + super.addAll(values, batchMessageAckIndexes_); + + return this; + } + public Builder clearBatchMessageAckIndexes() { + batchMessageAckIndexes_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); + + return this; + } + public Builder removeBatchMessageAckIndexes(int index) { + ensureBatchMessageAckIndexesIsMutable(); + batchMessageAckIndexes_.remove(index); + + return this; + } + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAck) } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index b3447008ce887..b1f5d09d020fb 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -48,6 +48,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod; import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand; import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type; +import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; 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.CommandAck.ValidationError; @@ -104,6 +105,7 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess; import org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe; import org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags; +import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue; import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; @@ -437,13 +439,16 @@ public static void skipMessageMetadata(ByteBuf buffer) { } public static ByteBufPair newMessage(long consumerId, MessageIdData messageId, int redeliveryCount, - ByteBuf metadataAndPayload) { + ByteBuf metadataAndPayload, List ackedBatchIndexRanges) { CommandMessage.Builder msgBuilder = CommandMessage.newBuilder(); msgBuilder.setConsumerId(consumerId); msgBuilder.setMessageId(messageId); if (redeliveryCount > 0) { msgBuilder.setRedeliveryCount(redeliveryCount); } + if (ackedBatchIndexRanges != null && ackedBatchIndexRanges.size() > 0) { + msgBuilder.addAllAckedIndexes(ackedBatchIndexRanges); + } CommandMessage msg = msgBuilder.build(); BaseCommand.Builder cmdBuilder = BaseCommand.newBuilder(); BaseCommand cmd = cmdBuilder.setType(Type.MESSAGE).setMessage(msg).build(); @@ -929,6 +934,56 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, AckTy return res; } + public static ByteBuf newBatchIndexAck(long consumerId, + List batchMessageIndexesAckData, + AckType ackType, ValidationError validationError, Map properties) { + CommandAck.Builder ackBuilder = CommandAck.newBuilder(); + ackBuilder.setConsumerId(consumerId); + ackBuilder.setAckType(ackType); + MessageIdData.Builder messageIdDataBuilder = MessageIdData.newBuilder(); + batchMessageIndexesAckData.forEach(singleBatchMessageIndexesAck -> { + BatchMessageIndexesAckData.Builder indexAckBuilder = BatchMessageIndexesAckData.newBuilder(); + messageIdDataBuilder.setLedgerId(singleBatchMessageIndexesAck.getLedgerId()); + messageIdDataBuilder.setEntryId(singleBatchMessageIndexesAck.getEntryId()); + indexAckBuilder.setMessageId(messageIdDataBuilder.build()); + indexAckBuilder.setBatchSize(singleBatchMessageIndexesAck.getBatchSize()); + if (ackType == AckType.Individual) { + indexAckBuilder.addAllAckIndexes(singleBatchMessageIndexesAck.getIndexRangesToAck()); + } else { + if (singleBatchMessageIndexesAck.getIndexRangesToAck() != null + && singleBatchMessageIndexesAck.getIndexRangesToAck().size() == 1) { + indexAckBuilder.addAckIndexes(singleBatchMessageIndexesAck.getIndexRangesToAck().get(0)); + } + } + ackBuilder.addBatchMessageAckIndexes(indexAckBuilder.build()); + indexAckBuilder.recycle(); + }); + + if (validationError != null) { + ackBuilder.setValidationError(validationError); + } + + if (properties != null) { + for (Map.Entry e : properties.entrySet()) { + ackBuilder.addProperties( + KeyLongValue.newBuilder().setKey(e.getKey()).setValue(e.getValue()).build()); + } + } + + CommandAck ack = ackBuilder.build(); + + ByteBuf res = serializeWithSize(BaseCommand.newBuilder().setType(Type.ACK).setAck(ack)); + ack.getBatchMessageAckIndexesList().forEach(ackData -> { + ackData.getMessageId().recycle(); + ackData.getAckIndexesList().forEach(IntRange::recycle); + ackData.recycle(); + }); + ack.recycle(); + messageIdDataBuilder.recycle(); + ackBuilder.recycle(); + return res; + } + public static ByteBuf newAckResponse(long consumerId, long txnIdLeastBits, long txnIdMostBits) { CommandAckResponse.Builder commandAckResponseBuilder = CommandAckResponse.newBuilder(); commandAckResponseBuilder.setConsumerId(consumerId); @@ -1762,4 +1817,54 @@ public static boolean peerSupportJsonSchemaAvroFormat(int peerVersion) { public static boolean peerSupportsGetOrCreateSchema(int peerVersion) { return peerVersion >= ProtocolVersion.v15.getNumber(); } + + public static class SingleBatchMessageIndexesAck { + private long ledgerId; + private long entryId; + private int batchSize; + private List indexRangesToAck; + + public SingleBatchMessageIndexesAck() { + } + + public SingleBatchMessageIndexesAck(long ledgerId, long entryId, int batchSize, + List indexRangesToAck) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.batchSize = batchSize; + this.indexRangesToAck = indexRangesToAck; + } + + public long getLedgerId() { + return ledgerId; + } + + public void setLedgerId(long ledgerId) { + this.ledgerId = ledgerId; + } + + public long getEntryId() { + return entryId; + } + + public void setEntryId(long entryId) { + this.entryId = entryId; + } + + public int getBatchSize() { + return batchSize; + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + public List getIndexRangesToAck() { + return indexRangesToAck; + } + + public void setIndexRangesToAck(List indexRangesToAck) { + this.indexRangesToAck = indexRangesToAck; + } + } } diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index 43a813c3b1710..f16083998a29a 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -56,6 +56,12 @@ message MessageIdData { optional int32 batch_index = 4 [default = -1]; } +message BatchMessageIndexesAckData { + required MessageIdData message_id = 1; + required int32 batchSize = 2; + repeated IntRange ack_indexes = 3; +} + message KeyValue { required string key = 1; required string value = 2; @@ -453,6 +459,7 @@ message CommandMessage { required uint64 consumer_id = 1; required MessageIdData message_id = 2; optional uint32 redelivery_count = 3 [default = 0]; + repeated IntRange acked_indexes = 4; } message CommandAck { @@ -483,6 +490,8 @@ message CommandAck { optional uint64 txnid_least_bits = 6 [default = 0]; optional uint64 txnid_most_bits = 7 [default = 0]; + + repeated BatchMessageIndexesAckData batch_message_ack_indexes = 8; } message CommandAckResponse { From 0bd8eabfceb01251f998630e861f960db0bcab52 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 30 Jan 2020 22:18:20 +0800 Subject: [PATCH 02/21] Fix comments --- .../bookkeeper/mledger/ManagedCursor.java | 18 +- .../mledger/ManagedLedgerConfig.java | 9 + .../mledger/impl/ManagedCursorImpl.java | 197 +- .../bookkeeper/mledger/impl/PositionImpl.java | 13 + .../src/main/proto/MLDataFormats.proto | 7 +- .../impl/ManagedCursorContainerTest.java | 8 +- .../mledger/impl/ManagedCursorTest.java | 51 +- .../service/AbstractBaseDispatcher.java | 12 +- .../pulsar/broker/service/BrokerService.java | 1 + .../pulsar/broker/service/Consumer.java | 31 +- .../pulsar/broker/service/Subscription.java | 3 +- .../NonPersistentSubscription.java | 3 +- .../persistent/CompactorSubscription.java | 3 +- .../persistent/PersistentSubscription.java | 32 +- .../service/persistent/PersistentTopic.java | 3 +- .../ReplicatedSubscriptionsController.java | 2 +- .../broker/service/PersistentTopicTest.java | 2 +- .../pulsar/broker/service/ServerCnxTest.java | 2 +- .../PersistentSubscriptionTest.java | 2 +- .../pulsar/client/impl/ConsumerImpl.java | 22 +- ...sistentAcknowledgmentsGroupingTracker.java | 96 +- .../pulsar/common/api/proto/PulsarApi.java | 44375 ---------------- .../pulsar/common/protocol/Commands.java | 128 +- .../common/util/SafeCollectionUtils.java | 53 + pulsar-common/src/main/proto/PulsarApi.proto | 11 +- 25 files changed, 308 insertions(+), 44776 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index e02eebd0443df..3cb55e3d0774e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -34,7 +34,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.ReadEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.SkipEntriesCallback; import org.apache.bookkeeper.mledger.impl.PositionImpl; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; /** * A ManangedCursor is a persisted cursor inside a ManagedLedger. @@ -321,21 +320,6 @@ void markDelete(Position position, Map properties) */ void asyncDelete(Iterable position, DeleteCallback callback, Object ctx); - /** - * Delete a group of batch indexes in a batch message asynchronously - * - *

- * Mark a group of batch indexed for deletion. When all indexes of a batch message are all deleted, then will mark the batch message - * for deletion - * - * @param position the position of the message to deleted - * @param batchSize batch size of the batch message - * @param deleteIndexRanges delete index ranges for a batch message - * @param callback callback object - * @param ctx opaque context - */ - void asyncDelete(Position position, int batchSize, List deleteIndexRanges, DeleteCallback callback, Object ctx); - /** * Get the read position. This points to the next message to be read from the cursor. * @@ -629,5 +613,5 @@ Set asyncReplayEntries( /** * Get deleted batch indexes list for a batch message. */ - List getDeletedBatchIndexes(Position position); + long[] getDeletedBatchIndexesLongArray(PositionImpl position); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 56dd9064d7c5f..57ebbe8c14a45 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -43,6 +43,7 @@ public class ManagedLedgerConfig { private boolean createIfMissing = true; private int maxUnackedRangesToPersist = 10000; private int maxBatchDeletedIndexToPersist = 10000; + private boolean batchIndexDeleteEnabled = true; private int maxUnackedRangesToPersistInZk = 1000; private int maxEntriesPerLedger = 50000; private int maxSizePerLedgerMb = 100; @@ -593,4 +594,12 @@ public void setBookKeeperEnsemblePlacementPolicyProperties( Map bookKeeperEnsemblePlacementPolicyProperties) { this.bookKeeperEnsemblePlacementPolicyProperties = bookKeeperEnsemblePlacementPolicyProperties; } + + public boolean isBatchIndexDeleteEnabled() { + return batchIndexDeleteEnabled; + } + + public void setBatchIndexDeleteEnabled(boolean batchIndexDeleteEnabled) { + this.batchIndexDeleteEnabled = batchIndexDeleteEnabled; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index c431d997f5331..2e318f2e91225 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -43,13 +43,14 @@ import java.time.Clock; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -92,8 +93,6 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; -import org.apache.pulsar.common.util.collections.ConcurrentBitSet; import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; @@ -162,8 +161,8 @@ public class ManagedCursorImpl implements ManagedCursor { private final LongPairRangeSet individualDeletedMessages; // Maintain the indexes deleted status of batch messages that not deleted completely - // (ledgerId, entryId, batchSize) -> deleted indexes - private final ConcurrentHashMap batchDeletedIndexes; + // (ledgerId, entryId) -> deleted indexes + private final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -235,7 +234,11 @@ public interface VoidCallback { this.individualDeletedMessages = config.isUnackedRangesOpenCacheSetEnabled() ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); - this.batchDeletedIndexes = new ConcurrentHashMap<>(); + if (config.isBatchIndexDeleteEnabled()) { + this.batchDeletedIndexes = new ConcurrentSkipListMap<>(); + } else { + this.batchDeletedIndexes = null; + } this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType()); STATE_UPDATER.set(this, State.Uninitialized); PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0); @@ -383,7 +386,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } - if (positionInfo.getBatchDeletedIndexesCount() > 0) { + if (config.isBatchIndexDeleteEnabled() && positionInfo.getBatchDeletedIndexesCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchDeletedIndexesList()); } recoveredCursor(position, recoveredProperties, lh); @@ -410,12 +413,13 @@ private void recoverBatchDeletedIndexes (List { - if (batchDeletedIndexInfo.getBatchDeletedIndexesCount() > 0) { - ConcurrentBitSet value = new ConcurrentBitSet(0); - batchDeletedIndexInfo.getBatchDeletedIndexesList().forEach(deletedIndexRange -> - value.set(deletedIndexRange.getLowerIndex(), deletedIndexRange.getUpperIndex())); + if (batchDeletedIndexInfo.getAckBitSetCount() > 0) { + long[] array = new long[batchDeletedIndexInfo.getAckBitSetCount()]; + for (int i = 0; i < batchDeletedIndexInfo.getAckBitSetList().size(); i++) { + array[i] = batchDeletedIndexInfo.getAckBitSetList().get(i); + } this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(), - batchDeletedIndexInfo.getPosition().getEntryId()), value); + batchDeletedIndexInfo.getPosition().getEntryId()), BitSet.valueOf(array)); } }); } finally { @@ -932,7 +936,9 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - batchDeletedIndexes.clear(); + if (config.isBatchIndexDeleteEnabled()) { + batchDeletedIndexes.clear(); + } PositionImpl oldReadPosition = readPosition; if (oldReadPosition.compareTo(newPosition) >= 0) { @@ -1520,8 +1526,20 @@ public void asyncMarkDelete(final Position position, Map propertie if (log.isDebugEnabled()) { log.debug("[{}] Mark delete cursor {} up to position: {}", ledger.getName(), name, position); } + PositionImpl newPosition = (PositionImpl) position; + if (config.isBatchIndexDeleteEnabled()) { + if (newPosition.ackSet != null) { + batchDeletedIndexes.put(newPosition, newPosition.ackSet); + } + batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition).clear(); + newPosition = ledger.getPreviousPosition(newPosition); + } else if (newPosition.ackSet != null) { + callback.markDeleteFailed(new ManagedLedgerException("Batch ack set not support"), ctx); + return; + } + if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(newPosition) < 0) { if (log.isDebugEnabled()) { log.debug( @@ -1613,9 +1631,9 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - batchDeletedIndexes.entrySet().removeIf(entry -> - entry.getKey().compareTo( - PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId())) <= 0); + if (config.isBatchIndexDeleteEnabled()) { + batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true).clear(); + } } finally { lock.writeLock().unlock(); } @@ -1738,37 +1756,51 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb for (Position pos : positions) { PositionImpl position = (PositionImpl) checkNotNull(pos); - if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(position) < 0) { if (log.isDebugEnabled()) { log.debug( - "[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {} for cursor [{}]", - ledger.getName(), position, ledger.getLastConfirmedEntry(), name); + "[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {} for cursor [{}]", + ledger.getName(), position, ledger.getLastConfirmedEntry(), name); } callback.deleteFailed(new ManagedLedgerException("Invalid mark deleted position"), ctx); return; } if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) - || position.compareTo(markDeletePosition) <= 0) { + || position.compareTo(markDeletePosition) <= 0) { + if (config.isBatchIndexDeleteEnabled()) { + batchDeletedIndexes.remove(position); + } if (log.isDebugEnabled()) { log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); } continue; } - - batchDeletedIndexes.remove(position); - - // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make - // the RangeSet recognize the "continuity" between adjacent Positions - PositionImpl previousPosition = ledger.getPreviousPosition(position); - individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), + if (position.ackSet == null) { + if (config.isBatchIndexDeleteEnabled()) { + batchDeletedIndexes.remove(position); + } + // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make + // the RangeSet recognize the "continuity" between adjacent Positions + PositionImpl previousPosition = ledger.getPreviousPosition(position); + individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); - MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); + MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); + } + } else if (config.isBatchIndexDeleteEnabled()) { + BitSet bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSet.valueOf(position.ackSet.toLongArray())); + bitSet.and(position.ackSet); + if (bitSet.isEmpty()) { + PositionImpl previousPosition = ledger.getPreviousPosition(position); + individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), + position.getLedgerId(), position.getEntryId()); + ++messagesConsumedCounter; + batchDeletedIndexes.remove(position); + } } } @@ -1844,58 +1876,6 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { } } - @Override - public void asyncDelete(Position position, int batchSize, List deleteIndexRanges, AsyncCallbacks.DeleteCallback callback, Object ctx) { - if (isClosed()) { - callback.deleteFailed(new ManagedLedgerException("Cursor was already closed"), null); - return; - } - lock.writeLock().lock(); - try { - - ConcurrentBitSet bitSet; - PositionImpl pos = (PositionImpl) checkNotNull(position); - if (markDeletePosition.compareTo(pos) >= 0 || individualDeletedMessages.contains(pos.ledgerId, pos.entryId)) { - batchDeletedIndexes.remove(pos); - callback.deleteComplete(ctx); - return; - } - - if (!batchDeletedIndexes.containsKey(pos)) { - bitSet = new ConcurrentBitSet(batchSize); - batchDeletedIndexes.put(pos, bitSet); - } else { - bitSet = batchDeletedIndexes.get(pos); - } - - for (IntRange deleteIndexRange : deleteIndexRanges) { - bitSet.set(deleteIndexRange.getStart(), deleteIndexRange.getEnd() + 1); - } - - if (bitSet.nextClearBit(0) == batchSize) { - asyncDelete(pos, new AsyncCallbacks.DeleteCallback() { - @Override - public void deleteComplete(Object ctx) { - callback.deleteComplete(ctx); - } - - @Override - public void deleteFailed(ManagedLedgerException exception, Object ctx) { - callback.deleteFailed(getManagedLedgerException(exception), ctx); - } - }, ctx); - } else { - callback.deleteComplete(ctx); - } - } catch (Exception e) { - log.warn("[{}] [{}] Error while updating batchDeletedMessages [{}]", ledger.getName(), name, - e.getMessage(), e); - callback.deleteFailed(getManagedLedgerException(e), ctx); - } finally { - lock.writeLock().unlock(); - } - } - /** * Given a list of entries, filter out the entries that have already been individually deleted. * @@ -2132,7 +2112,9 @@ private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl positio info.addAllProperties(buildPropertiesMap(properties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); - info.addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()); + if (config.isBatchIndexDeleteEnabled()) { + info.addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()); + } } if (log.isDebugEnabled()) { @@ -2380,29 +2362,24 @@ private List buildIndividualDeletedMessageRanges() { private List buildBatchDeletedIndexInfoList() { lock.readLock().lock(); try { - if (batchDeletedIndexes.isEmpty()) { + if (!config.isBatchIndexDeleteEnabled() || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo .newBuilder(); MLDataFormats.BatchDeletedIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchDeletedIndexInfo .newBuilder(); - MLDataFormats.BatchDeletedIndexRange.Builder batchDeletedIndexRangeBuilder = MLDataFormats.BatchDeletedIndexRange - .newBuilder(); List result = Lists.newArrayList(); - for (Map.Entry entry : batchDeletedIndexes.entrySet()) { + for (Map.Entry entry : batchDeletedIndexes.entrySet()) { nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); - BitSet bitSet = entry.getValue(); - int nextSetBit = bitSet.nextSetBit(0); - while (nextSetBit != -1) { - int nextClearBit = bitSet.nextClearBit(nextSetBit); - batchDeletedIndexRangeBuilder.setLowerIndex(nextSetBit); - batchDeletedIndexRangeBuilder.setUpperIndex(nextClearBit); - batchDeletedIndexInfoBuilder.addBatchDeletedIndexes(batchDeletedIndexRangeBuilder.build()); - nextSetBit = bitSet.nextSetBit(nextClearBit); + long[] array = entry.getValue().toLongArray(); + List ackBitSet = new ArrayList<>(array.length); + for (long l : array) { + ackBitSet.add(l); } + batchDeletedIndexInfoBuilder.addAllAckBitSet(ackBitSet); result.add(batchDeletedIndexInfoBuilder.build()); if (result.size() >= config.getMaxBatchDeletedIndexToPersist()) { break; @@ -2798,31 +2775,19 @@ public void trimDeletedEntries(List entries) { || individualDeletedMessages.contains(entry.getLedgerId(), entry.getEntryId())); } -<<<<<<< HEAD private ManagedCursorImpl cursorImpl() { return this; -======= + } + @Override - public List getDeletedBatchIndexes(Position position) { - if (batchDeletedIndexes.isEmpty()) { - return null; - } - PositionImpl pos = (PositionImpl) checkNotNull(position); - if (batchDeletedIndexes.containsKey(pos)) { - List result = Lists.newArrayList(); - BitSet bitSet = batchDeletedIndexes.get(pos); - int nextSetBit = bitSet.nextSetBit(0); - IntRange.Builder builder = IntRange.newBuilder(); - while (nextSetBit != -1) { - int nextClearBit = bitSet.nextClearBit(nextSetBit); - result.add(builder.setStart(nextSetBit).setEnd(nextClearBit - 1).build()); - nextSetBit = bitSet.nextSetBit(nextClearBit); - } - return result; - } else { - return null; + public long[] getDeletedBatchIndexesLongArray(PositionImpl position) { + lock.readLock().lock(); + try { + BitSet bitSet = batchDeletedIndexes.get(position); + return bitSet == null ? null : bitSet.toLongArray(); + } finally { + lock.readLock().unlock(); } ->>>>>>> Add support for batch local index ack } private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java index f090b5268d233..7995840d5cf9c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java @@ -26,10 +26,13 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import java.util.BitSet; + public class PositionImpl implements Position, Comparable { protected long ledgerId; protected long entryId; + protected BitSet ackSet; public static final PositionImpl earliest = new PositionImpl(-1, -1); public static final PositionImpl latest = new PositionImpl(Long.MAX_VALUE, Long.MAX_VALUE); @@ -49,6 +52,12 @@ public PositionImpl(long ledgerId, long entryId) { this.entryId = entryId; } + public PositionImpl(long ledgerId, long entryId, BitSet ackSet) { + this.ledgerId = ledgerId; + this.entryId = entryId; + this.ackSet = ackSet; + } + public PositionImpl(PositionImpl other) { this.ledgerId = other.ledgerId; this.entryId = other.entryId; @@ -58,6 +67,10 @@ public static PositionImpl get(long ledgerId, long entryId) { return new PositionImpl(ledgerId, entryId); } + public static PositionImpl get(long ledgerId, long entryId, BitSet ackSet) { + return new PositionImpl(ledgerId, entryId, ackSet); + } + public static PositionImpl get(PositionImpl other) { return new PositionImpl(other); } diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 2c12a950b6004..9110663069c56 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -81,14 +81,9 @@ message MessageRange { required NestedPositionInfo upperEndpoint = 2; } -message BatchDeletedIndexRange { - required uint32 lowerIndex = 1; - required uint32 upperIndex = 2; -} - message BatchDeletedIndexInfo { required NestedPositionInfo position = 1; - repeated BatchDeletedIndexRange batchDeletedIndexes = 2; + repeated int64 ackBitSet = 2; } // Generic string and long tuple diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index d781e1c2440e8..0c796b4604889 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -180,10 +180,6 @@ public void delete(Iterable positions) throws InterruptedException, Ma public void asyncDelete(Iterable position, DeleteCallback callback, Object ctx) { } - @Override - public void asyncDelete(Position position, int batchSize, List deleteIndexRanges, DeleteCallback callback, Object ctx) { - } - @Override public void clearBacklog() throws InterruptedException, ManagedLedgerException { } @@ -332,8 +328,8 @@ public void trimDeletedEntries(List entries) { } @Override - public List getDeletedBatchIndexes(Position position) { - return null; + public long[] getDeletedBatchIndexesLongArray(PositionImpl position) { + return new long[0]; } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 9eac3bc848946..8c1d02534bafb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -38,6 +38,7 @@ import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Arrays; +import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -3065,19 +3066,19 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc } assertEquals(cursor.getNumberOfEntries(), totalEntries); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(2).setEnd(4).build())); - List deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(3).setEnd(8).build())); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(8, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(0).build())); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(2, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(0, deletedIndexes.get(0).getEnd()); @@ -3086,24 +3087,24 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(1).setEnd(1).build())); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(9).setEnd(9).build())); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[0]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(positions[0], cursor.getMarkDeletedPosition()); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.delete(positions[1]); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(6).setEnd(8).build())); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[1]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[1]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[2], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.markDelete(positions[3]); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[2]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[2]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[3], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.resetCursor(positions[0]); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[3]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[3]), 10); Assert.assertNull(deletedIndexes); } @@ -3128,13 +3129,13 @@ public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerExce ledger = factory.open("test_batch_indexes_deletion_persistent"); cursor = ledger.openCursor("c1"); - List deletedIndexes = cursor.getDeletedBatchIndexes(positions[5]); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[5]), 10); Assert.assertEquals(deletedIndexes.size(), 1); Assert.assertEquals(deletedIndexes.get(0).getStart(), 3); Assert.assertEquals(deletedIndexes.get(0).getEnd(), 6); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[4]); deleteBatchIndex(cursor, positions[5], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); - deletedIndexes = cursor.getDeletedBatchIndexes(positions[5]); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[5]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[5]); } @@ -3142,7 +3143,15 @@ public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerExce private void deleteBatchIndex(ManagedCursor cursor, Position position, int batchSize, List deleteIndexes) throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - cursor.asyncDelete(position, batchSize, deleteIndexes, + PositionImpl pos = (PositionImpl) position; + BitSet bitSet = new BitSet(batchSize); + bitSet.set(0, batchSize); + deleteIndexes.forEach(intRange -> { + bitSet.clear(intRange.getStart(), intRange.getEnd() + 1); + }); + pos.ackSet = bitSet; + + cursor.asyncDelete(pos, new DeleteCallback() { @Override public void deleteComplete(Object ctx) { @@ -3155,7 +3164,27 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) { } }, null); latch.await(); + pos.ackSet = null; + } + + private List getAckedIndexRange(long[] bitSetLongArray, int batchSize) { + if (bitSetLongArray == null) { + return null; + } + List result = new ArrayList<>(); + BitSet bitSet = BitSet.valueOf(bitSetLongArray); + int nextClearBit = bitSet.nextClearBit(0); + IntRange.Builder builder = IntRange.newBuilder(); + while (nextClearBit != -1 && nextClearBit <= batchSize) { + int nextSetBit = bitSet.nextSetBit(nextClearBit); + if (nextSetBit == -1) { + break; + } + result.add(builder.setStart(nextClearBit).setEnd(nextSetBit - 1).build()); + nextClearBit = bitSet.nextClearBit(nextSetBit); + } + return result; } - + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } 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 fa1500008594d..d2b863a17a163 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 @@ -90,7 +90,7 @@ public void filterEntriesForConsumer(List entries, EntryBatchSizes batchS entries.set(i, null); entry.release(); - subscription.acknowledgeMessage(Collections.singletonList(pos), null, AckType.Individual, + subscription.acknowledgeMessage(Collections.singletonList(pos), AckType.Individual, Collections.emptyMap()); continue; } else if (msgMetadata.hasDeliverAtTime() @@ -106,13 +106,9 @@ && trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { totalBytes += metadataAndPayload.readableBytes(); batchSizes.setBatchSize(i, batchSize); if (indexesAcks != null && cursor != null) { - List ranges = cursor.getDeletedBatchIndexes(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); - if (ranges != null) { - int indexesAckedCount = 0; - for (IntRange range : ranges) { - indexesAckedCount += range.getEnd() - range.getStart() + 1; - } - indexesAcks.setIndexesAcks(Pair.of(indexesAckedCount, ranges)); + long[] ackSet = cursor.getDeletedBatchIndexesLongArray(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); + if (ackSet != null) { + indexesAcks.setIndexesAcks(Pair.of(batchSize, ackSet)); } else { indexesAcks.setIndexesAcks(null); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 7ddb13887bf73..7eccd141ebcd1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1056,6 +1056,7 @@ public CompletableFuture getManagedLedgerConfig(TopicName t ); } managedLedgerConfig.setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); + managedLedgerConfig.setBatchIndexDeleteEnabled(serviceConfig.isBatchIndexAcknowledgeEnable()); future.complete(managedLedgerConfig); }, (exception) -> future.completeExceptionally(exception))); 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 3bf55a760f773..733e38b095ddd 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 @@ -30,6 +30,7 @@ import io.netty.channel.ChannelPromise; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; @@ -57,6 +58,7 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.stats.Rate; import org.apache.pulsar.common.util.DateFormatter; +import org.apache.pulsar.common.util.SafeCollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -273,12 +275,8 @@ public ChannelPromise sendMessages(final List entries, EntryBatchSizes ba if (redeliveryTracker.contains(position)) { redeliveryCount = redeliveryTracker.incrementAndGetRedeliveryCount(position); } - - List ackedIndexRanges = null; - if (batchIndexesAcks != null && batchIndexesAcks.getIndexesAcks(i) != null) { - ackedIndexRanges = batchIndexesAcks.getIndexesAcks(i).getRight(); - } - ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload, ackedIndexRanges), ctx.voidPromise()); + ctx.write(Commands.newMessage(consumerId, messageId, redeliveryCount, metadataAndPayload, + batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i)), ctx.voidPromise()); messageId.recycle(); messageIdBuilder.recycle(); entry.release(); @@ -364,7 +362,7 @@ void messageAcked(CommandAck ack) { } if (ack.getAckType() == AckType.Cumulative) { - if (ack.getMessageIdCount() != 1 && ack.getBatchMessageAckIndexesCount() != 1) { + if (ack.getMessageIdCount() != 1) { log.warn("[{}] [{}] Received multi-message ack", subscription, consumerId); return; } @@ -376,18 +374,27 @@ void messageAcked(CommandAck ack) { PositionImpl position = PositionImpl.earliest; if (ack.getMessageIdCount() == 1) { MessageIdData msgId = ack.getMessageId(0); - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + if (msgId.getAckSetCount() > 0) { + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + } else { + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + } } - subscription.acknowledgeMessage(Collections.singletonList(position), ack.getBatchMessageAckIndexesList(), AckType.Cumulative, properties); + subscription.acknowledgeMessage(Collections.singletonList(position), AckType.Cumulative, properties); } else { // Individual ack List positionsAcked = new ArrayList<>(); for (int i = 0; i < ack.getMessageIdCount(); i++) { MessageIdData msgId = ack.getMessageId(i); - PositionImpl position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + PositionImpl position; + if (msgId.getAckSetCount() > 0) { + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + } else { + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + } positionsAcked.add(position); - if (Subscription.isIndividualAckMode(subType)) { + if (Subscription.isIndividualAckMode(subType) && msgId.getAckSetCount() == 0) { removePendingAcks(position); } @@ -396,7 +403,7 @@ void messageAcked(CommandAck ack) { consumerId, position, ack.getValidationError()); } } - subscription.acknowledgeMessage(positionsAcked, ack.getBatchMessageAckIndexesList(), AckType.Individual, properties); + subscription.acknowledgeMessage(positionsAcked, AckType.Individual, properties); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java index 1c915852f6aca..85b54152ad9db 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Subscription.java @@ -25,7 +25,6 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; -import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot; @@ -46,7 +45,7 @@ default void removeConsumer(Consumer consumer) throws BrokerServiceException { void consumerFlow(Consumer consumer, int additionalNumberOfMessages); - void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties); + void acknowledgeMessage(List positions, AckType ackType, Map properties); String getTopicName(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java index e1fd8f1777b23..73152c32370ed 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentSubscription.java @@ -38,7 +38,6 @@ import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.naming.TopicName; @@ -186,7 +185,7 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { } @Override - public void acknowledgeMessage(List position, List batchMessageAckIndexes, AckType ackType, Map properties) { + public void acknowledgeMessage(List position, AckType ackType, Map properties) { // No-op } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java index b0d64ca7c423d..94af8063f5ad4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/CompactorSubscription.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.service.Consumer; -import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.compaction.CompactedTopic; import org.apache.pulsar.compaction.Compactor; @@ -57,7 +56,7 @@ public CompactorSubscription(PersistentTopic topic, CompactedTopic compactedTopi } @Override - public void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties) { + public void acknowledgeMessage(List positions, AckType ackType, Map properties) { checkArgument(ackType == AckType.Cumulative); checkArgument(positions.size() == 1); checkArgument(properties.containsKey(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java index 2544a013930f6..908cffd8f5b77 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentSubscription.java @@ -48,7 +48,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap; -import org.apache.commons.collections.CollectionUtils; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.SubscriptionBusyException; @@ -60,7 +59,6 @@ import org.apache.pulsar.broker.service.HashRangeExclusiveStickyKeyConsumerSelector; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType; import org.apache.pulsar.common.api.proto.PulsarMarkers.ReplicatedSubscriptionsSnapshot; @@ -108,8 +106,6 @@ public class PersistentSubscription implements Subscription { // This parameter only keep the the largest Position it cumulative ack,as any Position smaller will also be covered. private volatile Position pendingCumulativeAckMessage; - private boolean enableBatchLocalIndexAck = false; - private static final AtomicReferenceFieldUpdater POSITION_UPDATER = AtomicReferenceFieldUpdater.newUpdater(PersistentSubscription.class, Position.class, "pendingCumulativeAckMessage"); @@ -144,17 +140,11 @@ static boolean isCursorFromReplicatedSubscription(ManagedCursor cursor) { } public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - boolean replicated) { - this(topic, subscriptionName, cursor, replicated, false); - } - - public PersistentSubscription(PersistentTopic topic, String subscriptionName, ManagedCursor cursor, - boolean replicated, boolean enableBatchLocalIndexAck) { + boolean replicated) { this.topic = topic; this.cursor = cursor; this.topicName = topic.getName(); this.subName = subscriptionName; - this.enableBatchLocalIndexAck = enableBatchLocalIndexAck; this.fullName = MoreObjects.toStringHelper(this).add("topic", topicName).add("name", subName).toString();; this.expiryMonitor = new PersistentMessageExpiryMonitor(topicName, subscriptionName, cursor); this.setReplicated(replicated); @@ -331,7 +321,7 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) { } @Override - public void acknowledgeMessage(List positions, List batchMessageAckIndexes, AckType ackType, Map properties) { + public void acknowledgeMessage(List positions, AckType ackType, Map properties) { Position previousMarkDeletePosition = cursor.getMarkDeletedPosition(); if (ackType == AckType.Cumulative) { @@ -352,15 +342,6 @@ public void acknowledgeMessage(List positions, List positions, List cursor.asyncDelete( - PositionImpl.get(ackedIndex.getMessageId().getLedgerId(), ackedIndex.getMessageId().getEntryId()), - ackedIndex.getBatchSize(), - ackedIndex.getAckIndexesList(), - deleteCallback, - null)); - } } if (!cursor.getMarkDeletedPosition().equals(previousMarkDeletePosition)) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 67c911ff49f28..5a1814ca86c9b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -312,8 +312,7 @@ private PersistentSubscription createPersistentSubscription(String subscriptionN if (subscriptionName.equals(Compactor.COMPACTION_SUBSCRIPTION)) { return new CompactorSubscription(this, compactedTopic, subscriptionName, cursor); } else { - return new PersistentSubscription(this, subscriptionName, cursor, replicated, - brokerService.getPulsar().getConfiguration().isBatchIndexAcknowledgeEnable()); + return new PersistentSubscription(this, subscriptionName, cursor, replicated); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java index c2c13d4be935b..5a8875770451a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java @@ -174,7 +174,7 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { PersistentSubscription sub = topic.getSubscription(update.getSubscriptionName()); if (sub != null) { - sub.acknowledgeMessage(Collections.singletonList(pos), null, AckType.Cumulative, Collections.emptyMap()); + sub.acknowledgeMessage(Collections.singletonList(pos), AckType.Cumulative, Collections.emptyMap()); } else { // Subscription doesn't exist. We need to force the creation of the subscription in this cluster, because log.info("[{}][{}] Creating subscription at {}:{} after receiving update from replicated subcription", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java index e040a5721d4e8..37cdec6371c66 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicTest.java @@ -1417,7 +1417,7 @@ public void testCompactorSubscription() throws Exception { cursorMock); PositionImpl position = new PositionImpl(1, 1); long ledgerId = 0xc0bfefeL; - sub.acknowledgeMessage(Collections.singletonList(position), null, AckType.Cumulative, + sub.acknowledgeMessage(Collections.singletonList(position), AckType.Cumulative, ImmutableMap.of(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY, ledgerId)); verify(compactedTopic, Mockito.times(1)).newCompactedLedger(position, ledgerId); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java index cb913fb9348b3..44c956f1a2bc6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ServerCnxTest.java @@ -1249,7 +1249,7 @@ public void testAckCommand() throws Exception { PositionImpl pos = new PositionImpl(0, 0); - clientCommand = Commands.newAck(1 /* consumer id */, pos.getLedgerId(), pos.getEntryId(), AckType.Individual, + clientCommand = Commands.newAck(1 /* consumer id */, pos.getLedgerId(), pos.getEntryId(), null, AckType.Individual, null, Collections.emptyMap()); channel.writeInbound(clientCommand); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java index 34bbb5b769b79..038b22df7dad5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentSubscriptionTest.java @@ -273,7 +273,7 @@ public void testCanAcknowledgeAndAbortForTransaction() throws TransactionConflic positions.add(new PositionImpl(3, 5)); // Acknowledge from normal consumer will succeed ignoring message acked by ongoing transaction. - persistentSubscription.acknowledgeMessage(positions, null, AckType.Individual, Collections.emptyMap()); + persistentSubscription.acknowledgeMessage(positions, AckType.Individual, Collections.emptyMap()); //Abort txn. persistentSubscription.abortTxn(txnID1, consumerMock); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index fefb646a119b4..07f3341323b50 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -96,6 +97,7 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.SafeCollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -989,7 +991,7 @@ void activeConsumerChanged(boolean isActive) { }); } - void messageReceived(MessageIdData messageId, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf headersAndPayload, ClientCnx cnx) { + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackSet, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received message: {}/{}", topic, subscription, messageId.getLedgerId(), messageId.getEntryId()); @@ -1081,7 +1083,7 @@ void messageReceived(MessageIdData messageId, int redeliveryCount, List message, final CompletableFut listenerExecutor.execute(() -> receivedFuture.complete(interceptMessage)); } - void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf uncompressedPayload, + void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, List ackSet, ByteBuf uncompressedPayload, MessageIdData messageId, ClientCnx cnx) { int batchSize = msgMetadata.getNumMessagesInBatch(); @@ -1218,17 +1220,7 @@ void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliv continue; } - boolean isAcked = false; - if (ackedBatchIndexRanges != null) { - for (IntRange range : ackedBatchIndexRanges) { - if (i >= range.getStart() && i <= range.getEnd()) { - isAcked = true; - break; - } - } - } - - if (isAcked) { + if (ackSet != null && BitSet.valueOf(SafeCollectionUtils.longListToArray(ackSet)).get(i)) { singleMessagePayload.release(); singleMessageMetadataBuilder.recycle(); ++skippedMessages; @@ -1469,7 +1461,7 @@ private void discardCorruptedMessage(MessageIdData messageId, ClientCnx currentC } private void discardMessage(MessageIdData messageId, ClientCnx currentCnx, ValidationError validationError) { - ByteBuf cmd = Commands.newAck(consumerId, messageId.getLedgerId(), messageId.getEntryId(), AckType.Individual, + ByteBuf cmd = Commands.newAck(consumerId, messageId.getLedgerId(), messageId.getEntryId(), null, AckType.Individual, validationError, Collections.emptyMap()); currentCnx.ctx().writeAndFlush(cmd, currentCnx.ctx().voidPromise()); increaseAvailablePermits(currentCnx); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 48e84d82e8410..64acdbb92ce56 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.impl; -import com.google.common.collect.Lists; import io.netty.buffer.ByteBuf; import io.netty.channel.EventLoopGroup; @@ -37,12 +36,11 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; -import org.apache.pulsar.common.protocol.Commands.SingleBatchMessageIndexesAck; import org.apache.pulsar.common.util.collections.ConcurrentBitSet; /** @@ -64,17 +62,21 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * Latest cumulative ack sent to broker */ private volatile MessageIdImpl lastCumulativeAck = (MessageIdImpl) MessageId.earliest; + private volatile ConcurrentBitSet lastCumulativeAckSet = null; private volatile boolean cumulativeAckFlushRequired = false; private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_UPDATER = AtomicReferenceFieldUpdater .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, MessageIdImpl.class, "lastCumulativeAck"); + private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_SET_UPDATER = AtomicReferenceFieldUpdater + .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, ConcurrentBitSet.class, "lastCumulativeAckSet"); + /** * This is a set of all the individual acks that the application has issued and that were not already sent to * broker. */ private final ConcurrentSkipListSet pendingIndividualAcks; - private final ConcurrentHashMap> pendingIndividualBatchIndexAcks; + private final ConcurrentHashMap pendingIndividualBatchIndexAcks; private final ScheduledFuture scheduledTask; @@ -112,7 +114,7 @@ public void addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map= MAX_ACK_GROUP_SIZE) { flush(); } @@ -129,25 +132,35 @@ public void addAcknowledgment(MessageIdImpl msgId, AckType ackType, Map properties) { - if (acknowledgementGroupTimeMicros == 0 || !properties.isEmpty() || ackType == AckType.Cumulative) { + if (acknowledgementGroupTimeMicros == 0 || !properties.isEmpty()) { doImmediateBatchIndexAck(msgId, batchIndex, batchSize, ackType, properties); + } else if (ackType == AckType.Cumulative) { + ConcurrentBitSet bitSet = new ConcurrentBitSet(batchSize); + bitSet.set(0, batchSize); + bitSet.clear(0, batchIndex + 1); + doCumulativeAck(msgId, bitSet); } else if (ackType == AckType.Individual) { ConcurrentBitSet bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( - new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), msgId.getPartitionIndex()), (v) -> - Pair.of(batchSize, new ConcurrentBitSet(msgId.getBatchSize()))).getRight(); - bitSet.set(batchIndex); + new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), msgId.getPartitionIndex()), (v) -> { + ConcurrentBitSet value = new ConcurrentBitSet(batchSize); + value.set(0, batchSize + 1); + value.clear(batchIndex); + return value; + }); + bitSet.clear(batchIndex); if (pendingIndividualBatchIndexAcks.size() >= MAX_ACK_GROUP_SIZE) { flush(); } } } - private void doCumulativeAck(MessageIdImpl msgId) { + private void doCumulativeAck(MessageIdImpl msgId, ConcurrentBitSet bitSet) { // Handle concurrent updates from different threads while (true) { MessageIdImpl lastCumlativeAck = this.lastCumulativeAck; + ConcurrentBitSet lastBitSet = this.lastCumulativeAckSet; if (msgId.compareTo(lastCumlativeAck) > 0) { - if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId)) { + if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId) && LAST_CUMULATIVE_ACK_SET_UPDATER.compareAndSet(this, lastBitSet, bitSet)) { // Successfully updated the last cumulative ack. Next flush iteration will send this to broker. cumulativeAckFlushRequired = true; return; @@ -166,7 +179,7 @@ private boolean doImmediateAck(MessageIdImpl msgId, AckType ackType, Map> entriesToAck = new ArrayList<>(pendingIndividualAcks.size() + pendingIndividualBatchIndexAcks.size()); if (!pendingIndividualAcks.isEmpty()) { if (Commands.peerSupportsMultiMessageAcknowledgment(cnx.getRemoteEndpointProtocolVersion())) { // We can send 1 single protobuf command with all individual acks - List> entriesToAck = new ArrayList<>(pendingIndividualAcks.size()); while (true) { MessageIdImpl msgId = pendingIndividualAcks.pollFirst(); if (msgId == null) { break; } - entriesToAck.add(Pair.of(msgId.getLedgerId(), msgId.getEntryId())); + entriesToAck.add(Triple.of(msgId.getLedgerId(), msgId.getEntryId(), null)); } - - cnx.ctx().write(Commands.newMultiMessageAck(consumer.consumerId, entriesToAck), - cnx.ctx().voidPromise()); - shouldFlush = true; } else { // When talking to older brokers, send the acknowledgements individually while (true) { @@ -240,7 +248,7 @@ public void flush() { break; } - cnx.ctx().write(Commands.newAck(consumer.consumerId, msgId.getLedgerId(), msgId.getEntryId(), + cnx.ctx().write(Commands.newAck(consumer.consumerId, msgId.getLedgerId(), msgId.getEntryId(), null, AckType.Individual, null, Collections.emptyMap()), cnx.ctx().voidPromise()); shouldFlush = true; } @@ -248,28 +256,18 @@ public void flush() { } if (!pendingIndividualBatchIndexAcks.isEmpty()) { - Iterator>> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); - List batchIndexesAcks = new ArrayList<>(pendingIndividualBatchIndexAcks.size()); - IntRange.Builder rangeBuilder = IntRange.newBuilder(); + Iterator> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); + while (iterator.hasNext()) { - Map.Entry> entry = iterator.next(); - SingleBatchMessageIndexesAck ack = new SingleBatchMessageIndexesAck(); - ack.setLedgerId(entry.getKey().getLedgerId()); - ack.setEntryId(entry.getKey().getEntryId()); - ack.setBatchSize(entry.getValue().getLeft()); - ack.setIndexRangesToAck(Lists.newArrayList()); - BitSet bitSet = entry.getValue().getRight(); - int nextSetBit = bitSet.nextSetBit(0); - while (nextSetBit != -1) { - int nextClearBit = bitSet.nextClearBit(nextSetBit); - ack.getIndexRangesToAck().add(rangeBuilder.setStart(nextSetBit).setEnd(nextClearBit - 1).build()); - nextSetBit = bitSet.nextSetBit(nextClearBit); - } - batchIndexesAcks.add(ack); + Map.Entry entry = iterator.next(); + entriesToAck.add(Triple.of(entry.getKey().ledgerId, entry.getKey().entryId, entry.getValue())); iterator.remove(); } - cnx.ctx().write(Commands.newBatchIndexAck(consumer.consumerId, batchIndexesAcks, AckType.Individual, null, - null)); + } + + if (entriesToAck.size() > 0) { + cnx.ctx().write(Commands.newMultiMessageAck(consumer.consumerId, entriesToAck), + cnx.ctx().voidPromise()); shouldFlush = true; } 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 deleted file mode 100644 index ef259d7c698cb..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java +++ /dev/null @@ -1,44375 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: src/main/proto/PulsarApi.proto - -package org.apache.pulsar.common.api.proto; - -public final class PulsarApi { - private PulsarApi() {} - public static void registerAllExtensions( - org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite registry) { - } - public enum CompressionType - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - NONE(0, 0), - LZ4(1, 1), - ZLIB(2, 2), - ZSTD(3, 3), - SNAPPY(4, 4), - ; - - public static final int NONE_VALUE = 0; - public static final int LZ4_VALUE = 1; - public static final int ZLIB_VALUE = 2; - public static final int ZSTD_VALUE = 3; - public static final int SNAPPY_VALUE = 4; - - - public final int getNumber() { return value; } - - public static CompressionType valueOf(int value) { - switch (value) { - case 0: return NONE; - case 1: return LZ4; - case 2: return ZLIB; - case 3: return ZSTD; - case 4: return SNAPPY; - 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 CompressionType findValueByNumber(int number) { - return CompressionType.valueOf(number); - } - }; - - private final int value; - - private CompressionType(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CompressionType) - } - - public enum ServerError - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - UnknownError(0, 0), - MetadataError(1, 1), - PersistenceError(2, 2), - AuthenticationError(3, 3), - AuthorizationError(4, 4), - ConsumerBusy(5, 5), - ServiceNotReady(6, 6), - ProducerBlockedQuotaExceededError(7, 7), - ProducerBlockedQuotaExceededException(8, 8), - ChecksumError(9, 9), - UnsupportedVersionError(10, 10), - TopicNotFound(11, 11), - SubscriptionNotFound(12, 12), - ConsumerNotFound(13, 13), - TooManyRequests(14, 14), - TopicTerminatedError(15, 15), - ProducerBusy(16, 16), - InvalidTopicName(17, 17), - IncompatibleSchema(18, 18), - ConsumerAssignError(19, 19), - TransactionCoordinatorNotFound(20, 20), - InvalidTxnStatus(21, 21), - ; - - public static final int UnknownError_VALUE = 0; - public static final int MetadataError_VALUE = 1; - public static final int PersistenceError_VALUE = 2; - public static final int AuthenticationError_VALUE = 3; - public static final int AuthorizationError_VALUE = 4; - public static final int ConsumerBusy_VALUE = 5; - public static final int ServiceNotReady_VALUE = 6; - public static final int ProducerBlockedQuotaExceededError_VALUE = 7; - public static final int ProducerBlockedQuotaExceededException_VALUE = 8; - public static final int ChecksumError_VALUE = 9; - public static final int UnsupportedVersionError_VALUE = 10; - public static final int TopicNotFound_VALUE = 11; - public static final int SubscriptionNotFound_VALUE = 12; - public static final int ConsumerNotFound_VALUE = 13; - public static final int TooManyRequests_VALUE = 14; - public static final int TopicTerminatedError_VALUE = 15; - public static final int ProducerBusy_VALUE = 16; - public static final int InvalidTopicName_VALUE = 17; - public static final int IncompatibleSchema_VALUE = 18; - public static final int ConsumerAssignError_VALUE = 19; - public static final int TransactionCoordinatorNotFound_VALUE = 20; - public static final int InvalidTxnStatus_VALUE = 21; - - - public final int getNumber() { return value; } - - public static ServerError valueOf(int value) { - switch (value) { - case 0: return UnknownError; - case 1: return MetadataError; - case 2: return PersistenceError; - case 3: return AuthenticationError; - case 4: return AuthorizationError; - case 5: return ConsumerBusy; - case 6: return ServiceNotReady; - case 7: return ProducerBlockedQuotaExceededError; - case 8: return ProducerBlockedQuotaExceededException; - case 9: return ChecksumError; - case 10: return UnsupportedVersionError; - case 11: return TopicNotFound; - case 12: return SubscriptionNotFound; - case 13: return ConsumerNotFound; - case 14: return TooManyRequests; - case 15: return TopicTerminatedError; - case 16: return ProducerBusy; - case 17: return InvalidTopicName; - case 18: return IncompatibleSchema; - case 19: return ConsumerAssignError; - case 20: return TransactionCoordinatorNotFound; - case 21: return InvalidTxnStatus; - 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 ServerError findValueByNumber(int number) { - return ServerError.valueOf(number); - } - }; - - private final int value; - - private ServerError(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.ServerError) - } - - public enum AuthMethod - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - AuthMethodNone(0, 0), - AuthMethodYcaV1(1, 1), - AuthMethodAthens(2, 2), - ; - - public static final int AuthMethodNone_VALUE = 0; - public static final int AuthMethodYcaV1_VALUE = 1; - public static final int AuthMethodAthens_VALUE = 2; - - - public final int getNumber() { return value; } - - public static AuthMethod valueOf(int value) { - switch (value) { - case 0: return AuthMethodNone; - case 1: return AuthMethodYcaV1; - case 2: return AuthMethodAthens; - 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 AuthMethod findValueByNumber(int number) { - return AuthMethod.valueOf(number); - } - }; - - private final int value; - - private AuthMethod(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.AuthMethod) - } - - public enum ProtocolVersion - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - v0(0, 0), - v1(1, 1), - v2(2, 2), - v3(3, 3), - v4(4, 4), - v5(5, 5), - v6(6, 6), - v7(7, 7), - v8(8, 8), - v9(9, 9), - v10(10, 10), - v11(11, 11), - v12(12, 12), - v13(13, 13), - v14(14, 14), - v15(15, 15), - ; - - public static final int v0_VALUE = 0; - public static final int v1_VALUE = 1; - public static final int v2_VALUE = 2; - public static final int v3_VALUE = 3; - public static final int v4_VALUE = 4; - public static final int v5_VALUE = 5; - public static final int v6_VALUE = 6; - public static final int v7_VALUE = 7; - public static final int v8_VALUE = 8; - public static final int v9_VALUE = 9; - public static final int v10_VALUE = 10; - public static final int v11_VALUE = 11; - public static final int v12_VALUE = 12; - public static final int v13_VALUE = 13; - public static final int v14_VALUE = 14; - public static final int v15_VALUE = 15; - - - public final int getNumber() { return value; } - - public static ProtocolVersion valueOf(int value) { - switch (value) { - case 0: return v0; - case 1: return v1; - case 2: return v2; - case 3: return v3; - case 4: return v4; - case 5: return v5; - case 6: return v6; - case 7: return v7; - case 8: return v8; - case 9: return v9; - case 10: return v10; - case 11: return v11; - case 12: return v12; - case 13: return v13; - case 14: return v14; - case 15: return v15; - 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 ProtocolVersion findValueByNumber(int number) { - return ProtocolVersion.valueOf(number); - } - }; - - private final int value; - - private ProtocolVersion(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.ProtocolVersion) - } - - public enum KeySharedMode - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - AUTO_SPLIT(0, 0), - STICKY(1, 1), - ; - - public static final int AUTO_SPLIT_VALUE = 0; - public static final int STICKY_VALUE = 1; - - - public final int getNumber() { return value; } - - public static KeySharedMode valueOf(int value) { - switch (value) { - case 0: return AUTO_SPLIT; - case 1: return STICKY; - 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 KeySharedMode findValueByNumber(int number) { - return KeySharedMode.valueOf(number); - } - }; - - private final int value; - - private KeySharedMode(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.KeySharedMode) - } - - public enum TxnAction - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - COMMIT(0, 0), - ABORT(1, 1), - ; - - public static final int COMMIT_VALUE = 0; - public static final int ABORT_VALUE = 1; - - - public final int getNumber() { return value; } - - public static TxnAction valueOf(int value) { - switch (value) { - case 0: return COMMIT; - case 1: return ABORT; - 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 TxnAction findValueByNumber(int number) { - return TxnAction.valueOf(number); - } - }; - - private final int value; - - private TxnAction(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.TxnAction) - } - - public interface SchemaOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string name = 1; - boolean hasName(); - String getName(); - - // required bytes schema_data = 3; - boolean hasSchemaData(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData(); - - // required .pulsar.proto.Schema.Type type = 4; - boolean hasType(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType(); - - // repeated .pulsar.proto.KeyValue properties = 5; - java.util.List - getPropertiesList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); - int getPropertiesCount(); - } - public static final class Schema extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements SchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use Schema.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private Schema(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected Schema newObject(Handle handle) { - return new Schema(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private Schema(boolean noInit) {} - - private static final Schema defaultInstance; - public static Schema getDefaultInstance() { - return defaultInstance; - } - - public Schema getDefaultInstanceForType() { - return defaultInstance; - } - - public enum Type - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - None(0, 0), - String(1, 1), - Json(2, 2), - Protobuf(3, 3), - Avro(4, 4), - Bool(5, 5), - Int8(6, 6), - Int16(7, 7), - Int32(8, 8), - Int64(9, 9), - Float(10, 10), - Double(11, 11), - Date(12, 12), - Time(13, 13), - Timestamp(14, 14), - KeyValue(15, 15), - ; - - public static final int None_VALUE = 0; - public static final int String_VALUE = 1; - public static final int Json_VALUE = 2; - public static final int Protobuf_VALUE = 3; - public static final int Avro_VALUE = 4; - public static final int Bool_VALUE = 5; - public static final int Int8_VALUE = 6; - public static final int Int16_VALUE = 7; - public static final int Int32_VALUE = 8; - public static final int Int64_VALUE = 9; - public static final int Float_VALUE = 10; - public static final int Double_VALUE = 11; - public static final int Date_VALUE = 12; - public static final int Time_VALUE = 13; - public static final int Timestamp_VALUE = 14; - public static final int KeyValue_VALUE = 15; - - - public final int getNumber() { return value; } - - public static Type valueOf(int value) { - switch (value) { - case 0: return None; - case 1: return String; - case 2: return Json; - case 3: return Protobuf; - case 4: return Avro; - case 5: return Bool; - case 6: return Int8; - case 7: return Int16; - case 8: return Int32; - case 9: return Int64; - case 10: return Float; - case 11: return Double; - case 12: return Date; - case 13: return Time; - case 14: return Timestamp; - case 15: return KeyValue; - 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 Type findValueByNumber(int number) { - return Type.valueOf(number); - } - }; - - private final int value; - - private Type(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.Schema.Type) - } - - private int bitField0_; - // required string name = 1; - public static final int NAME_FIELD_NUMBER = 1; - private java.lang.Object name_; - public boolean hasName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getName() { - java.lang.Object ref = name_; - 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)) { - name_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getNameBytes() { - java.lang.Object ref = name_; - 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); - name_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required bytes schema_data = 3; - public static final int SCHEMA_DATA_FIELD_NUMBER = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaData_; - public boolean hasSchemaData() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData() { - return schemaData_; - } - - // required .pulsar.proto.Schema.Type type = 4; - public static final int TYPE_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type type_; - public boolean hasType() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType() { - return type_; - } - - // repeated .pulsar.proto.KeyValue properties = 5; - public static final int PROPERTIES_FIELD_NUMBER = 5; - private java.util.List properties_; - public java.util.List getPropertiesList() { - return properties_; - } - public java.util.List - getPropertiesOrBuilderList() { - return properties_; - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( - int index) { - return properties_.get(index); - } - - private void initFields() { - name_ = ""; - schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; - properties_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasName()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSchemaData()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasType()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(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, getNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(3, schemaData_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(4, type_.getNumber()); - } - for (int i = 0; i < properties_.size(); i++) { - output.writeMessage(5, properties_.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, getNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, schemaData_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, type_.getNumber()); - } - for (int i = 0; i < properties_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(5, properties_.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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.SchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.Schema.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(); - name_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); - type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; - bitField0_ = (bitField0_ & ~0x00000004); - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Schema build() { - org.apache.pulsar.common.api.proto.PulsarApi.Schema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.Schema buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.Schema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Schema buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.Schema result = org.apache.pulsar.common.api.proto.PulsarApi.Schema.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.name_ = name_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.schemaData_ = schemaData_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.type_ = type_; - if (((bitField0_ & 0x00000008) == 0x00000008)) { - properties_ = java.util.Collections.unmodifiableList(properties_); - bitField0_ = (bitField0_ & ~0x00000008); - } - result.properties_ = properties_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.Schema other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) return this; - if (other.hasName()) { - setName(other.getName()); - } - if (other.hasSchemaData()) { - setSchemaData(other.getSchemaData()); - } - if (other.hasType()) { - setType(other.getType()); - } - if (!other.properties_.isEmpty()) { - if (properties_.isEmpty()) { - properties_ = other.properties_; - bitField0_ = (bitField0_ & ~0x00000008); - } else { - ensurePropertiesIsMutable(); - properties_.addAll(other.properties_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasName()) { - - return false; - } - if (!hasSchemaData()) { - - return false; - } - if (!hasType()) { - - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(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; - name_ = input.readBytes(); - break; - } - case 26: { - bitField0_ |= 0x00000002; - schemaData_ = input.readBytes(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type value = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - type_ = value; - } - break; - } - case 42: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addProperties(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required string name = 1; - private java.lang.Object name_ = ""; - public boolean hasName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getName() { - java.lang.Object ref = name_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - name_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - name_ = value; - - return this; - } - public Builder clearName() { - bitField0_ = (bitField0_ & ~0x00000001); - name_ = getDefaultInstance().getName(); - - return this; - } - void setName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - name_ = value; - - } - - // required bytes schema_data = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaData() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData() { - return schemaData_; - } - public Builder setSchemaData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - schemaData_ = value; - - return this; - } - public Builder clearSchemaData() { - bitField0_ = (bitField0_ & ~0x00000002); - schemaData_ = getDefaultInstance().getSchemaData(); - - return this; - } - - // required .pulsar.proto.Schema.Type type = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; - public boolean hasType() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType() { - return type_; - } - public Builder setType(org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - type_ = value; - - return this; - } - public Builder clearType() { - bitField0_ = (bitField0_ & ~0x00000004); - type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; - - return this; - } - - // repeated .pulsar.proto.KeyValue properties = 5; - private java.util.List properties_ = - java.util.Collections.emptyList(); - private void ensurePropertiesIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { - properties_ = new java.util.ArrayList(properties_); - bitField0_ |= 0x00000008; - } - } - - public java.util.List getPropertiesList() { - return java.util.Collections.unmodifiableList(properties_); - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.set(index, value); - - return this; - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.set(index, builderForValue.build()); - - return this; - } - public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(value); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(index, value); - - return this; - } - public Builder addProperties( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(builderForValue.build()); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllProperties( - java.lang.Iterable values) { - ensurePropertiesIsMutable(); - super.addAll(values, properties_); - - return this; - } - public Builder clearProperties() { - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - - return this; - } - public Builder removeProperties(int index) { - ensurePropertiesIsMutable(); - properties_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.Schema) - } - - static { - defaultInstance = new Schema(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.Schema) - } - - public interface MessageIdDataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 ledgerId = 1; - boolean hasLedgerId(); - long getLedgerId(); - - // required uint64 entryId = 2; - boolean hasEntryId(); - long getEntryId(); - - // optional int32 partition = 3 [default = -1]; - boolean hasPartition(); - int getPartition(); - - // optional int32 batch_index = 4 [default = -1]; - boolean hasBatchIndex(); - int getBatchIndex(); - } - 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 ledgerId = 1; - public static final int LEDGERID_FIELD_NUMBER = 1; - private long ledgerId_; - public boolean hasLedgerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getLedgerId() { - return ledgerId_; - } - - // required uint64 entryId = 2; - public static final int ENTRYID_FIELD_NUMBER = 2; - private long entryId_; - public boolean hasEntryId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getEntryId() { - return entryId_; - } - - // optional int32 partition = 3 [default = -1]; - public static final int PARTITION_FIELD_NUMBER = 3; - private int partition_; - public boolean hasPartition() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getPartition() { - return partition_; - } - - // optional int32 batch_index = 4 [default = -1]; - public static final int BATCH_INDEX_FIELD_NUMBER = 4; - private int batchIndex_; - public boolean hasBatchIndex() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getBatchIndex() { - return batchIndex_; - } - - private void initFields() { - ledgerId_ = 0L; - entryId_ = 0L; - partition_ = -1; - batchIndex_ = -1; - } - 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_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt32(3, partition_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeInt32(4, batchIndex_); - } - } - - 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_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, partition_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(4, batchIndex_); - } - 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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.MessageIdData parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.MessageIdData, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.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); - partition_ = -1; - bitField0_ = (bitField0_ & ~0x00000004); - batchIndex_ = -1; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData build() { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = org.apache.pulsar.common.api.proto.PulsarApi.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_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.partition_ = partition_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.batchIndex_ = batchIndex_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) return this; - if (other.hasLedgerId()) { - setLedgerId(other.getLedgerId()); - } - if (other.hasEntryId()) { - setEntryId(other.getEntryId()); - } - if (other.hasPartition()) { - setPartition(other.getPartition()); - } - if (other.hasBatchIndex()) { - setBatchIndex(other.getBatchIndex()); - } - 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; - } - case 24: { - bitField0_ |= 0x00000004; - partition_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - batchIndex_ = input.readInt32(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 ledgerId = 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 entryId = 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; - } - - // optional int32 partition = 3 [default = -1]; - private int partition_ = -1; - public boolean hasPartition() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getPartition() { - return partition_; - } - public Builder setPartition(int value) { - bitField0_ |= 0x00000004; - partition_ = value; - - return this; - } - public Builder clearPartition() { - bitField0_ = (bitField0_ & ~0x00000004); - partition_ = -1; - - return this; - } - - // optional int32 batch_index = 4 [default = -1]; - private int batchIndex_ = -1; - public boolean hasBatchIndex() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getBatchIndex() { - return batchIndex_; - } - public Builder setBatchIndex(int value) { - bitField0_ |= 0x00000008; - batchIndex_ = value; - - return this; - } - public Builder clearBatchIndex() { - bitField0_ = (bitField0_ & ~0x00000008); - batchIndex_ = -1; - - 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) - } - - public interface BatchMessageIndexesAckDataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required .pulsar.proto.MessageIdData message_id = 1; - boolean hasMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); - - // required int32 batchSize = 2; - boolean hasBatchSize(); - int getBatchSize(); - - // repeated .pulsar.proto.IntRange ack_indexes = 3; - java.util.List - getAckIndexesList(); - org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index); - int getAckIndexesCount(); - } - public static final class BatchMessageIndexesAckData extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements BatchMessageIndexesAckDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use BatchMessageIndexesAckData.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private BatchMessageIndexesAckData(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected BatchMessageIndexesAckData newObject(Handle handle) { - return new BatchMessageIndexesAckData(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private BatchMessageIndexesAckData(boolean noInit) {} - - private static final BatchMessageIndexesAckData defaultInstance; - public static BatchMessageIndexesAckData getDefaultInstance() { - return defaultInstance; - } - - public BatchMessageIndexesAckData getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required .pulsar.proto.MessageIdData message_id = 1; - public static final int MESSAGE_ID_FIELD_NUMBER = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; - public boolean hasMessageId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - - // required int32 batchSize = 2; - public static final int BATCHSIZE_FIELD_NUMBER = 2; - private int batchSize_; - public boolean hasBatchSize() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getBatchSize() { - return batchSize_; - } - - // repeated .pulsar.proto.IntRange ack_indexes = 3; - public static final int ACK_INDEXES_FIELD_NUMBER = 3; - private java.util.List ackIndexes_; - public java.util.List getAckIndexesList() { - return ackIndexes_; - } - public java.util.List - getAckIndexesOrBuilderList() { - return ackIndexes_; - } - public int getAckIndexesCount() { - return ackIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index) { - return ackIndexes_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getAckIndexesOrBuilder( - int index) { - return ackIndexes_.get(index); - } - - private void initFields() { - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - batchSize_ = 0; - ackIndexes_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasMessageId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasBatchSize()) { - memoizedIsInitialized = 0; - return false; - } - if (!getMessageId().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getAckIndexesCount(); i++) { - if (!getAckIndexes(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.writeMessage(1, messageId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeInt32(2, batchSize_); - } - for (int i = 0; i < ackIndexes_.size(); i++) { - output.writeMessage(3, ackIndexes_.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 - .computeMessageSize(1, messageId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(2, batchSize_); - } - for (int i = 0; i < ackIndexes_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, ackIndexes_.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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData 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.PulsarApi.BatchMessageIndexesAckData, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.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(); - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000001); - batchSize_ = 0; - bitField0_ = (bitField0_ & ~0x00000002); - ackIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData build() { - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData result = org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.messageId_ = messageId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.batchSize_ = batchSize_; - if (((bitField0_ & 0x00000004) == 0x00000004)) { - ackIndexes_ = java.util.Collections.unmodifiableList(ackIndexes_); - bitField0_ = (bitField0_ & ~0x00000004); - } - result.ackIndexes_ = ackIndexes_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.getDefaultInstance()) return this; - if (other.hasMessageId()) { - mergeMessageId(other.getMessageId()); - } - if (other.hasBatchSize()) { - setBatchSize(other.getBatchSize()); - } - if (!other.ackIndexes_.isEmpty()) { - if (ackIndexes_.isEmpty()) { - ackIndexes_ = other.ackIndexes_; - bitField0_ = (bitField0_ & ~0x00000004); - } else { - ensureAckIndexesIsMutable(); - ackIndexes_.addAll(other.ackIndexes_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasMessageId()) { - - return false; - } - if (!hasBatchSize()) { - - return false; - } - if (!getMessageId().isInitialized()) { - - return false; - } - for (int i = 0; i < getAckIndexesCount(); i++) { - if (!getAckIndexes(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: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasMessageId()) { - subBuilder.mergeFrom(getMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - batchSize_ = input.readInt32(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addAckIndexes(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required .pulsar.proto.MessageIdData message_id = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasMessageId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - messageId_ = value; - - bitField0_ |= 0x00000001; - return this; - } - public Builder setMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - messageId_ = builderForValue.build(); - - bitField0_ |= 0x00000001; - return this; - } - public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000001) == 0x00000001) && - messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - messageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); - } else { - messageId_ = value; - } - - bitField0_ |= 0x00000001; - return this; - } - public Builder clearMessageId() { - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - // required int32 batchSize = 2; - private int batchSize_ ; - public boolean hasBatchSize() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getBatchSize() { - return batchSize_; - } - public Builder setBatchSize(int value) { - bitField0_ |= 0x00000002; - batchSize_ = value; - - return this; - } - public Builder clearBatchSize() { - bitField0_ = (bitField0_ & ~0x00000002); - batchSize_ = 0; - - return this; - } - - // repeated .pulsar.proto.IntRange ack_indexes = 3; - private java.util.List ackIndexes_ = - java.util.Collections.emptyList(); - private void ensureAckIndexesIsMutable() { - if (!((bitField0_ & 0x00000004) == 0x00000004)) { - ackIndexes_ = new java.util.ArrayList(ackIndexes_); - bitField0_ |= 0x00000004; - } - } - - public java.util.List getAckIndexesList() { - return java.util.Collections.unmodifiableList(ackIndexes_); - } - public int getAckIndexesCount() { - return ackIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckIndexes(int index) { - return ackIndexes_.get(index); - } - public Builder setAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckIndexesIsMutable(); - ackIndexes_.set(index, value); - - return this; - } - public Builder setAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckIndexesIsMutable(); - ackIndexes_.set(index, builderForValue.build()); - - return this; - } - public Builder addAckIndexes(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckIndexesIsMutable(); - ackIndexes_.add(value); - - return this; - } - public Builder addAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckIndexesIsMutable(); - ackIndexes_.add(index, value); - - return this; - } - public Builder addAckIndexes( - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckIndexesIsMutable(); - ackIndexes_.add(builderForValue.build()); - - return this; - } - public Builder addAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckIndexesIsMutable(); - ackIndexes_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllAckIndexes( - java.lang.Iterable values) { - ensureAckIndexesIsMutable(); - super.addAll(values, ackIndexes_); - - return this; - } - public Builder clearAckIndexes() { - ackIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - - return this; - } - public Builder removeAckIndexes(int index) { - ensureAckIndexesIsMutable(); - ackIndexes_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.BatchMessageIndexesAckData) - } - - static { - defaultInstance = new BatchMessageIndexesAckData(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.BatchMessageIndexesAckData) - } - - public interface KeyValueOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string key = 1; - boolean hasKey(); - String getKey(); - - // required string value = 2; - boolean hasValue(); - String getValue(); - } - public static final class KeyValue extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements KeyValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use KeyValue.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private KeyValue(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected KeyValue newObject(Handle handle) { - return new KeyValue(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private KeyValue(boolean noInit) {} - - private static final KeyValue defaultInstance; - public static KeyValue getDefaultInstance() { - return defaultInstance; - } - - public KeyValue getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string key = 1; - public static final int KEY_FIELD_NUMBER = 1; - private java.lang.Object key_; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - 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)) { - key_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { - java.lang.Object ref = key_; - 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); - key_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required string value = 2; - public static final int VALUE_FIELD_NUMBER = 2; - private java.lang.Object value_; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getValue() { - java.lang.Object ref = value_; - 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)) { - value_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValueBytes() { - java.lang.Object ref = value_; - 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); - value_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - key_ = ""; - value_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasKey()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasValue()) { - 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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getValueBytes()); - } - } - - 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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getValueBytes()); - } - 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.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(); - key_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - value_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue build() { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.KeyValue buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.key_ = key_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.value_ = value_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.getDefaultInstance()) return this; - if (other.hasKey()) { - setKey(other.getKey()); - } - if (other.hasValue()) { - setValue(other.getValue()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasKey()) { - - return false; - } - if (!hasValue()) { - - 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; - key_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - value_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required string key = 1; - private java.lang.Object key_ = ""; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - key_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setKey(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - key_ = value; - - return this; - } - public Builder clearKey() { - bitField0_ = (bitField0_ & ~0x00000001); - key_ = getDefaultInstance().getKey(); - - return this; - } - void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - key_ = value; - - } - - // required string value = 2; - private java.lang.Object value_ = ""; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getValue() { - java.lang.Object ref = value_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - value_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setValue(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - value_ = value; - - return this; - } - public Builder clearValue() { - bitField0_ = (bitField0_ & ~0x00000002); - value_ = getDefaultInstance().getValue(); - - return this; - } - void setValue(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - value_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.KeyValue) - } - - static { - defaultInstance = new KeyValue(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.KeyValue) - } - - public interface KeyLongValueOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string key = 1; - boolean hasKey(); - String getKey(); - - // required uint64 value = 2; - boolean hasValue(); - long getValue(); - } - public static final class KeyLongValue extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements KeyLongValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use KeyLongValue.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private KeyLongValue(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected KeyLongValue newObject(Handle handle) { - return new KeyLongValue(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private KeyLongValue(boolean noInit) {} - - private static final KeyLongValue defaultInstance; - public static KeyLongValue getDefaultInstance() { - return defaultInstance; - } - - public KeyLongValue getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string key = 1; - public static final int KEY_FIELD_NUMBER = 1; - private java.lang.Object key_; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - 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)) { - key_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { - java.lang.Object ref = key_; - 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); - key_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required uint64 value = 2; - public static final int VALUE_FIELD_NUMBER = 2; - private long value_; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getValue() { - return value_; - } - - private void initFields() { - key_ = ""; - value_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasKey()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasValue()) { - 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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, value_); - } - } - - 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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, value_); - } - 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.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(); - key_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - value_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue build() { - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.key_ = key_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.value_ = value_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.getDefaultInstance()) return this; - if (other.hasKey()) { - setKey(other.getKey()); - } - if (other.hasValue()) { - setValue(other.getValue()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasKey()) { - - return false; - } - if (!hasValue()) { - - 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; - key_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - value_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required string key = 1; - private java.lang.Object key_ = ""; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - key_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setKey(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - key_ = value; - - return this; - } - public Builder clearKey() { - bitField0_ = (bitField0_ & ~0x00000001); - key_ = getDefaultInstance().getKey(); - - return this; - } - void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - key_ = value; - - } - - // required uint64 value = 2; - private long value_ ; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getValue() { - return value_; - } - public Builder setValue(long value) { - bitField0_ |= 0x00000002; - value_ = value; - - return this; - } - public Builder clearValue() { - bitField0_ = (bitField0_ & ~0x00000002); - value_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.KeyLongValue) - } - - static { - defaultInstance = new KeyLongValue(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.KeyLongValue) - } - - public interface IntRangeOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required int32 start = 1; - boolean hasStart(); - int getStart(); - - // required int32 end = 2; - boolean hasEnd(); - int getEnd(); - } - public static final class IntRange extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements IntRangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use IntRange.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private IntRange(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected IntRange newObject(Handle handle) { - return new IntRange(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private IntRange(boolean noInit) {} - - private static final IntRange defaultInstance; - public static IntRange getDefaultInstance() { - return defaultInstance; - } - - public IntRange getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required int32 start = 1; - public static final int START_FIELD_NUMBER = 1; - private int start_; - public boolean hasStart() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public int getStart() { - return start_; - } - - // required int32 end = 2; - public static final int END_FIELD_NUMBER = 2; - private int end_; - public boolean hasEnd() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getEnd() { - return end_; - } - - private void initFields() { - start_ = 0; - end_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasStart()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasEnd()) { - 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.writeInt32(1, start_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeInt32(2, end_); - } - } - - 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 - .computeInt32Size(1, start_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(2, end_); - } - 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.IntRange.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(); - start_ = 0; - bitField0_ = (bitField0_ & ~0x00000001); - end_ = 0; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.IntRange.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange build() { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.IntRange buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.start_ = start_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.end_ = end_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.IntRange other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.IntRange.getDefaultInstance()) return this; - if (other.hasStart()) { - setStart(other.getStart()); - } - if (other.hasEnd()) { - setEnd(other.getEnd()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasStart()) { - - return false; - } - if (!hasEnd()) { - - 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; - start_ = input.readInt32(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - end_ = input.readInt32(); - break; - } - } - } - } - - private int bitField0_; - - // required int32 start = 1; - private int start_ ; - public boolean hasStart() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public int getStart() { - return start_; - } - public Builder setStart(int value) { - bitField0_ |= 0x00000001; - start_ = value; - - return this; - } - public Builder clearStart() { - bitField0_ = (bitField0_ & ~0x00000001); - start_ = 0; - - return this; - } - - // required int32 end = 2; - private int end_ ; - public boolean hasEnd() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getEnd() { - return end_; - } - public Builder setEnd(int value) { - bitField0_ |= 0x00000002; - end_ = value; - - return this; - } - public Builder clearEnd() { - bitField0_ = (bitField0_ & ~0x00000002); - end_ = 0; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.IntRange) - } - - static { - defaultInstance = new IntRange(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.IntRange) - } - - public interface EncryptionKeysOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string key = 1; - boolean hasKey(); - String getKey(); - - // required bytes value = 2; - boolean hasValue(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue(); - - // repeated .pulsar.proto.KeyValue metadata = 3; - java.util.List - getMetadataList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); - int getMetadataCount(); - } - public static final class EncryptionKeys extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements EncryptionKeysOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use EncryptionKeys.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private EncryptionKeys(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected EncryptionKeys newObject(Handle handle) { - return new EncryptionKeys(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private EncryptionKeys(boolean noInit) {} - - private static final EncryptionKeys defaultInstance; - public static EncryptionKeys getDefaultInstance() { - return defaultInstance; - } - - public EncryptionKeys getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string key = 1; - public static final int KEY_FIELD_NUMBER = 1; - private java.lang.Object key_; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - 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)) { - key_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { - java.lang.Object ref = key_; - 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); - key_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required bytes value = 2; - public static final int VALUE_FIELD_NUMBER = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value_; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue() { - return value_; - } - - // repeated .pulsar.proto.KeyValue metadata = 3; - public static final int METADATA_FIELD_NUMBER = 3; - private java.util.List metadata_; - public java.util.List getMetadataList() { - return metadata_; - } - public java.util.List - getMetadataOrBuilderList() { - return metadata_; - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( - int index) { - return metadata_.get(index); - } - - private void initFields() { - key_ = ""; - value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - metadata_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasKey()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasValue()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, value_); - } - for (int i = 0; i < metadata_.size(); i++) { - output.writeMessage(3, metadata_.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, getKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, value_); - } - for (int i = 0; i < metadata_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, metadata_.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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeysOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.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(); - key_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys build() { - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.key_ = key_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.value_ = value_; - if (((bitField0_ & 0x00000004) == 0x00000004)) { - metadata_ = java.util.Collections.unmodifiableList(metadata_); - bitField0_ = (bitField0_ & ~0x00000004); - } - result.metadata_ = metadata_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.getDefaultInstance()) return this; - if (other.hasKey()) { - setKey(other.getKey()); - } - if (other.hasValue()) { - setValue(other.getValue()); - } - if (!other.metadata_.isEmpty()) { - if (metadata_.isEmpty()) { - metadata_ = other.metadata_; - bitField0_ = (bitField0_ & ~0x00000004); - } else { - ensureMetadataIsMutable(); - metadata_.addAll(other.metadata_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasKey()) { - - return false; - } - if (!hasValue()) { - - return false; - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(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; - key_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - value_ = input.readBytes(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMetadata(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required string key = 1; - private java.lang.Object key_ = ""; - public boolean hasKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getKey() { - java.lang.Object ref = key_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - key_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setKey(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - key_ = value; - - return this; - } - public Builder clearKey() { - bitField0_ = (bitField0_ & ~0x00000001); - key_ = getDefaultInstance().getKey(); - - return this; - } - void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - key_ = value; - - } - - // required bytes value = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasValue() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue() { - return value_; - } - public Builder setValue(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - value_ = value; - - return this; - } - public Builder clearValue() { - bitField0_ = (bitField0_ & ~0x00000002); - value_ = getDefaultInstance().getValue(); - - return this; - } - - // repeated .pulsar.proto.KeyValue metadata = 3; - private java.util.List metadata_ = - java.util.Collections.emptyList(); - private void ensureMetadataIsMutable() { - if (!((bitField0_ & 0x00000004) == 0x00000004)) { - metadata_ = new java.util.ArrayList(metadata_); - bitField0_ |= 0x00000004; - } - } - - public java.util.List getMetadataList() { - return java.util.Collections.unmodifiableList(metadata_); - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.set(index, value); - - return this; - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.set(index, builderForValue.build()); - - return this; - } - public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(value); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(index, value); - - return this; - } - public Builder addMetadata( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(builderForValue.build()); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllMetadata( - java.lang.Iterable values) { - ensureMetadataIsMutable(); - super.addAll(values, metadata_); - - return this; - } - public Builder clearMetadata() { - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - - return this; - } - public Builder removeMetadata(int index) { - ensureMetadataIsMutable(); - metadata_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.EncryptionKeys) - } - - static { - defaultInstance = new EncryptionKeys(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.EncryptionKeys) - } - - public interface MessageMetadataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string producer_name = 1; - boolean hasProducerName(); - String getProducerName(); - - // required uint64 sequence_id = 2; - boolean hasSequenceId(); - long getSequenceId(); - - // required uint64 publish_time = 3; - boolean hasPublishTime(); - long getPublishTime(); - - // repeated .pulsar.proto.KeyValue properties = 4; - java.util.List - getPropertiesList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); - int getPropertiesCount(); - - // optional string replicated_from = 5; - boolean hasReplicatedFrom(); - String getReplicatedFrom(); - - // optional string partition_key = 6; - boolean hasPartitionKey(); - String getPartitionKey(); - - // repeated string replicate_to = 7; - java.util.List getReplicateToList(); - int getReplicateToCount(); - String getReplicateTo(int index); - - // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; - boolean hasCompression(); - org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression(); - - // optional uint32 uncompressed_size = 9 [default = 0]; - boolean hasUncompressedSize(); - int getUncompressedSize(); - - // optional int32 num_messages_in_batch = 11 [default = 1]; - boolean hasNumMessagesInBatch(); - int getNumMessagesInBatch(); - - // optional uint64 event_time = 12 [default = 0]; - boolean hasEventTime(); - long getEventTime(); - - // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; - java.util.List - getEncryptionKeysList(); - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index); - int getEncryptionKeysCount(); - - // optional string encryption_algo = 14; - boolean hasEncryptionAlgo(); - String getEncryptionAlgo(); - - // optional bytes encryption_param = 15; - boolean hasEncryptionParam(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam(); - - // optional bytes schema_version = 16; - boolean hasSchemaVersion(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); - - // optional bool partition_key_b64_encoded = 17 [default = false]; - boolean hasPartitionKeyB64Encoded(); - boolean getPartitionKeyB64Encoded(); - - // optional bytes ordering_key = 18; - boolean hasOrderingKey(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey(); - - // optional int64 deliver_at_time = 19; - boolean hasDeliverAtTime(); - long getDeliverAtTime(); - - // optional int32 marker_type = 20; - boolean hasMarkerType(); - int getMarkerType(); - - // optional uint64 txnid_least_bits = 22 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 23 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional uint64 highest_sequence_id = 24 [default = 0]; - boolean hasHighestSequenceId(); - long getHighestSequenceId(); - } - public static final class MessageMetadata extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements MessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use MessageMetadata.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private MessageMetadata(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected MessageMetadata newObject(Handle handle) { - return new MessageMetadata(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private MessageMetadata(boolean noInit) {} - - private static final MessageMetadata defaultInstance; - public static MessageMetadata getDefaultInstance() { - return defaultInstance; - } - - public MessageMetadata getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string producer_name = 1; - public static final int PRODUCER_NAME_FIELD_NUMBER = 1; - private java.lang.Object producerName_; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - 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)) { - producerName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { - java.lang.Object ref = producerName_; - 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); - producerName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required uint64 sequence_id = 2; - public static final int SEQUENCE_ID_FIELD_NUMBER = 2; - private long sequenceId_; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - - // required uint64 publish_time = 3; - public static final int PUBLISH_TIME_FIELD_NUMBER = 3; - private long publishTime_; - public boolean hasPublishTime() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getPublishTime() { - return publishTime_; - } - - // repeated .pulsar.proto.KeyValue properties = 4; - public static final int PROPERTIES_FIELD_NUMBER = 4; - private java.util.List properties_; - public java.util.List getPropertiesList() { - return properties_; - } - public java.util.List - getPropertiesOrBuilderList() { - return properties_; - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( - int index) { - return properties_.get(index); - } - - // optional string replicated_from = 5; - public static final int REPLICATED_FROM_FIELD_NUMBER = 5; - private java.lang.Object replicatedFrom_; - public boolean hasReplicatedFrom() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getReplicatedFrom() { - java.lang.Object ref = replicatedFrom_; - 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)) { - replicatedFrom_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getReplicatedFromBytes() { - java.lang.Object ref = replicatedFrom_; - 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); - replicatedFrom_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string partition_key = 6; - public static final int PARTITION_KEY_FIELD_NUMBER = 6; - private java.lang.Object partitionKey_; - public boolean hasPartitionKey() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getPartitionKey() { - java.lang.Object ref = partitionKey_; - 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)) { - partitionKey_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getPartitionKeyBytes() { - java.lang.Object ref = partitionKey_; - 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); - partitionKey_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // repeated string replicate_to = 7; - public static final int REPLICATE_TO_FIELD_NUMBER = 7; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList replicateTo_; - public java.util.List - getReplicateToList() { - return replicateTo_; - } - public int getReplicateToCount() { - return replicateTo_.size(); - } - public String getReplicateTo(int index) { - return replicateTo_.get(index); - } - - // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; - public static final int COMPRESSION_FIELD_NUMBER = 8; - private org.apache.pulsar.common.api.proto.PulsarApi.CompressionType compression_; - public boolean hasCompression() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression() { - return compression_; - } - - // optional uint32 uncompressed_size = 9 [default = 0]; - public static final int UNCOMPRESSED_SIZE_FIELD_NUMBER = 9; - private int uncompressedSize_; - public boolean hasUncompressedSize() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public int getUncompressedSize() { - return uncompressedSize_; - } - - // optional int32 num_messages_in_batch = 11 [default = 1]; - public static final int NUM_MESSAGES_IN_BATCH_FIELD_NUMBER = 11; - private int numMessagesInBatch_; - public boolean hasNumMessagesInBatch() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public int getNumMessagesInBatch() { - return numMessagesInBatch_; - } - - // optional uint64 event_time = 12 [default = 0]; - public static final int EVENT_TIME_FIELD_NUMBER = 12; - private long eventTime_; - public boolean hasEventTime() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public long getEventTime() { - return eventTime_; - } - - // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; - public static final int ENCRYPTION_KEYS_FIELD_NUMBER = 13; - private java.util.List encryptionKeys_; - public java.util.List getEncryptionKeysList() { - return encryptionKeys_; - } - public java.util.List - getEncryptionKeysOrBuilderList() { - return encryptionKeys_; - } - public int getEncryptionKeysCount() { - return encryptionKeys_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index) { - return encryptionKeys_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeysOrBuilder getEncryptionKeysOrBuilder( - int index) { - return encryptionKeys_.get(index); - } - - // optional string encryption_algo = 14; - public static final int ENCRYPTION_ALGO_FIELD_NUMBER = 14; - private java.lang.Object encryptionAlgo_; - public boolean hasEncryptionAlgo() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public String getEncryptionAlgo() { - java.lang.Object ref = encryptionAlgo_; - 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)) { - encryptionAlgo_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionAlgoBytes() { - java.lang.Object ref = encryptionAlgo_; - 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); - encryptionAlgo_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bytes encryption_param = 15; - public static final int ENCRYPTION_PARAM_FIELD_NUMBER = 15; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString encryptionParam_; - public boolean hasEncryptionParam() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam() { - return encryptionParam_; - } - - // optional bytes schema_version = 16; - public static final int SCHEMA_VERSION_FIELD_NUMBER = 16; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - - // optional bool partition_key_b64_encoded = 17 [default = false]; - public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 17; - private boolean partitionKeyB64Encoded_; - public boolean hasPartitionKeyB64Encoded() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public boolean getPartitionKeyB64Encoded() { - return partitionKeyB64Encoded_; - } - - // optional bytes ordering_key = 18; - public static final int ORDERING_KEY_FIELD_NUMBER = 18; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_; - public boolean hasOrderingKey() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { - return orderingKey_; - } - - // optional int64 deliver_at_time = 19; - public static final int DELIVER_AT_TIME_FIELD_NUMBER = 19; - private long deliverAtTime_; - public boolean hasDeliverAtTime() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public long getDeliverAtTime() { - return deliverAtTime_; - } - - // optional int32 marker_type = 20; - public static final int MARKER_TYPE_FIELD_NUMBER = 20; - private int markerType_; - public boolean hasMarkerType() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public int getMarkerType() { - return markerType_; - } - - // optional uint64 txnid_least_bits = 22 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 22; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00010000) == 0x00010000); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 23 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 23; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00020000) == 0x00020000); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional uint64 highest_sequence_id = 24 [default = 0]; - public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 24; - private long highestSequenceId_; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00040000) == 0x00040000); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - - private void initFields() { - producerName_ = ""; - sequenceId_ = 0L; - publishTime_ = 0L; - properties_ = java.util.Collections.emptyList(); - replicatedFrom_ = ""; - partitionKey_ = ""; - replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; - uncompressedSize_ = 0; - numMessagesInBatch_ = 1; - eventTime_ = 0L; - encryptionKeys_ = java.util.Collections.emptyList(); - encryptionAlgo_ = ""; - encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - partitionKeyB64Encoded_ = false; - orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - deliverAtTime_ = 0L; - markerType_ = 0; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - highestSequenceId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasProducerName()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSequenceId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasPublishTime()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - for (int i = 0; i < getEncryptionKeysCount(); i++) { - if (!getEncryptionKeys(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, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, publishTime_); - } - for (int i = 0; i < properties_.size(); i++) { - output.writeMessage(4, properties_.get(i)); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(5, getReplicatedFromBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(6, getPartitionKeyBytes()); - } - for (int i = 0; i < replicateTo_.size(); i++) { - output.writeBytes(7, replicateTo_.getByteString(i)); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeEnum(8, compression_.getNumber()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeUInt32(9, uncompressedSize_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeInt32(11, numMessagesInBatch_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeUInt64(12, eventTime_); - } - for (int i = 0; i < encryptionKeys_.size(); i++) { - output.writeMessage(13, encryptionKeys_.get(i)); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - output.writeBytes(14, getEncryptionAlgoBytes()); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - output.writeBytes(15, encryptionParam_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - output.writeBytes(16, schemaVersion_); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - output.writeBool(17, partitionKeyB64Encoded_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - output.writeBytes(18, orderingKey_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - output.writeInt64(19, deliverAtTime_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - output.writeInt32(20, markerType_); - } - if (((bitField0_ & 0x00010000) == 0x00010000)) { - output.writeUInt64(22, txnidLeastBits_); - } - if (((bitField0_ & 0x00020000) == 0x00020000)) { - output.writeUInt64(23, txnidMostBits_); - } - if (((bitField0_ & 0x00040000) == 0x00040000)) { - output.writeUInt64(24, highestSequenceId_); - } - } - - 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, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, publishTime_); - } - for (int i = 0; i < properties_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, properties_.get(i)); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getReplicatedFromBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(6, getPartitionKeyBytes()); - } - { - int dataSize = 0; - for (int i = 0; i < replicateTo_.size(); i++) { - dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSizeNoTag(replicateTo_.getByteString(i)); - } - size += dataSize; - size += 1 * getReplicateToList().size(); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(8, compression_.getNumber()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt32Size(9, uncompressedSize_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(11, numMessagesInBatch_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(12, eventTime_); - } - for (int i = 0; i < encryptionKeys_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(13, encryptionKeys_.get(i)); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(14, getEncryptionAlgoBytes()); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(15, encryptionParam_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(16, schemaVersion_); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(17, partitionKeyB64Encoded_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - 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 - .computeInt64Size(19, deliverAtTime_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(20, markerType_); - } - if (((bitField0_ & 0x00010000) == 0x00010000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(22, txnidLeastBits_); - } - if (((bitField0_ & 0x00020000) == 0x00020000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(23, txnidMostBits_); - } - if (((bitField0_ & 0x00040000) == 0x00040000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(24, highestSequenceId_); - } - 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.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(); - producerName_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - sequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - publishTime_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - replicatedFrom_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - partitionKey_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000040); - compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; - bitField0_ = (bitField0_ & ~0x00000080); - uncompressedSize_ = 0; - bitField0_ = (bitField0_ & ~0x00000100); - numMessagesInBatch_ = 1; - bitField0_ = (bitField0_ & ~0x00000200); - eventTime_ = 0L; - bitField0_ = (bitField0_ & ~0x00000400); - encryptionKeys_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000800); - encryptionAlgo_ = ""; - bitField0_ = (bitField0_ & ~0x00001000); - encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00002000); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00004000); - partitionKeyB64Encoded_ = false; - bitField0_ = (bitField0_ & ~0x00008000); - orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00010000); - deliverAtTime_ = 0L; - bitField0_ = (bitField0_ & ~0x00020000); - markerType_ = 0; - bitField0_ = (bitField0_ & ~0x00040000); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00080000); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00100000); - highestSequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00200000); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata build() { - org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.producerName_ = producerName_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.sequenceId_ = sequenceId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.publishTime_ = publishTime_; - if (((bitField0_ & 0x00000008) == 0x00000008)) { - properties_ = java.util.Collections.unmodifiableList(properties_); - bitField0_ = (bitField0_ & ~0x00000008); - } - result.properties_ = properties_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000008; - } - result.replicatedFrom_ = replicatedFrom_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000010; - } - result.partitionKey_ = partitionKey_; - if (((bitField0_ & 0x00000040) == 0x00000040)) { - replicateTo_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( - replicateTo_); - bitField0_ = (bitField0_ & ~0x00000040); - } - result.replicateTo_ = replicateTo_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000020; - } - result.compression_ = compression_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000040; - } - result.uncompressedSize_ = uncompressedSize_; - if (((from_bitField0_ & 0x00000200) == 0x00000200)) { - to_bitField0_ |= 0x00000080; - } - result.numMessagesInBatch_ = numMessagesInBatch_; - if (((from_bitField0_ & 0x00000400) == 0x00000400)) { - to_bitField0_ |= 0x00000100; - } - result.eventTime_ = eventTime_; - if (((bitField0_ & 0x00000800) == 0x00000800)) { - encryptionKeys_ = java.util.Collections.unmodifiableList(encryptionKeys_); - bitField0_ = (bitField0_ & ~0x00000800); - } - result.encryptionKeys_ = encryptionKeys_; - if (((from_bitField0_ & 0x00001000) == 0x00001000)) { - to_bitField0_ |= 0x00000200; - } - result.encryptionAlgo_ = encryptionAlgo_; - if (((from_bitField0_ & 0x00002000) == 0x00002000)) { - to_bitField0_ |= 0x00000400; - } - result.encryptionParam_ = encryptionParam_; - if (((from_bitField0_ & 0x00004000) == 0x00004000)) { - to_bitField0_ |= 0x00000800; - } - result.schemaVersion_ = schemaVersion_; - if (((from_bitField0_ & 0x00008000) == 0x00008000)) { - to_bitField0_ |= 0x00001000; - } - result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_; - if (((from_bitField0_ & 0x00010000) == 0x00010000)) { - to_bitField0_ |= 0x00002000; - } - result.orderingKey_ = orderingKey_; - if (((from_bitField0_ & 0x00020000) == 0x00020000)) { - to_bitField0_ |= 0x00004000; - } - result.deliverAtTime_ = deliverAtTime_; - if (((from_bitField0_ & 0x00040000) == 0x00040000)) { - to_bitField0_ |= 0x00008000; - } - result.markerType_ = markerType_; - if (((from_bitField0_ & 0x00080000) == 0x00080000)) { - to_bitField0_ |= 0x00010000; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00100000) == 0x00100000)) { - to_bitField0_ |= 0x00020000; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00200000) == 0x00200000)) { - to_bitField0_ |= 0x00040000; - } - result.highestSequenceId_ = highestSequenceId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.getDefaultInstance()) return this; - if (other.hasProducerName()) { - setProducerName(other.getProducerName()); - } - if (other.hasSequenceId()) { - setSequenceId(other.getSequenceId()); - } - if (other.hasPublishTime()) { - setPublishTime(other.getPublishTime()); - } - if (!other.properties_.isEmpty()) { - if (properties_.isEmpty()) { - properties_ = other.properties_; - bitField0_ = (bitField0_ & ~0x00000008); - } else { - ensurePropertiesIsMutable(); - properties_.addAll(other.properties_); - } - - } - if (other.hasReplicatedFrom()) { - setReplicatedFrom(other.getReplicatedFrom()); - } - if (other.hasPartitionKey()) { - setPartitionKey(other.getPartitionKey()); - } - if (!other.replicateTo_.isEmpty()) { - if (replicateTo_.isEmpty()) { - replicateTo_ = other.replicateTo_; - bitField0_ = (bitField0_ & ~0x00000040); - } else { - ensureReplicateToIsMutable(); - replicateTo_.addAll(other.replicateTo_); - } - - } - if (other.hasCompression()) { - setCompression(other.getCompression()); - } - if (other.hasUncompressedSize()) { - setUncompressedSize(other.getUncompressedSize()); - } - if (other.hasNumMessagesInBatch()) { - setNumMessagesInBatch(other.getNumMessagesInBatch()); - } - if (other.hasEventTime()) { - setEventTime(other.getEventTime()); - } - if (!other.encryptionKeys_.isEmpty()) { - if (encryptionKeys_.isEmpty()) { - encryptionKeys_ = other.encryptionKeys_; - bitField0_ = (bitField0_ & ~0x00000800); - } else { - ensureEncryptionKeysIsMutable(); - encryptionKeys_.addAll(other.encryptionKeys_); - } - - } - if (other.hasEncryptionAlgo()) { - setEncryptionAlgo(other.getEncryptionAlgo()); - } - if (other.hasEncryptionParam()) { - setEncryptionParam(other.getEncryptionParam()); - } - if (other.hasSchemaVersion()) { - setSchemaVersion(other.getSchemaVersion()); - } - if (other.hasPartitionKeyB64Encoded()) { - setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded()); - } - if (other.hasOrderingKey()) { - setOrderingKey(other.getOrderingKey()); - } - if (other.hasDeliverAtTime()) { - setDeliverAtTime(other.getDeliverAtTime()); - } - if (other.hasMarkerType()) { - setMarkerType(other.getMarkerType()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasHighestSequenceId()) { - setHighestSequenceId(other.getHighestSequenceId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasProducerName()) { - - return false; - } - if (!hasSequenceId()) { - - return false; - } - if (!hasPublishTime()) { - - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(i).isInitialized()) { - - return false; - } - } - for (int i = 0; i < getEncryptionKeysCount(); i++) { - if (!getEncryptionKeys(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; - producerName_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - sequenceId_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - publishTime_ = input.readUInt64(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addProperties(subBuilder.buildPartial()); - break; - } - case 42: { - bitField0_ |= 0x00000010; - replicatedFrom_ = input.readBytes(); - break; - } - case 50: { - bitField0_ |= 0x00000020; - partitionKey_ = input.readBytes(); - break; - } - case 58: { - ensureReplicateToIsMutable(); - replicateTo_.add(input.readBytes()); - break; - } - case 64: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CompressionType value = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000080; - compression_ = value; - } - break; - } - case 72: { - bitField0_ |= 0x00000100; - uncompressedSize_ = input.readUInt32(); - break; - } - case 88: { - bitField0_ |= 0x00000200; - numMessagesInBatch_ = input.readInt32(); - break; - } - case 96: { - bitField0_ |= 0x00000400; - eventTime_ = input.readUInt64(); - break; - } - case 106: { - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addEncryptionKeys(subBuilder.buildPartial()); - break; - } - case 114: { - bitField0_ |= 0x00001000; - encryptionAlgo_ = input.readBytes(); - break; - } - case 122: { - bitField0_ |= 0x00002000; - encryptionParam_ = input.readBytes(); - break; - } - case 130: { - bitField0_ |= 0x00004000; - schemaVersion_ = input.readBytes(); - break; - } - case 136: { - bitField0_ |= 0x00008000; - partitionKeyB64Encoded_ = input.readBool(); - break; - } - case 146: { - bitField0_ |= 0x00010000; - orderingKey_ = input.readBytes(); - break; - } - case 152: { - bitField0_ |= 0x00020000; - deliverAtTime_ = input.readInt64(); - break; - } - case 160: { - bitField0_ |= 0x00040000; - markerType_ = input.readInt32(); - break; - } - case 176: { - bitField0_ |= 0x00080000; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 184: { - bitField0_ |= 0x00100000; - txnidMostBits_ = input.readUInt64(); - break; - } - case 192: { - bitField0_ |= 0x00200000; - highestSequenceId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required string producer_name = 1; - private java.lang.Object producerName_ = ""; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - producerName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setProducerName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - producerName_ = value; - - return this; - } - public Builder clearProducerName() { - bitField0_ = (bitField0_ & ~0x00000001); - producerName_ = getDefaultInstance().getProducerName(); - - return this; - } - void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - producerName_ = value; - - } - - // required uint64 sequence_id = 2; - private long sequenceId_ ; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - public Builder setSequenceId(long value) { - bitField0_ |= 0x00000002; - sequenceId_ = value; - - return this; - } - public Builder clearSequenceId() { - bitField0_ = (bitField0_ & ~0x00000002); - sequenceId_ = 0L; - - return this; - } - - // required uint64 publish_time = 3; - private long publishTime_ ; - public boolean hasPublishTime() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getPublishTime() { - return publishTime_; - } - public Builder setPublishTime(long value) { - bitField0_ |= 0x00000004; - publishTime_ = value; - - return this; - } - public Builder clearPublishTime() { - bitField0_ = (bitField0_ & ~0x00000004); - publishTime_ = 0L; - - return this; - } - - // repeated .pulsar.proto.KeyValue properties = 4; - private java.util.List properties_ = - java.util.Collections.emptyList(); - private void ensurePropertiesIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { - properties_ = new java.util.ArrayList(properties_); - bitField0_ |= 0x00000008; - } - } - - public java.util.List getPropertiesList() { - return java.util.Collections.unmodifiableList(properties_); - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.set(index, value); - - return this; - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.set(index, builderForValue.build()); - - return this; - } - public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(value); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(index, value); - - return this; - } - public Builder addProperties( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(builderForValue.build()); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllProperties( - java.lang.Iterable values) { - ensurePropertiesIsMutable(); - super.addAll(values, properties_); - - return this; - } - public Builder clearProperties() { - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - - return this; - } - public Builder removeProperties(int index) { - ensurePropertiesIsMutable(); - properties_.remove(index); - - return this; - } - - // optional string replicated_from = 5; - private java.lang.Object replicatedFrom_ = ""; - public boolean hasReplicatedFrom() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getReplicatedFrom() { - java.lang.Object ref = replicatedFrom_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - replicatedFrom_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setReplicatedFrom(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - replicatedFrom_ = value; - - return this; - } - public Builder clearReplicatedFrom() { - bitField0_ = (bitField0_ & ~0x00000010); - replicatedFrom_ = getDefaultInstance().getReplicatedFrom(); - - return this; - } - void setReplicatedFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - replicatedFrom_ = value; - - } - - // optional string partition_key = 6; - private java.lang.Object partitionKey_ = ""; - public boolean hasPartitionKey() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getPartitionKey() { - java.lang.Object ref = partitionKey_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - partitionKey_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setPartitionKey(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - partitionKey_ = value; - - return this; - } - public Builder clearPartitionKey() { - bitField0_ = (bitField0_ & ~0x00000020); - partitionKey_ = getDefaultInstance().getPartitionKey(); - - return this; - } - void setPartitionKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000020; - partitionKey_ = value; - - } - - // repeated string replicate_to = 7; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - private void ensureReplicateToIsMutable() { - if (!((bitField0_ & 0x00000040) == 0x00000040)) { - replicateTo_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(replicateTo_); - bitField0_ |= 0x00000040; - } - } - public java.util.List - getReplicateToList() { - return java.util.Collections.unmodifiableList(replicateTo_); - } - public int getReplicateToCount() { - return replicateTo_.size(); - } - public String getReplicateTo(int index) { - return replicateTo_.get(index); - } - public Builder setReplicateTo( - int index, String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureReplicateToIsMutable(); - replicateTo_.set(index, value); - - return this; - } - public Builder addReplicateTo(String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureReplicateToIsMutable(); - replicateTo_.add(value); - - return this; - } - public Builder addAllReplicateTo( - java.lang.Iterable values) { - ensureReplicateToIsMutable(); - super.addAll(values, replicateTo_); - - return this; - } - public Builder clearReplicateTo() { - replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000040); - - return this; - } - void addReplicateTo(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - ensureReplicateToIsMutable(); - replicateTo_.add(value); - - } - - // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; - private org.apache.pulsar.common.api.proto.PulsarApi.CompressionType compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; - public boolean hasCompression() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression() { - return compression_; - } - public Builder setCompression(org.apache.pulsar.common.api.proto.PulsarApi.CompressionType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000080; - compression_ = value; - - return this; - } - public Builder clearCompression() { - bitField0_ = (bitField0_ & ~0x00000080); - compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; - - return this; - } - - // optional uint32 uncompressed_size = 9 [default = 0]; - private int uncompressedSize_ ; - public boolean hasUncompressedSize() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public int getUncompressedSize() { - return uncompressedSize_; - } - public Builder setUncompressedSize(int value) { - bitField0_ |= 0x00000100; - uncompressedSize_ = value; - - return this; - } - public Builder clearUncompressedSize() { - bitField0_ = (bitField0_ & ~0x00000100); - uncompressedSize_ = 0; - - return this; - } - - // optional int32 num_messages_in_batch = 11 [default = 1]; - private int numMessagesInBatch_ = 1; - public boolean hasNumMessagesInBatch() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public int getNumMessagesInBatch() { - return numMessagesInBatch_; - } - public Builder setNumMessagesInBatch(int value) { - bitField0_ |= 0x00000200; - numMessagesInBatch_ = value; - - return this; - } - public Builder clearNumMessagesInBatch() { - bitField0_ = (bitField0_ & ~0x00000200); - numMessagesInBatch_ = 1; - - return this; - } - - // optional uint64 event_time = 12 [default = 0]; - private long eventTime_ ; - public boolean hasEventTime() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public long getEventTime() { - return eventTime_; - } - public Builder setEventTime(long value) { - bitField0_ |= 0x00000400; - eventTime_ = value; - - return this; - } - public Builder clearEventTime() { - bitField0_ = (bitField0_ & ~0x00000400); - eventTime_ = 0L; - - return this; - } - - // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; - private java.util.List encryptionKeys_ = - java.util.Collections.emptyList(); - private void ensureEncryptionKeysIsMutable() { - if (!((bitField0_ & 0x00000800) == 0x00000800)) { - encryptionKeys_ = new java.util.ArrayList(encryptionKeys_); - bitField0_ |= 0x00000800; - } - } - - public java.util.List getEncryptionKeysList() { - return java.util.Collections.unmodifiableList(encryptionKeys_); - } - public int getEncryptionKeysCount() { - return encryptionKeys_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index) { - return encryptionKeys_.get(index); - } - public Builder setEncryptionKeys( - int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { - if (value == null) { - throw new NullPointerException(); - } - ensureEncryptionKeysIsMutable(); - encryptionKeys_.set(index, value); - - return this; - } - public Builder setEncryptionKeys( - int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { - ensureEncryptionKeysIsMutable(); - encryptionKeys_.set(index, builderForValue.build()); - - return this; - } - public Builder addEncryptionKeys(org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { - if (value == null) { - throw new NullPointerException(); - } - ensureEncryptionKeysIsMutable(); - encryptionKeys_.add(value); - - return this; - } - public Builder addEncryptionKeys( - int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { - if (value == null) { - throw new NullPointerException(); - } - ensureEncryptionKeysIsMutable(); - encryptionKeys_.add(index, value); - - return this; - } - public Builder addEncryptionKeys( - org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { - ensureEncryptionKeysIsMutable(); - encryptionKeys_.add(builderForValue.build()); - - return this; - } - public Builder addEncryptionKeys( - int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { - ensureEncryptionKeysIsMutable(); - encryptionKeys_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllEncryptionKeys( - java.lang.Iterable values) { - ensureEncryptionKeysIsMutable(); - super.addAll(values, encryptionKeys_); - - return this; - } - public Builder clearEncryptionKeys() { - encryptionKeys_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000800); - - return this; - } - public Builder removeEncryptionKeys(int index) { - ensureEncryptionKeysIsMutable(); - encryptionKeys_.remove(index); - - return this; - } - - // optional string encryption_algo = 14; - private java.lang.Object encryptionAlgo_ = ""; - public boolean hasEncryptionAlgo() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public String getEncryptionAlgo() { - java.lang.Object ref = encryptionAlgo_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - encryptionAlgo_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setEncryptionAlgo(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00001000; - encryptionAlgo_ = value; - - return this; - } - public Builder clearEncryptionAlgo() { - bitField0_ = (bitField0_ & ~0x00001000); - encryptionAlgo_ = getDefaultInstance().getEncryptionAlgo(); - - return this; - } - void setEncryptionAlgo(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00001000; - encryptionAlgo_ = value; - - } - - // optional bytes encryption_param = 15; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasEncryptionParam() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam() { - return encryptionParam_; - } - public Builder setEncryptionParam(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00002000; - encryptionParam_ = value; - - return this; - } - public Builder clearEncryptionParam() { - bitField0_ = (bitField0_ & ~0x00002000); - encryptionParam_ = getDefaultInstance().getEncryptionParam(); - - return this; - } - - // optional bytes schema_version = 16; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00004000; - schemaVersion_ = value; - - return this; - } - public Builder clearSchemaVersion() { - bitField0_ = (bitField0_ & ~0x00004000); - schemaVersion_ = getDefaultInstance().getSchemaVersion(); - - return this; - } - - // optional bool partition_key_b64_encoded = 17 [default = false]; - private boolean partitionKeyB64Encoded_ ; - public boolean hasPartitionKeyB64Encoded() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public boolean getPartitionKeyB64Encoded() { - return partitionKeyB64Encoded_; - } - public Builder setPartitionKeyB64Encoded(boolean value) { - bitField0_ |= 0x00008000; - partitionKeyB64Encoded_ = value; - - return this; - } - public Builder clearPartitionKeyB64Encoded() { - bitField0_ = (bitField0_ & ~0x00008000); - partitionKeyB64Encoded_ = false; - - return this; - } - - // optional bytes ordering_key = 18; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasOrderingKey() { - return ((bitField0_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { - return orderingKey_; - } - public Builder setOrderingKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00010000; - orderingKey_ = value; - - return this; - } - public Builder clearOrderingKey() { - bitField0_ = (bitField0_ & ~0x00010000); - orderingKey_ = getDefaultInstance().getOrderingKey(); - - return this; - } - - // optional int64 deliver_at_time = 19; - private long deliverAtTime_ ; - public boolean hasDeliverAtTime() { - return ((bitField0_ & 0x00020000) == 0x00020000); - } - public long getDeliverAtTime() { - return deliverAtTime_; - } - public Builder setDeliverAtTime(long value) { - bitField0_ |= 0x00020000; - deliverAtTime_ = value; - - return this; - } - public Builder clearDeliverAtTime() { - bitField0_ = (bitField0_ & ~0x00020000); - deliverAtTime_ = 0L; - - return this; - } - - // optional int32 marker_type = 20; - private int markerType_ ; - public boolean hasMarkerType() { - return ((bitField0_ & 0x00040000) == 0x00040000); - } - public int getMarkerType() { - return markerType_; - } - public Builder setMarkerType(int value) { - bitField0_ |= 0x00040000; - markerType_ = value; - - return this; - } - public Builder clearMarkerType() { - bitField0_ = (bitField0_ & ~0x00040000); - markerType_ = 0; - - return this; - } - - // optional uint64 txnid_least_bits = 22 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00080000) == 0x00080000); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00080000; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00080000); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 23 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00100000) == 0x00100000); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00100000; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00100000); - txnidMostBits_ = 0L; - - return this; - } - - // optional uint64 highest_sequence_id = 24 [default = 0]; - private long highestSequenceId_ ; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00200000) == 0x00200000); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - public Builder setHighestSequenceId(long value) { - bitField0_ |= 0x00200000; - highestSequenceId_ = value; - - return this; - } - public Builder clearHighestSequenceId() { - bitField0_ = (bitField0_ & ~0x00200000); - highestSequenceId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.MessageMetadata) - } - - static { - defaultInstance = new MessageMetadata(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.MessageMetadata) - } - - public interface SingleMessageMetadataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // repeated .pulsar.proto.KeyValue properties = 1; - java.util.List - getPropertiesList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); - int getPropertiesCount(); - - // optional string partition_key = 2; - boolean hasPartitionKey(); - String getPartitionKey(); - - // required int32 payload_size = 3; - boolean hasPayloadSize(); - int getPayloadSize(); - - // optional bool compacted_out = 4 [default = false]; - boolean hasCompactedOut(); - boolean getCompactedOut(); - - // optional uint64 event_time = 5 [default = 0]; - boolean hasEventTime(); - long getEventTime(); - - // optional bool partition_key_b64_encoded = 6 [default = false]; - boolean hasPartitionKeyB64Encoded(); - boolean getPartitionKeyB64Encoded(); - - // optional bytes ordering_key = 7; - boolean hasOrderingKey(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey(); - - // optional uint64 sequence_id = 8; - boolean hasSequenceId(); - long getSequenceId(); - } - public static final class SingleMessageMetadata extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements SingleMessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use SingleMessageMetadata.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private SingleMessageMetadata(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected SingleMessageMetadata newObject(Handle handle) { - return new SingleMessageMetadata(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private SingleMessageMetadata(boolean noInit) {} - - private static final SingleMessageMetadata defaultInstance; - public static SingleMessageMetadata getDefaultInstance() { - return defaultInstance; - } - - public SingleMessageMetadata getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // repeated .pulsar.proto.KeyValue properties = 1; - public static final int PROPERTIES_FIELD_NUMBER = 1; - private java.util.List properties_; - public java.util.List getPropertiesList() { - return properties_; - } - public java.util.List - getPropertiesOrBuilderList() { - return properties_; - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( - int index) { - return properties_.get(index); - } - - // optional string partition_key = 2; - public static final int PARTITION_KEY_FIELD_NUMBER = 2; - private java.lang.Object partitionKey_; - public boolean hasPartitionKey() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getPartitionKey() { - java.lang.Object ref = partitionKey_; - 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)) { - partitionKey_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getPartitionKeyBytes() { - java.lang.Object ref = partitionKey_; - 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); - partitionKey_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required int32 payload_size = 3; - public static final int PAYLOAD_SIZE_FIELD_NUMBER = 3; - private int payloadSize_; - public boolean hasPayloadSize() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getPayloadSize() { - return payloadSize_; - } - - // optional bool compacted_out = 4 [default = false]; - public static final int COMPACTED_OUT_FIELD_NUMBER = 4; - private boolean compactedOut_; - public boolean hasCompactedOut() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public boolean getCompactedOut() { - return compactedOut_; - } - - // optional uint64 event_time = 5 [default = 0]; - public static final int EVENT_TIME_FIELD_NUMBER = 5; - private long eventTime_; - public boolean hasEventTime() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getEventTime() { - return eventTime_; - } - - // optional bool partition_key_b64_encoded = 6 [default = false]; - public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 6; - private boolean partitionKeyB64Encoded_; - public boolean hasPartitionKeyB64Encoded() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getPartitionKeyB64Encoded() { - return partitionKeyB64Encoded_; - } - - // optional bytes ordering_key = 7; - public static final int ORDERING_KEY_FIELD_NUMBER = 7; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_; - public boolean hasOrderingKey() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { - return orderingKey_; - } - - // optional uint64 sequence_id = 8; - public static final int SEQUENCE_ID_FIELD_NUMBER = 8; - private long sequenceId_; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public long getSequenceId() { - return sequenceId_; - } - - private void initFields() { - properties_ = java.util.Collections.emptyList(); - partitionKey_ = ""; - payloadSize_ = 0; - compactedOut_ = false; - eventTime_ = 0L; - partitionKeyB64Encoded_ = false; - orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - sequenceId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasPayloadSize()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(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(); - for (int i = 0; i < properties_.size(); i++) { - output.writeMessage(1, properties_.get(i)); - } - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeBytes(2, getPartitionKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeInt32(3, payloadSize_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBool(4, compactedOut_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(5, eventTime_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBool(6, partitionKeyB64Encoded_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(7, orderingKey_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeUInt64(8, sequenceId_); - } - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - for (int i = 0; i < properties_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(1, properties_.get(i)); - } - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getPartitionKeyBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, payloadSize_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(4, compactedOut_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(5, eventTime_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(6, partitionKeyB64Encoded_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(7, orderingKey_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(8, sequenceId_); - } - 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.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(); - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - partitionKey_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - payloadSize_ = 0; - bitField0_ = (bitField0_ & ~0x00000004); - compactedOut_ = false; - bitField0_ = (bitField0_ & ~0x00000008); - eventTime_ = 0L; - bitField0_ = (bitField0_ & ~0x00000010); - partitionKeyB64Encoded_ = false; - bitField0_ = (bitField0_ & ~0x00000020); - orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000040); - sequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000080); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata build() { - org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - properties_ = java.util.Collections.unmodifiableList(properties_); - bitField0_ = (bitField0_ & ~0x00000001); - } - result.properties_ = properties_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000001; - } - result.partitionKey_ = partitionKey_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000002; - } - result.payloadSize_ = payloadSize_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000004; - } - result.compactedOut_ = compactedOut_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000008; - } - result.eventTime_ = eventTime_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000010; - } - result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000020; - } - result.orderingKey_ = orderingKey_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000040; - } - result.sequenceId_ = sequenceId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.getDefaultInstance()) return this; - if (!other.properties_.isEmpty()) { - if (properties_.isEmpty()) { - properties_ = other.properties_; - bitField0_ = (bitField0_ & ~0x00000001); - } else { - ensurePropertiesIsMutable(); - properties_.addAll(other.properties_); - } - - } - if (other.hasPartitionKey()) { - setPartitionKey(other.getPartitionKey()); - } - if (other.hasPayloadSize()) { - setPayloadSize(other.getPayloadSize()); - } - if (other.hasCompactedOut()) { - setCompactedOut(other.getCompactedOut()); - } - if (other.hasEventTime()) { - setEventTime(other.getEventTime()); - } - if (other.hasPartitionKeyB64Encoded()) { - setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded()); - } - if (other.hasOrderingKey()) { - setOrderingKey(other.getOrderingKey()); - } - if (other.hasSequenceId()) { - setSequenceId(other.getSequenceId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasPayloadSize()) { - - return false; - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(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: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addProperties(subBuilder.buildPartial()); - break; - } - case 18: { - bitField0_ |= 0x00000002; - partitionKey_ = input.readBytes(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - payloadSize_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - compactedOut_ = input.readBool(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - eventTime_ = input.readUInt64(); - break; - } - case 48: { - bitField0_ |= 0x00000020; - partitionKeyB64Encoded_ = input.readBool(); - break; - } - case 58: { - bitField0_ |= 0x00000040; - orderingKey_ = input.readBytes(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - sequenceId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // repeated .pulsar.proto.KeyValue properties = 1; - private java.util.List properties_ = - java.util.Collections.emptyList(); - private void ensurePropertiesIsMutable() { - if (!((bitField0_ & 0x00000001) == 0x00000001)) { - properties_ = new java.util.ArrayList(properties_); - bitField0_ |= 0x00000001; - } - } - - public java.util.List getPropertiesList() { - return java.util.Collections.unmodifiableList(properties_); - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { - return properties_.get(index); - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.set(index, value); - - return this; - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.set(index, builderForValue.build()); - - return this; - } - public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(value); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(index, value); - - return this; - } - public Builder addProperties( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(builderForValue.build()); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllProperties( - java.lang.Iterable values) { - ensurePropertiesIsMutable(); - super.addAll(values, properties_); - - return this; - } - public Builder clearProperties() { - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000001); - - return this; - } - public Builder removeProperties(int index) { - ensurePropertiesIsMutable(); - properties_.remove(index); - - return this; - } - - // optional string partition_key = 2; - private java.lang.Object partitionKey_ = ""; - public boolean hasPartitionKey() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getPartitionKey() { - java.lang.Object ref = partitionKey_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - partitionKey_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setPartitionKey(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - partitionKey_ = value; - - return this; - } - public Builder clearPartitionKey() { - bitField0_ = (bitField0_ & ~0x00000002); - partitionKey_ = getDefaultInstance().getPartitionKey(); - - return this; - } - void setPartitionKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - partitionKey_ = value; - - } - - // required int32 payload_size = 3; - private int payloadSize_ ; - public boolean hasPayloadSize() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getPayloadSize() { - return payloadSize_; - } - public Builder setPayloadSize(int value) { - bitField0_ |= 0x00000004; - payloadSize_ = value; - - return this; - } - public Builder clearPayloadSize() { - bitField0_ = (bitField0_ & ~0x00000004); - payloadSize_ = 0; - - return this; - } - - // optional bool compacted_out = 4 [default = false]; - private boolean compactedOut_ ; - public boolean hasCompactedOut() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public boolean getCompactedOut() { - return compactedOut_; - } - public Builder setCompactedOut(boolean value) { - bitField0_ |= 0x00000008; - compactedOut_ = value; - - return this; - } - public Builder clearCompactedOut() { - bitField0_ = (bitField0_ & ~0x00000008); - compactedOut_ = false; - - return this; - } - - // optional uint64 event_time = 5 [default = 0]; - private long eventTime_ ; - public boolean hasEventTime() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getEventTime() { - return eventTime_; - } - public Builder setEventTime(long value) { - bitField0_ |= 0x00000010; - eventTime_ = value; - - return this; - } - public Builder clearEventTime() { - bitField0_ = (bitField0_ & ~0x00000010); - eventTime_ = 0L; - - return this; - } - - // optional bool partition_key_b64_encoded = 6 [default = false]; - private boolean partitionKeyB64Encoded_ ; - public boolean hasPartitionKeyB64Encoded() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public boolean getPartitionKeyB64Encoded() { - return partitionKeyB64Encoded_; - } - public Builder setPartitionKeyB64Encoded(boolean value) { - bitField0_ |= 0x00000020; - partitionKeyB64Encoded_ = value; - - return this; - } - public Builder clearPartitionKeyB64Encoded() { - bitField0_ = (bitField0_ & ~0x00000020); - partitionKeyB64Encoded_ = false; - - return this; - } - - // optional bytes ordering_key = 7; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasOrderingKey() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { - return orderingKey_; - } - public Builder setOrderingKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000040; - orderingKey_ = value; - - return this; - } - public Builder clearOrderingKey() { - bitField0_ = (bitField0_ & ~0x00000040); - orderingKey_ = getDefaultInstance().getOrderingKey(); - - return this; - } - - // optional uint64 sequence_id = 8; - private long sequenceId_ ; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public long getSequenceId() { - return sequenceId_; - } - public Builder setSequenceId(long value) { - bitField0_ |= 0x00000080; - sequenceId_ = value; - - return this; - } - public Builder clearSequenceId() { - bitField0_ = (bitField0_ & ~0x00000080); - sequenceId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.SingleMessageMetadata) - } - - static { - defaultInstance = new SingleMessageMetadata(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.SingleMessageMetadata) - } - - public interface CommandConnectOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string client_version = 1; - boolean hasClientVersion(); - String getClientVersion(); - - // optional .pulsar.proto.AuthMethod auth_method = 2; - boolean hasAuthMethod(); - org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod(); - - // optional string auth_method_name = 5; - boolean hasAuthMethodName(); - String getAuthMethodName(); - - // optional bytes auth_data = 3; - boolean hasAuthData(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData(); - - // optional int32 protocol_version = 4 [default = 0]; - boolean hasProtocolVersion(); - int getProtocolVersion(); - - // optional string proxy_to_broker_url = 6; - boolean hasProxyToBrokerUrl(); - String getProxyToBrokerUrl(); - - // optional string original_principal = 7; - boolean hasOriginalPrincipal(); - String getOriginalPrincipal(); - - // optional string original_auth_data = 8; - boolean hasOriginalAuthData(); - String getOriginalAuthData(); - - // optional string original_auth_method = 9; - boolean hasOriginalAuthMethod(); - String getOriginalAuthMethod(); - - // optional .pulsar.proto.FeatureFlags feature_flags = 10; - boolean hasFeatureFlags(); - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags(); - } - public static final class CommandConnect extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandConnectOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandConnect.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandConnect(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandConnect newObject(Handle handle) { - return new CommandConnect(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandConnect(boolean noInit) {} - - private static final CommandConnect defaultInstance; - public static CommandConnect getDefaultInstance() { - return defaultInstance; - } - - public CommandConnect getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string client_version = 1; - public static final int CLIENT_VERSION_FIELD_NUMBER = 1; - private java.lang.Object clientVersion_; - public boolean hasClientVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getClientVersion() { - java.lang.Object ref = clientVersion_; - 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)) { - clientVersion_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getClientVersionBytes() { - java.lang.Object ref = clientVersion_; - 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); - clientVersion_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.AuthMethod auth_method = 2; - public static final int AUTH_METHOD_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod authMethod_; - public boolean hasAuthMethod() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod() { - return authMethod_; - } - - // optional string auth_method_name = 5; - public static final int AUTH_METHOD_NAME_FIELD_NUMBER = 5; - private java.lang.Object authMethodName_; - public boolean hasAuthMethodName() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getAuthMethodName() { - java.lang.Object ref = authMethodName_; - 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)) { - authMethodName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthMethodNameBytes() { - java.lang.Object ref = authMethodName_; - 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); - authMethodName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bytes auth_data = 3; - public static final int AUTH_DATA_FIELD_NUMBER = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_; - public boolean hasAuthData() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { - return authData_; - } - - // optional int32 protocol_version = 4 [default = 0]; - public static final int PROTOCOL_VERSION_FIELD_NUMBER = 4; - private int protocolVersion_; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public int getProtocolVersion() { - return protocolVersion_; - } - - // optional string proxy_to_broker_url = 6; - public static final int PROXY_TO_BROKER_URL_FIELD_NUMBER = 6; - private java.lang.Object proxyToBrokerUrl_; - public boolean hasProxyToBrokerUrl() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getProxyToBrokerUrl() { - java.lang.Object ref = proxyToBrokerUrl_; - 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)) { - proxyToBrokerUrl_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProxyToBrokerUrlBytes() { - java.lang.Object ref = proxyToBrokerUrl_; - 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); - proxyToBrokerUrl_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_principal = 7; - public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 7; - private java.lang.Object originalPrincipal_; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - 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)) { - originalPrincipal_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { - java.lang.Object ref = originalPrincipal_; - 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); - originalPrincipal_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_data = 8; - public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 8; - private java.lang.Object originalAuthData_; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - 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)) { - originalAuthData_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { - java.lang.Object ref = originalAuthData_; - 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); - originalAuthData_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_method = 9; - public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 9; - private java.lang.Object originalAuthMethod_; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - 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)) { - originalAuthMethod_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { - java.lang.Object ref = originalAuthMethod_; - 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); - originalAuthMethod_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.FeatureFlags feature_flags = 10; - public static final int FEATURE_FLAGS_FIELD_NUMBER = 10; - private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags featureFlags_; - public boolean hasFeatureFlags() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags() { - return featureFlags_; - } - - private void initFields() { - clientVersion_ = ""; - authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; - authMethodName_ = ""; - authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - protocolVersion_ = 0; - proxyToBrokerUrl_ = ""; - originalPrincipal_ = ""; - originalAuthData_ = ""; - originalAuthMethod_ = ""; - featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasClientVersion()) { - 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, getClientVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, authMethod_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(3, authData_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeInt32(4, protocolVersion_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(5, getAuthMethodNameBytes()); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(6, getProxyToBrokerUrlBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeBytes(7, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeBytes(8, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeBytes(9, getOriginalAuthMethodBytes()); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - output.writeMessage(10, featureFlags_); - } - } - - 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, getClientVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, authMethod_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, authData_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(4, protocolVersion_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getAuthMethodNameBytes()); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(6, getProxyToBrokerUrlBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(7, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(8, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(9, getOriginalAuthMethodBytes()); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(10, featureFlags_); - } - 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConnectOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.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(); - clientVersion_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; - bitField0_ = (bitField0_ & ~0x00000002); - authMethodName_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000008); - protocolVersion_ = 0; - bitField0_ = (bitField0_ & ~0x00000010); - proxyToBrokerUrl_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - originalPrincipal_ = ""; - bitField0_ = (bitField0_ & ~0x00000040); - originalAuthData_ = ""; - bitField0_ = (bitField0_ & ~0x00000080); - originalAuthMethod_ = ""; - bitField0_ = (bitField0_ & ~0x00000100); - featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000200); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.clientVersion_ = clientVersion_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.authMethod_ = authMethod_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.authMethodName_ = authMethodName_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.authData_ = authData_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.protocolVersion_ = protocolVersion_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.proxyToBrokerUrl_ = proxyToBrokerUrl_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.originalPrincipal_ = originalPrincipal_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.originalAuthData_ = originalAuthData_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.originalAuthMethod_ = originalAuthMethod_; - if (((from_bitField0_ & 0x00000200) == 0x00000200)) { - to_bitField0_ |= 0x00000200; - } - result.featureFlags_ = featureFlags_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance()) return this; - if (other.hasClientVersion()) { - setClientVersion(other.getClientVersion()); - } - if (other.hasAuthMethod()) { - setAuthMethod(other.getAuthMethod()); - } - if (other.hasAuthMethodName()) { - setAuthMethodName(other.getAuthMethodName()); - } - if (other.hasAuthData()) { - setAuthData(other.getAuthData()); - } - if (other.hasProtocolVersion()) { - setProtocolVersion(other.getProtocolVersion()); - } - if (other.hasProxyToBrokerUrl()) { - setProxyToBrokerUrl(other.getProxyToBrokerUrl()); - } - if (other.hasOriginalPrincipal()) { - setOriginalPrincipal(other.getOriginalPrincipal()); - } - if (other.hasOriginalAuthData()) { - setOriginalAuthData(other.getOriginalAuthData()); - } - if (other.hasOriginalAuthMethod()) { - setOriginalAuthMethod(other.getOriginalAuthMethod()); - } - if (other.hasFeatureFlags()) { - mergeFeatureFlags(other.getFeatureFlags()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasClientVersion()) { - - 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; - clientVersion_ = input.readBytes(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod value = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - authMethod_ = value; - } - break; - } - case 26: { - bitField0_ |= 0x00000008; - authData_ = input.readBytes(); - break; - } - case 32: { - bitField0_ |= 0x00000010; - protocolVersion_ = input.readInt32(); - break; - } - case 42: { - bitField0_ |= 0x00000004; - authMethodName_ = input.readBytes(); - break; - } - case 50: { - bitField0_ |= 0x00000020; - proxyToBrokerUrl_ = input.readBytes(); - break; - } - case 58: { - bitField0_ |= 0x00000040; - originalPrincipal_ = input.readBytes(); - break; - } - case 66: { - bitField0_ |= 0x00000080; - originalAuthData_ = input.readBytes(); - break; - } - case 74: { - bitField0_ |= 0x00000100; - originalAuthMethod_ = input.readBytes(); - break; - } - case 82: { - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.newBuilder(); - if (hasFeatureFlags()) { - subBuilder.mergeFrom(getFeatureFlags()); - } - input.readMessage(subBuilder, extensionRegistry); - setFeatureFlags(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - } - } - } - - private int bitField0_; - - // required string client_version = 1; - private java.lang.Object clientVersion_ = ""; - public boolean hasClientVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getClientVersion() { - java.lang.Object ref = clientVersion_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - clientVersion_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setClientVersion(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - clientVersion_ = value; - - return this; - } - public Builder clearClientVersion() { - bitField0_ = (bitField0_ & ~0x00000001); - clientVersion_ = getDefaultInstance().getClientVersion(); - - return this; - } - void setClientVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - clientVersion_ = value; - - } - - // optional .pulsar.proto.AuthMethod auth_method = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; - public boolean hasAuthMethod() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod() { - return authMethod_; - } - public Builder setAuthMethod(org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - authMethod_ = value; - - return this; - } - public Builder clearAuthMethod() { - bitField0_ = (bitField0_ & ~0x00000002); - authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; - - return this; - } - - // optional string auth_method_name = 5; - private java.lang.Object authMethodName_ = ""; - public boolean hasAuthMethodName() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getAuthMethodName() { - java.lang.Object ref = authMethodName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - authMethodName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setAuthMethodName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - authMethodName_ = value; - - return this; - } - public Builder clearAuthMethodName() { - bitField0_ = (bitField0_ & ~0x00000004); - authMethodName_ = getDefaultInstance().getAuthMethodName(); - - return this; - } - void setAuthMethodName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - authMethodName_ = value; - - } - - // optional bytes auth_data = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasAuthData() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { - return authData_; - } - public Builder setAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - authData_ = value; - - return this; - } - public Builder clearAuthData() { - bitField0_ = (bitField0_ & ~0x00000008); - authData_ = getDefaultInstance().getAuthData(); - - return this; - } - - // optional int32 protocol_version = 4 [default = 0]; - private int protocolVersion_ ; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public int getProtocolVersion() { - return protocolVersion_; - } - public Builder setProtocolVersion(int value) { - bitField0_ |= 0x00000010; - protocolVersion_ = value; - - return this; - } - public Builder clearProtocolVersion() { - bitField0_ = (bitField0_ & ~0x00000010); - protocolVersion_ = 0; - - return this; - } - - // optional string proxy_to_broker_url = 6; - private java.lang.Object proxyToBrokerUrl_ = ""; - public boolean hasProxyToBrokerUrl() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getProxyToBrokerUrl() { - java.lang.Object ref = proxyToBrokerUrl_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - proxyToBrokerUrl_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setProxyToBrokerUrl(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - proxyToBrokerUrl_ = value; - - return this; - } - public Builder clearProxyToBrokerUrl() { - bitField0_ = (bitField0_ & ~0x00000020); - proxyToBrokerUrl_ = getDefaultInstance().getProxyToBrokerUrl(); - - return this; - } - void setProxyToBrokerUrl(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000020; - proxyToBrokerUrl_ = value; - - } - - // optional string original_principal = 7; - private java.lang.Object originalPrincipal_ = ""; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalPrincipal_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalPrincipal(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000040; - originalPrincipal_ = value; - - return this; - } - public Builder clearOriginalPrincipal() { - bitField0_ = (bitField0_ & ~0x00000040); - originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); - - return this; - } - void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000040; - originalPrincipal_ = value; - - } - - // optional string original_auth_data = 8; - private java.lang.Object originalAuthData_ = ""; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthData_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthData(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000080; - originalAuthData_ = value; - - return this; - } - public Builder clearOriginalAuthData() { - bitField0_ = (bitField0_ & ~0x00000080); - originalAuthData_ = getDefaultInstance().getOriginalAuthData(); - - return this; - } - void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000080; - originalAuthData_ = value; - - } - - // optional string original_auth_method = 9; - private java.lang.Object originalAuthMethod_ = ""; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthMethod_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthMethod(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000100; - originalAuthMethod_ = value; - - return this; - } - public Builder clearOriginalAuthMethod() { - bitField0_ = (bitField0_ & ~0x00000100); - originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); - - return this; - } - void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000100; - originalAuthMethod_ = value; - - } - - // optional .pulsar.proto.FeatureFlags feature_flags = 10; - private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); - public boolean hasFeatureFlags() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags() { - return featureFlags_; - } - public Builder setFeatureFlags(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags value) { - if (value == null) { - throw new NullPointerException(); - } - featureFlags_ = value; - - bitField0_ |= 0x00000200; - return this; - } - public Builder setFeatureFlags( - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.Builder builderForValue) { - featureFlags_ = builderForValue.build(); - - bitField0_ |= 0x00000200; - return this; - } - public Builder mergeFeatureFlags(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags value) { - if (((bitField0_ & 0x00000200) == 0x00000200) && - featureFlags_ != org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance()) { - featureFlags_ = - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.newBuilder(featureFlags_).mergeFrom(value).buildPartial(); - } else { - featureFlags_ = value; - } - - bitField0_ |= 0x00000200; - return this; - } - public Builder clearFeatureFlags() { - featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000200); - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConnect) - } - - static { - defaultInstance = new CommandConnect(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConnect) - } - - public interface FeatureFlagsOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional bool supports_auth_refresh = 1 [default = false]; - boolean hasSupportsAuthRefresh(); - boolean getSupportsAuthRefresh(); - } - public static final class FeatureFlags extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements FeatureFlagsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use FeatureFlags.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private FeatureFlags(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected FeatureFlags newObject(Handle handle) { - return new FeatureFlags(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private FeatureFlags(boolean noInit) {} - - private static final FeatureFlags defaultInstance; - public static FeatureFlags getDefaultInstance() { - return defaultInstance; - } - - public FeatureFlags getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // optional bool supports_auth_refresh = 1 [default = false]; - public static final int SUPPORTS_AUTH_REFRESH_FIELD_NUMBER = 1; - private boolean supportsAuthRefresh_; - public boolean hasSupportsAuthRefresh() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public boolean getSupportsAuthRefresh() { - return supportsAuthRefresh_; - } - - private void initFields() { - supportsAuthRefresh_ = false; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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.writeBool(1, supportsAuthRefresh_); - } - } - - 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 - .computeBoolSize(1, supportsAuthRefresh_); - } - 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlagsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.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(); - supportsAuthRefresh_ = false; - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags build() { - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.supportsAuthRefresh_ = supportsAuthRefresh_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance()) return this; - if (other.hasSupportsAuthRefresh()) { - setSupportsAuthRefresh(other.getSupportsAuthRefresh()); - } - return this; - } - - public final boolean isInitialized() { - 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; - supportsAuthRefresh_ = input.readBool(); - break; - } - } - } - } - - private int bitField0_; - - // optional bool supports_auth_refresh = 1 [default = false]; - private boolean supportsAuthRefresh_ ; - public boolean hasSupportsAuthRefresh() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public boolean getSupportsAuthRefresh() { - return supportsAuthRefresh_; - } - public Builder setSupportsAuthRefresh(boolean value) { - bitField0_ |= 0x00000001; - supportsAuthRefresh_ = value; - - return this; - } - public Builder clearSupportsAuthRefresh() { - bitField0_ = (bitField0_ & ~0x00000001); - supportsAuthRefresh_ = false; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.FeatureFlags) - } - - static { - defaultInstance = new FeatureFlags(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.FeatureFlags) - } - - public interface CommandConnectedOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string server_version = 1; - boolean hasServerVersion(); - String getServerVersion(); - - // optional int32 protocol_version = 2 [default = 0]; - boolean hasProtocolVersion(); - int getProtocolVersion(); - - // optional int32 max_message_size = 3; - boolean hasMaxMessageSize(); - int getMaxMessageSize(); - } - public static final class CommandConnected extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandConnectedOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandConnected.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandConnected(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandConnected newObject(Handle handle) { - return new CommandConnected(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandConnected(boolean noInit) {} - - private static final CommandConnected defaultInstance; - public static CommandConnected getDefaultInstance() { - return defaultInstance; - } - - public CommandConnected getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string server_version = 1; - public static final int SERVER_VERSION_FIELD_NUMBER = 1; - private java.lang.Object serverVersion_; - public boolean hasServerVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getServerVersion() { - java.lang.Object ref = serverVersion_; - 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)) { - serverVersion_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getServerVersionBytes() { - java.lang.Object ref = serverVersion_; - 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); - serverVersion_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional int32 protocol_version = 2 [default = 0]; - public static final int PROTOCOL_VERSION_FIELD_NUMBER = 2; - private int protocolVersion_; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getProtocolVersion() { - return protocolVersion_; - } - - // optional int32 max_message_size = 3; - public static final int MAX_MESSAGE_SIZE_FIELD_NUMBER = 3; - private int maxMessageSize_; - public boolean hasMaxMessageSize() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getMaxMessageSize() { - return maxMessageSize_; - } - - private void initFields() { - serverVersion_ = ""; - protocolVersion_ = 0; - maxMessageSize_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasServerVersion()) { - 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, getServerVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeInt32(2, protocolVersion_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt32(3, maxMessageSize_); - } - } - - 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, getServerVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(2, protocolVersion_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, maxMessageSize_); - } - 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConnectedOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.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(); - serverVersion_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - protocolVersion_ = 0; - bitField0_ = (bitField0_ & ~0x00000002); - maxMessageSize_ = 0; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.serverVersion_ = serverVersion_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.protocolVersion_ = protocolVersion_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.maxMessageSize_ = maxMessageSize_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance()) return this; - if (other.hasServerVersion()) { - setServerVersion(other.getServerVersion()); - } - if (other.hasProtocolVersion()) { - setProtocolVersion(other.getProtocolVersion()); - } - if (other.hasMaxMessageSize()) { - setMaxMessageSize(other.getMaxMessageSize()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasServerVersion()) { - - 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; - serverVersion_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - protocolVersion_ = input.readInt32(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - maxMessageSize_ = input.readInt32(); - break; - } - } - } - } - - private int bitField0_; - - // required string server_version = 1; - private java.lang.Object serverVersion_ = ""; - public boolean hasServerVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getServerVersion() { - java.lang.Object ref = serverVersion_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - serverVersion_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setServerVersion(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - serverVersion_ = value; - - return this; - } - public Builder clearServerVersion() { - bitField0_ = (bitField0_ & ~0x00000001); - serverVersion_ = getDefaultInstance().getServerVersion(); - - return this; - } - void setServerVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - serverVersion_ = value; - - } - - // optional int32 protocol_version = 2 [default = 0]; - private int protocolVersion_ ; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getProtocolVersion() { - return protocolVersion_; - } - public Builder setProtocolVersion(int value) { - bitField0_ |= 0x00000002; - protocolVersion_ = value; - - return this; - } - public Builder clearProtocolVersion() { - bitField0_ = (bitField0_ & ~0x00000002); - protocolVersion_ = 0; - - return this; - } - - // optional int32 max_message_size = 3; - private int maxMessageSize_ ; - public boolean hasMaxMessageSize() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getMaxMessageSize() { - return maxMessageSize_; - } - public Builder setMaxMessageSize(int value) { - bitField0_ |= 0x00000004; - maxMessageSize_ = value; - - return this; - } - public Builder clearMaxMessageSize() { - bitField0_ = (bitField0_ & ~0x00000004); - maxMessageSize_ = 0; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConnected) - } - - static { - defaultInstance = new CommandConnected(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConnected) - } - - public interface CommandAuthResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional string client_version = 1; - boolean hasClientVersion(); - String getClientVersion(); - - // optional .pulsar.proto.AuthData response = 2; - boolean hasResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse(); - - // optional int32 protocol_version = 3 [default = 0]; - boolean hasProtocolVersion(); - int getProtocolVersion(); - } - public static final class CommandAuthResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAuthResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAuthResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAuthResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAuthResponse newObject(Handle handle) { - return new CommandAuthResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAuthResponse(boolean noInit) {} - - private static final CommandAuthResponse defaultInstance; - public static CommandAuthResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandAuthResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // optional string client_version = 1; - public static final int CLIENT_VERSION_FIELD_NUMBER = 1; - private java.lang.Object clientVersion_; - public boolean hasClientVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getClientVersion() { - java.lang.Object ref = clientVersion_; - 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)) { - clientVersion_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getClientVersionBytes() { - java.lang.Object ref = clientVersion_; - 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); - clientVersion_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.AuthData response = 2; - public static final int RESPONSE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthData response_; - public boolean hasResponse() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse() { - return response_; - } - - // optional int32 protocol_version = 3 [default = 0]; - public static final int PROTOCOL_VERSION_FIELD_NUMBER = 3; - private int protocolVersion_; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getProtocolVersion() { - return protocolVersion_; - } - - private void initFields() { - clientVersion_ = ""; - response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - protocolVersion_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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, getClientVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, response_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt32(3, protocolVersion_); - } - } - - 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, getClientVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, response_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, protocolVersion_); - } - 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.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(); - clientVersion_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000002); - protocolVersion_ = 0; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.clientVersion_ = clientVersion_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.response_ = response_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.protocolVersion_ = protocolVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance()) return this; - if (other.hasClientVersion()) { - setClientVersion(other.getClientVersion()); - } - if (other.hasResponse()) { - mergeResponse(other.getResponse()); - } - if (other.hasProtocolVersion()) { - setProtocolVersion(other.getProtocolVersion()); - } - return this; - } - - public final boolean isInitialized() { - 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; - clientVersion_ = input.readBytes(); - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(); - if (hasResponse()) { - subBuilder.mergeFrom(getResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - protocolVersion_ = input.readInt32(); - break; - } - } - } - } - - private int bitField0_; - - // optional string client_version = 1; - private java.lang.Object clientVersion_ = ""; - public boolean hasClientVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getClientVersion() { - java.lang.Object ref = clientVersion_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - clientVersion_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setClientVersion(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - clientVersion_ = value; - - return this; - } - public Builder clearClientVersion() { - bitField0_ = (bitField0_ & ~0x00000001); - clientVersion_ = getDefaultInstance().getClientVersion(); - - return this; - } - void setClientVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - clientVersion_ = value; - - } - - // optional .pulsar.proto.AuthData response = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthData response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - public boolean hasResponse() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse() { - return response_; - } - public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { - if (value == null) { - throw new NullPointerException(); - } - response_ = value; - - bitField0_ |= 0x00000002; - return this; - } - public Builder setResponse( - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder builderForValue) { - response_ = builderForValue.build(); - - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeResponse(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - response_ != org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) { - response_ = - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(response_).mergeFrom(value).buildPartial(); - } else { - response_ = value; - } - - bitField0_ |= 0x00000002; - return this; - } - public Builder clearResponse() { - response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - // optional int32 protocol_version = 3 [default = 0]; - private int protocolVersion_ ; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getProtocolVersion() { - return protocolVersion_; - } - public Builder setProtocolVersion(int value) { - bitField0_ |= 0x00000004; - protocolVersion_ = value; - - return this; - } - public Builder clearProtocolVersion() { - bitField0_ = (bitField0_ & ~0x00000004); - protocolVersion_ = 0; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAuthResponse) - } - - static { - defaultInstance = new CommandAuthResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAuthResponse) - } - - public interface CommandAuthChallengeOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional string server_version = 1; - boolean hasServerVersion(); - String getServerVersion(); - - // optional .pulsar.proto.AuthData challenge = 2; - boolean hasChallenge(); - org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge(); - - // optional int32 protocol_version = 3 [default = 0]; - boolean hasProtocolVersion(); - int getProtocolVersion(); - } - public static final class CommandAuthChallenge extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAuthChallengeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAuthChallenge.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAuthChallenge(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAuthChallenge newObject(Handle handle) { - return new CommandAuthChallenge(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAuthChallenge(boolean noInit) {} - - private static final CommandAuthChallenge defaultInstance; - public static CommandAuthChallenge getDefaultInstance() { - return defaultInstance; - } - - public CommandAuthChallenge getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // optional string server_version = 1; - public static final int SERVER_VERSION_FIELD_NUMBER = 1; - private java.lang.Object serverVersion_; - public boolean hasServerVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getServerVersion() { - java.lang.Object ref = serverVersion_; - 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)) { - serverVersion_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getServerVersionBytes() { - java.lang.Object ref = serverVersion_; - 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); - serverVersion_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.AuthData challenge = 2; - public static final int CHALLENGE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthData challenge_; - public boolean hasChallenge() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge() { - return challenge_; - } - - // optional int32 protocol_version = 3 [default = 0]; - public static final int PROTOCOL_VERSION_FIELD_NUMBER = 3; - private int protocolVersion_; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getProtocolVersion() { - return protocolVersion_; - } - - private void initFields() { - serverVersion_ = ""; - challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - protocolVersion_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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, getServerVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, challenge_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt32(3, protocolVersion_); - } - } - - 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, getServerVersionBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, challenge_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, protocolVersion_); - } - 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallengeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.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(); - serverVersion_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000002); - protocolVersion_ = 0; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.serverVersion_ = serverVersion_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.challenge_ = challenge_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.protocolVersion_ = protocolVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance()) return this; - if (other.hasServerVersion()) { - setServerVersion(other.getServerVersion()); - } - if (other.hasChallenge()) { - mergeChallenge(other.getChallenge()); - } - if (other.hasProtocolVersion()) { - setProtocolVersion(other.getProtocolVersion()); - } - return this; - } - - public final boolean isInitialized() { - 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; - serverVersion_ = input.readBytes(); - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(); - if (hasChallenge()) { - subBuilder.mergeFrom(getChallenge()); - } - input.readMessage(subBuilder, extensionRegistry); - setChallenge(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - protocolVersion_ = input.readInt32(); - break; - } - } - } - } - - private int bitField0_; - - // optional string server_version = 1; - private java.lang.Object serverVersion_ = ""; - public boolean hasServerVersion() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getServerVersion() { - java.lang.Object ref = serverVersion_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - serverVersion_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setServerVersion(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - serverVersion_ = value; - - return this; - } - public Builder clearServerVersion() { - bitField0_ = (bitField0_ & ~0x00000001); - serverVersion_ = getDefaultInstance().getServerVersion(); - - return this; - } - void setServerVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - serverVersion_ = value; - - } - - // optional .pulsar.proto.AuthData challenge = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.AuthData challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - public boolean hasChallenge() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge() { - return challenge_; - } - public Builder setChallenge(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { - if (value == null) { - throw new NullPointerException(); - } - challenge_ = value; - - bitField0_ |= 0x00000002; - return this; - } - public Builder setChallenge( - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder builderForValue) { - challenge_ = builderForValue.build(); - - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeChallenge(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - challenge_ != org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) { - challenge_ = - org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(challenge_).mergeFrom(value).buildPartial(); - } else { - challenge_ = value; - } - - bitField0_ |= 0x00000002; - return this; - } - public Builder clearChallenge() { - challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - // optional int32 protocol_version = 3 [default = 0]; - private int protocolVersion_ ; - public boolean hasProtocolVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getProtocolVersion() { - return protocolVersion_; - } - public Builder setProtocolVersion(int value) { - bitField0_ |= 0x00000004; - protocolVersion_ = value; - - return this; - } - public Builder clearProtocolVersion() { - bitField0_ = (bitField0_ & ~0x00000004); - protocolVersion_ = 0; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAuthChallenge) - } - - static { - defaultInstance = new CommandAuthChallenge(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAuthChallenge) - } - - public interface AuthDataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional string auth_method_name = 1; - boolean hasAuthMethodName(); - String getAuthMethodName(); - - // optional bytes auth_data = 2; - boolean hasAuthData(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData(); - } - public static final class AuthData extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements AuthDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use AuthData.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private AuthData(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected AuthData newObject(Handle handle) { - return new AuthData(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private AuthData(boolean noInit) {} - - private static final AuthData defaultInstance; - public static AuthData getDefaultInstance() { - return defaultInstance; - } - - public AuthData getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // optional string auth_method_name = 1; - public static final int AUTH_METHOD_NAME_FIELD_NUMBER = 1; - private java.lang.Object authMethodName_; - public boolean hasAuthMethodName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getAuthMethodName() { - java.lang.Object ref = authMethodName_; - 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)) { - authMethodName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthMethodNameBytes() { - java.lang.Object ref = authMethodName_; - 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); - authMethodName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bytes auth_data = 2; - public static final int AUTH_DATA_FIELD_NUMBER = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_; - public boolean hasAuthData() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { - return authData_; - } - - private void initFields() { - authMethodName_ = ""; - authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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, getAuthMethodNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, authData_); - } - } - - 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, getAuthMethodNameBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, authData_); - } - 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.AuthDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.AuthData.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(); - authMethodName_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData build() { - org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.AuthData buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.AuthData buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.authMethodName_ = authMethodName_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.authData_ = authData_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.AuthData other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) return this; - if (other.hasAuthMethodName()) { - setAuthMethodName(other.getAuthMethodName()); - } - if (other.hasAuthData()) { - setAuthData(other.getAuthData()); - } - return this; - } - - public final boolean isInitialized() { - 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; - authMethodName_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - authData_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // optional string auth_method_name = 1; - private java.lang.Object authMethodName_ = ""; - public boolean hasAuthMethodName() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getAuthMethodName() { - java.lang.Object ref = authMethodName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - authMethodName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setAuthMethodName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - authMethodName_ = value; - - return this; - } - public Builder clearAuthMethodName() { - bitField0_ = (bitField0_ & ~0x00000001); - authMethodName_ = getDefaultInstance().getAuthMethodName(); - - return this; - } - void setAuthMethodName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - authMethodName_ = value; - - } - - // optional bytes auth_data = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasAuthData() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { - return authData_; - } - public Builder setAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - authData_ = value; - - return this; - } - public Builder clearAuthData() { - bitField0_ = (bitField0_ & ~0x00000002); - authData_ = getDefaultInstance().getAuthData(); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.AuthData) - } - - static { - defaultInstance = new AuthData(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.AuthData) - } - - public interface KeySharedMetaOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required .pulsar.proto.KeySharedMode keySharedMode = 1; - boolean hasKeySharedMode(); - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode(); - - // repeated .pulsar.proto.IntRange hashRanges = 3; - java.util.List - getHashRangesList(); - org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index); - int getHashRangesCount(); - } - public static final class KeySharedMeta extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements KeySharedMetaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use KeySharedMeta.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private KeySharedMeta(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected KeySharedMeta newObject(Handle handle) { - return new KeySharedMeta(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private KeySharedMeta(boolean noInit) {} - - private static final KeySharedMeta defaultInstance; - public static KeySharedMeta getDefaultInstance() { - return defaultInstance; - } - - public KeySharedMeta getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required .pulsar.proto.KeySharedMode keySharedMode = 1; - public static final int KEYSHAREDMODE_FIELD_NUMBER = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode keySharedMode_; - public boolean hasKeySharedMode() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode() { - return keySharedMode_; - } - - // repeated .pulsar.proto.IntRange hashRanges = 3; - public static final int HASHRANGES_FIELD_NUMBER = 3; - private java.util.List hashRanges_; - public java.util.List getHashRangesList() { - return hashRanges_; - } - public java.util.List - getHashRangesOrBuilderList() { - return hashRanges_; - } - public int getHashRangesCount() { - return hashRanges_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index) { - return hashRanges_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getHashRangesOrBuilder( - int index) { - return hashRanges_.get(index); - } - - private void initFields() { - keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; - hashRanges_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasKeySharedMode()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getHashRangesCount(); i++) { - if (!getHashRanges(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.writeEnum(1, keySharedMode_.getNumber()); - } - for (int i = 0; i < hashRanges_.size(); i++) { - output.writeMessage(3, hashRanges_.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 - .computeEnumSize(1, keySharedMode_.getNumber()); - } - for (int i = 0; i < hashRanges_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, hashRanges_.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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMetaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.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(); - keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; - bitField0_ = (bitField0_ & ~0x00000001); - hashRanges_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta build() { - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.keySharedMode_ = keySharedMode_; - if (((bitField0_ & 0x00000002) == 0x00000002)) { - hashRanges_ = java.util.Collections.unmodifiableList(hashRanges_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.hashRanges_ = hashRanges_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance()) return this; - if (other.hasKeySharedMode()) { - setKeySharedMode(other.getKeySharedMode()); - } - if (!other.hashRanges_.isEmpty()) { - if (hashRanges_.isEmpty()) { - hashRanges_ = other.hashRanges_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureHashRangesIsMutable(); - hashRanges_.addAll(other.hashRanges_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasKeySharedMode()) { - - return false; - } - for (int i = 0; i < getHashRangesCount(); i++) { - if (!getHashRanges(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 8: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode value = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000001; - keySharedMode_ = value; - } - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addHashRanges(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required .pulsar.proto.KeySharedMode keySharedMode = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; - public boolean hasKeySharedMode() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode() { - return keySharedMode_; - } - public Builder setKeySharedMode(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - keySharedMode_ = value; - - return this; - } - public Builder clearKeySharedMode() { - bitField0_ = (bitField0_ & ~0x00000001); - keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; - - return this; - } - - // repeated .pulsar.proto.IntRange hashRanges = 3; - private java.util.List hashRanges_ = - java.util.Collections.emptyList(); - private void ensureHashRangesIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - hashRanges_ = new java.util.ArrayList(hashRanges_); - bitField0_ |= 0x00000002; - } - } - - public java.util.List getHashRangesList() { - return java.util.Collections.unmodifiableList(hashRanges_); - } - public int getHashRangesCount() { - return hashRanges_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index) { - return hashRanges_.get(index); - } - public Builder setHashRanges( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureHashRangesIsMutable(); - hashRanges_.set(index, value); - - return this; - } - public Builder setHashRanges( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureHashRangesIsMutable(); - hashRanges_.set(index, builderForValue.build()); - - return this; - } - public Builder addHashRanges(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureHashRangesIsMutable(); - hashRanges_.add(value); - - return this; - } - public Builder addHashRanges( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureHashRangesIsMutable(); - hashRanges_.add(index, value); - - return this; - } - public Builder addHashRanges( - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureHashRangesIsMutable(); - hashRanges_.add(builderForValue.build()); - - return this; - } - public Builder addHashRanges( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureHashRangesIsMutable(); - hashRanges_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllHashRanges( - java.lang.Iterable values) { - ensureHashRangesIsMutable(); - super.addAll(values, hashRanges_); - - return this; - } - public Builder clearHashRanges() { - hashRanges_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - - return this; - } - public Builder removeHashRanges(int index) { - ensureHashRangesIsMutable(); - hashRanges_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.KeySharedMeta) - } - - static { - defaultInstance = new KeySharedMeta(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.KeySharedMeta) - } - - public interface CommandSubscribeOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string topic = 1; - boolean hasTopic(); - String getTopic(); - - // required string subscription = 2; - boolean hasSubscription(); - String getSubscription(); - - // required .pulsar.proto.CommandSubscribe.SubType subType = 3; - boolean hasSubType(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType(); - - // required uint64 consumer_id = 4; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint64 request_id = 5; - boolean hasRequestId(); - long getRequestId(); - - // optional string consumer_name = 6; - boolean hasConsumerName(); - String getConsumerName(); - - // optional int32 priority_level = 7; - boolean hasPriorityLevel(); - int getPriorityLevel(); - - // optional bool durable = 8 [default = true]; - boolean hasDurable(); - boolean getDurable(); - - // optional .pulsar.proto.MessageIdData start_message_id = 9; - boolean hasStartMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId(); - - // repeated .pulsar.proto.KeyValue metadata = 10; - java.util.List - getMetadataList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); - int getMetadataCount(); - - // optional bool read_compacted = 11; - boolean hasReadCompacted(); - boolean getReadCompacted(); - - // optional .pulsar.proto.Schema schema = 12; - boolean hasSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); - - // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; - boolean hasInitialPosition(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition(); - - // optional bool replicate_subscription_state = 14; - boolean hasReplicateSubscriptionState(); - boolean getReplicateSubscriptionState(); - - // optional bool force_topic_creation = 15 [default = true]; - boolean hasForceTopicCreation(); - boolean getForceTopicCreation(); - - // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; - boolean hasStartMessageRollbackDurationSec(); - long getStartMessageRollbackDurationSec(); - - // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; - boolean hasKeySharedMeta(); - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta(); - } - public static final class CommandSubscribe extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSubscribe.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSubscribe(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSubscribe newObject(Handle handle) { - return new CommandSubscribe(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSubscribe(boolean noInit) {} - - private static final CommandSubscribe defaultInstance; - public static CommandSubscribe getDefaultInstance() { - return defaultInstance; - } - - public CommandSubscribe getDefaultInstanceForType() { - return defaultInstance; - } - - public enum SubType - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - Exclusive(0, 0), - Shared(1, 1), - Failover(2, 2), - Key_Shared(3, 3), - ; - - public static final int Exclusive_VALUE = 0; - public static final int Shared_VALUE = 1; - public static final int Failover_VALUE = 2; - public static final int Key_Shared_VALUE = 3; - - - public final int getNumber() { return value; } - - public static SubType valueOf(int value) { - switch (value) { - case 0: return Exclusive; - case 1: return Shared; - case 2: return Failover; - case 3: return Key_Shared; - 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 SubType findValueByNumber(int number) { - return SubType.valueOf(number); - } - }; - - private final int value; - - private SubType(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandSubscribe.SubType) - } - - public enum InitialPosition - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - Latest(0, 0), - Earliest(1, 1), - ; - - public static final int Latest_VALUE = 0; - public static final int Earliest_VALUE = 1; - - - public final int getNumber() { return value; } - - public static InitialPosition valueOf(int value) { - switch (value) { - case 0: return Latest; - case 1: return Earliest; - 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 InitialPosition findValueByNumber(int number) { - return InitialPosition.valueOf(number); - } - }; - - private final int value; - - private InitialPosition(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandSubscribe.InitialPosition) - } - - private int bitField0_; - // required string topic = 1; - public static final int TOPIC_FIELD_NUMBER = 1; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required string subscription = 2; - public static final int SUBSCRIPTION_FIELD_NUMBER = 2; - private java.lang.Object subscription_; - public boolean hasSubscription() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getSubscription() { - java.lang.Object ref = subscription_; - 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)) { - subscription_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSubscriptionBytes() { - java.lang.Object ref = subscription_; - 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); - subscription_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required .pulsar.proto.CommandSubscribe.SubType subType = 3; - public static final int SUBTYPE_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType subType_; - public boolean hasSubType() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType() { - return subType_; - } - - // required uint64 consumer_id = 4; - public static final int CONSUMER_ID_FIELD_NUMBER = 4; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint64 request_id = 5; - public static final int REQUEST_ID_FIELD_NUMBER = 5; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getRequestId() { - return requestId_; - } - - // optional string consumer_name = 6; - public static final int CONSUMER_NAME_FIELD_NUMBER = 6; - private java.lang.Object consumerName_; - public boolean hasConsumerName() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getConsumerName() { - java.lang.Object ref = consumerName_; - 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)) { - consumerName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConsumerNameBytes() { - java.lang.Object ref = consumerName_; - 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); - consumerName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional int32 priority_level = 7; - public static final int PRIORITY_LEVEL_FIELD_NUMBER = 7; - private int priorityLevel_; - public boolean hasPriorityLevel() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public int getPriorityLevel() { - return priorityLevel_; - } - - // optional bool durable = 8 [default = true]; - public static final int DURABLE_FIELD_NUMBER = 8; - private boolean durable_; - public boolean hasDurable() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public boolean getDurable() { - return durable_; - } - - // optional .pulsar.proto.MessageIdData start_message_id = 9; - public static final int START_MESSAGE_ID_FIELD_NUMBER = 9; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData startMessageId_; - public boolean hasStartMessageId() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId() { - return startMessageId_; - } - - // repeated .pulsar.proto.KeyValue metadata = 10; - public static final int METADATA_FIELD_NUMBER = 10; - private java.util.List metadata_; - public java.util.List getMetadataList() { - return metadata_; - } - public java.util.List - getMetadataOrBuilderList() { - return metadata_; - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( - int index) { - return metadata_.get(index); - } - - // optional bool read_compacted = 11; - public static final int READ_COMPACTED_FIELD_NUMBER = 11; - private boolean readCompacted_; - public boolean hasReadCompacted() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public boolean getReadCompacted() { - return readCompacted_; - } - - // optional .pulsar.proto.Schema schema = 12; - public static final int SCHEMA_FIELD_NUMBER = 12; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; - public boolean hasSchema() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - - // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; - public static final int INITIALPOSITION_FIELD_NUMBER = 13; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition initialPosition_; - public boolean hasInitialPosition() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition() { - return initialPosition_; - } - - // optional bool replicate_subscription_state = 14; - public static final int REPLICATE_SUBSCRIPTION_STATE_FIELD_NUMBER = 14; - private boolean replicateSubscriptionState_; - public boolean hasReplicateSubscriptionState() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public boolean getReplicateSubscriptionState() { - return replicateSubscriptionState_; - } - - // optional bool force_topic_creation = 15 [default = true]; - public static final int FORCE_TOPIC_CREATION_FIELD_NUMBER = 15; - private boolean forceTopicCreation_; - public boolean hasForceTopicCreation() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public boolean getForceTopicCreation() { - return forceTopicCreation_; - } - - // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; - public static final int START_MESSAGE_ROLLBACK_DURATION_SEC_FIELD_NUMBER = 16; - private long startMessageRollbackDurationSec_; - public boolean hasStartMessageRollbackDurationSec() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public long getStartMessageRollbackDurationSec() { - return startMessageRollbackDurationSec_; - } - - // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; - public static final int KEYSHAREDMETA_FIELD_NUMBER = 17; - private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta keySharedMeta_; - public boolean hasKeySharedMeta() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta() { - return keySharedMeta_; - } - - private void initFields() { - topic_ = ""; - subscription_ = ""; - subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; - consumerId_ = 0L; - requestId_ = 0L; - consumerName_ = ""; - priorityLevel_ = 0; - durable_ = true; - startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - metadata_ = java.util.Collections.emptyList(); - readCompacted_ = false; - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; - replicateSubscriptionState_ = false; - forceTopicCreation_ = true; - startMessageRollbackDurationSec_ = 0L; - keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSubscription()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSubType()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasStartMessageId()) { - if (!getStartMessageId().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSchema()) { - if (!getSchema().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasKeySharedMeta()) { - if (!getKeySharedMeta().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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getSubscriptionBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(3, subType_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(4, consumerId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeUInt64(5, requestId_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(6, getConsumerNameBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeInt32(7, priorityLevel_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeBool(8, durable_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeMessage(9, startMessageId_); - } - for (int i = 0; i < metadata_.size(); i++) { - output.writeMessage(10, metadata_.get(i)); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - output.writeBool(11, readCompacted_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - output.writeMessage(12, schema_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - output.writeEnum(13, initialPosition_.getNumber()); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - output.writeBool(14, replicateSubscriptionState_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - output.writeBool(15, forceTopicCreation_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - output.writeUInt64(16, startMessageRollbackDurationSec_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - output.writeMessage(17, keySharedMeta_); - } - } - - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getSubscriptionBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(3, subType_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, consumerId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(5, requestId_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(6, getConsumerNameBytes()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(7, priorityLevel_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(8, durable_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(9, startMessageId_); - } - for (int i = 0; i < metadata_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(10, metadata_.get(i)); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(11, readCompacted_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(12, schema_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(13, initialPosition_.getNumber()); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(14, replicateSubscriptionState_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(15, forceTopicCreation_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(16, startMessageRollbackDurationSec_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(17, keySharedMeta_); - } - 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.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(); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - subscription_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; - bitField0_ = (bitField0_ & ~0x00000004); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000010); - consumerName_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - priorityLevel_ = 0; - bitField0_ = (bitField0_ & ~0x00000040); - durable_ = true; - bitField0_ = (bitField0_ & ~0x00000080); - startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000100); - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000200); - readCompacted_ = false; - bitField0_ = (bitField0_ & ~0x00000400); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000800); - initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; - bitField0_ = (bitField0_ & ~0x00001000); - replicateSubscriptionState_ = false; - bitField0_ = (bitField0_ & ~0x00002000); - forceTopicCreation_ = true; - bitField0_ = (bitField0_ & ~0x00004000); - startMessageRollbackDurationSec_ = 0L; - bitField0_ = (bitField0_ & ~0x00008000); - keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00010000); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.subscription_ = subscription_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.subType_ = subType_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.consumerName_ = consumerName_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.priorityLevel_ = priorityLevel_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.durable_ = durable_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.startMessageId_ = startMessageId_; - if (((bitField0_ & 0x00000200) == 0x00000200)) { - metadata_ = java.util.Collections.unmodifiableList(metadata_); - bitField0_ = (bitField0_ & ~0x00000200); - } - result.metadata_ = metadata_; - if (((from_bitField0_ & 0x00000400) == 0x00000400)) { - to_bitField0_ |= 0x00000200; - } - result.readCompacted_ = readCompacted_; - if (((from_bitField0_ & 0x00000800) == 0x00000800)) { - to_bitField0_ |= 0x00000400; - } - result.schema_ = schema_; - if (((from_bitField0_ & 0x00001000) == 0x00001000)) { - to_bitField0_ |= 0x00000800; - } - result.initialPosition_ = initialPosition_; - if (((from_bitField0_ & 0x00002000) == 0x00002000)) { - to_bitField0_ |= 0x00001000; - } - result.replicateSubscriptionState_ = replicateSubscriptionState_; - if (((from_bitField0_ & 0x00004000) == 0x00004000)) { - to_bitField0_ |= 0x00002000; - } - result.forceTopicCreation_ = forceTopicCreation_; - if (((from_bitField0_ & 0x00008000) == 0x00008000)) { - to_bitField0_ |= 0x00004000; - } - result.startMessageRollbackDurationSec_ = startMessageRollbackDurationSec_; - if (((from_bitField0_ & 0x00010000) == 0x00010000)) { - to_bitField0_ |= 0x00008000; - } - result.keySharedMeta_ = keySharedMeta_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance()) return this; - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasSubscription()) { - setSubscription(other.getSubscription()); - } - if (other.hasSubType()) { - setSubType(other.getSubType()); - } - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasConsumerName()) { - setConsumerName(other.getConsumerName()); - } - if (other.hasPriorityLevel()) { - setPriorityLevel(other.getPriorityLevel()); - } - if (other.hasDurable()) { - setDurable(other.getDurable()); - } - if (other.hasStartMessageId()) { - mergeStartMessageId(other.getStartMessageId()); - } - if (!other.metadata_.isEmpty()) { - if (metadata_.isEmpty()) { - metadata_ = other.metadata_; - bitField0_ = (bitField0_ & ~0x00000200); - } else { - ensureMetadataIsMutable(); - metadata_.addAll(other.metadata_); - } - - } - if (other.hasReadCompacted()) { - setReadCompacted(other.getReadCompacted()); - } - if (other.hasSchema()) { - mergeSchema(other.getSchema()); - } - if (other.hasInitialPosition()) { - setInitialPosition(other.getInitialPosition()); - } - if (other.hasReplicateSubscriptionState()) { - setReplicateSubscriptionState(other.getReplicateSubscriptionState()); - } - if (other.hasForceTopicCreation()) { - setForceTopicCreation(other.getForceTopicCreation()); - } - if (other.hasStartMessageRollbackDurationSec()) { - setStartMessageRollbackDurationSec(other.getStartMessageRollbackDurationSec()); - } - if (other.hasKeySharedMeta()) { - mergeKeySharedMeta(other.getKeySharedMeta()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasTopic()) { - - return false; - } - if (!hasSubscription()) { - - return false; - } - if (!hasSubType()) { - - return false; - } - if (!hasConsumerId()) { - - return false; - } - if (!hasRequestId()) { - - return false; - } - if (hasStartMessageId()) { - if (!getStartMessageId().isInitialized()) { - - return false; - } - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(i).isInitialized()) { - - return false; - } - } - if (hasSchema()) { - if (!getSchema().isInitialized()) { - - return false; - } - } - if (hasKeySharedMeta()) { - if (!getKeySharedMeta().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; - topic_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - subscription_ = input.readBytes(); - break; - } - case 24: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - subType_ = value; - } - break; - } - case 32: { - bitField0_ |= 0x00000008; - consumerId_ = input.readUInt64(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - requestId_ = input.readUInt64(); - break; - } - case 50: { - bitField0_ |= 0x00000020; - consumerName_ = input.readBytes(); - break; - } - case 56: { - bitField0_ |= 0x00000040; - priorityLevel_ = input.readInt32(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - durable_ = input.readBool(); - break; - } - case 74: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasStartMessageId()) { - subBuilder.mergeFrom(getStartMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setStartMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 82: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMetadata(subBuilder.buildPartial()); - break; - } - case 88: { - bitField0_ |= 0x00000400; - readCompacted_ = input.readBool(); - break; - } - case 98: { - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); - if (hasSchema()) { - subBuilder.mergeFrom(getSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 104: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition value = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00001000; - initialPosition_ = value; - } - break; - } - case 112: { - bitField0_ |= 0x00002000; - replicateSubscriptionState_ = input.readBool(); - break; - } - case 120: { - bitField0_ |= 0x00004000; - forceTopicCreation_ = input.readBool(); - break; - } - case 128: { - bitField0_ |= 0x00008000; - startMessageRollbackDurationSec_ = input.readUInt64(); - break; - } - case 138: { - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.newBuilder(); - if (hasKeySharedMeta()) { - subBuilder.mergeFrom(getKeySharedMeta()); - } - input.readMessage(subBuilder, extensionRegistry); - setKeySharedMeta(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - } - } - } - - private int bitField0_; - - // required string topic = 1; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - topic_ = value; - - } - - // required string subscription = 2; - private java.lang.Object subscription_ = ""; - public boolean hasSubscription() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getSubscription() { - java.lang.Object ref = subscription_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - subscription_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setSubscription(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - subscription_ = value; - - return this; - } - public Builder clearSubscription() { - bitField0_ = (bitField0_ & ~0x00000002); - subscription_ = getDefaultInstance().getSubscription(); - - return this; - } - void setSubscription(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - subscription_ = value; - - } - - // required .pulsar.proto.CommandSubscribe.SubType subType = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; - public boolean hasSubType() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType() { - return subType_; - } - public Builder setSubType(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - subType_ = value; - - return this; - } - public Builder clearSubType() { - bitField0_ = (bitField0_ & ~0x00000004); - subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; - - return this; - } - - // required uint64 consumer_id = 4; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000008; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000008); - consumerId_ = 0L; - - return this; - } - - // required uint64 request_id = 5; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000010; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000010); - requestId_ = 0L; - - return this; - } - - // optional string consumer_name = 6; - private java.lang.Object consumerName_ = ""; - public boolean hasConsumerName() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getConsumerName() { - java.lang.Object ref = consumerName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - consumerName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setConsumerName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - consumerName_ = value; - - return this; - } - public Builder clearConsumerName() { - bitField0_ = (bitField0_ & ~0x00000020); - consumerName_ = getDefaultInstance().getConsumerName(); - - return this; - } - void setConsumerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000020; - consumerName_ = value; - - } - - // optional int32 priority_level = 7; - private int priorityLevel_ ; - public boolean hasPriorityLevel() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public int getPriorityLevel() { - return priorityLevel_; - } - public Builder setPriorityLevel(int value) { - bitField0_ |= 0x00000040; - priorityLevel_ = value; - - return this; - } - public Builder clearPriorityLevel() { - bitField0_ = (bitField0_ & ~0x00000040); - priorityLevel_ = 0; - - return this; - } - - // optional bool durable = 8 [default = true]; - private boolean durable_ = true; - public boolean hasDurable() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public boolean getDurable() { - return durable_; - } - public Builder setDurable(boolean value) { - bitField0_ |= 0x00000080; - durable_ = value; - - return this; - } - public Builder clearDurable() { - bitField0_ = (bitField0_ & ~0x00000080); - durable_ = true; - - return this; - } - - // optional .pulsar.proto.MessageIdData start_message_id = 9; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasStartMessageId() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId() { - return startMessageId_; - } - public Builder setStartMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - startMessageId_ = value; - - bitField0_ |= 0x00000100; - return this; - } - public Builder setStartMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - startMessageId_ = builderForValue.build(); - - bitField0_ |= 0x00000100; - return this; - } - public Builder mergeStartMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000100) == 0x00000100) && - startMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - startMessageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(startMessageId_).mergeFrom(value).buildPartial(); - } else { - startMessageId_ = value; - } - - bitField0_ |= 0x00000100; - return this; - } - public Builder clearStartMessageId() { - startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000100); - return this; - } - - // repeated .pulsar.proto.KeyValue metadata = 10; - private java.util.List metadata_ = - java.util.Collections.emptyList(); - private void ensureMetadataIsMutable() { - if (!((bitField0_ & 0x00000200) == 0x00000200)) { - metadata_ = new java.util.ArrayList(metadata_); - bitField0_ |= 0x00000200; - } - } - - public java.util.List getMetadataList() { - return java.util.Collections.unmodifiableList(metadata_); - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.set(index, value); - - return this; - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.set(index, builderForValue.build()); - - return this; - } - public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(value); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(index, value); - - return this; - } - public Builder addMetadata( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(builderForValue.build()); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllMetadata( - java.lang.Iterable values) { - ensureMetadataIsMutable(); - super.addAll(values, metadata_); - - return this; - } - public Builder clearMetadata() { - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000200); - - return this; - } - public Builder removeMetadata(int index) { - ensureMetadataIsMutable(); - metadata_.remove(index); - - return this; - } - - // optional bool read_compacted = 11; - private boolean readCompacted_ ; - public boolean hasReadCompacted() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public boolean getReadCompacted() { - return readCompacted_; - } - public Builder setReadCompacted(boolean value) { - bitField0_ |= 0x00000400; - readCompacted_ = value; - - return this; - } - public Builder clearReadCompacted() { - bitField0_ = (bitField0_ & ~0x00000400); - readCompacted_ = false; - - return this; - } - - // optional .pulsar.proto.Schema schema = 12; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - public boolean hasSchema() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (value == null) { - throw new NullPointerException(); - } - schema_ = value; - - bitField0_ |= 0x00000800; - return this; - } - public Builder setSchema( - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { - schema_ = builderForValue.build(); - - bitField0_ |= 0x00000800; - return this; - } - public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (((bitField0_ & 0x00000800) == 0x00000800) && - schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { - schema_ = - org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); - } else { - schema_ = value; - } - - bitField0_ |= 0x00000800; - return this; - } - public Builder clearSchema() { - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000800); - return this; - } - - // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; - public boolean hasInitialPosition() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition() { - return initialPosition_; - } - public Builder setInitialPosition(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00001000; - initialPosition_ = value; - - return this; - } - public Builder clearInitialPosition() { - bitField0_ = (bitField0_ & ~0x00001000); - initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; - - return this; - } - - // optional bool replicate_subscription_state = 14; - private boolean replicateSubscriptionState_ ; - public boolean hasReplicateSubscriptionState() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public boolean getReplicateSubscriptionState() { - return replicateSubscriptionState_; - } - public Builder setReplicateSubscriptionState(boolean value) { - bitField0_ |= 0x00002000; - replicateSubscriptionState_ = value; - - return this; - } - public Builder clearReplicateSubscriptionState() { - bitField0_ = (bitField0_ & ~0x00002000); - replicateSubscriptionState_ = false; - - return this; - } - - // optional bool force_topic_creation = 15 [default = true]; - private boolean forceTopicCreation_ = true; - public boolean hasForceTopicCreation() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public boolean getForceTopicCreation() { - return forceTopicCreation_; - } - public Builder setForceTopicCreation(boolean value) { - bitField0_ |= 0x00004000; - forceTopicCreation_ = value; - - return this; - } - public Builder clearForceTopicCreation() { - bitField0_ = (bitField0_ & ~0x00004000); - forceTopicCreation_ = true; - - return this; - } - - // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; - private long startMessageRollbackDurationSec_ ; - public boolean hasStartMessageRollbackDurationSec() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public long getStartMessageRollbackDurationSec() { - return startMessageRollbackDurationSec_; - } - public Builder setStartMessageRollbackDurationSec(long value) { - bitField0_ |= 0x00008000; - startMessageRollbackDurationSec_ = value; - - return this; - } - public Builder clearStartMessageRollbackDurationSec() { - bitField0_ = (bitField0_ & ~0x00008000); - startMessageRollbackDurationSec_ = 0L; - - return this; - } - - // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; - private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); - public boolean hasKeySharedMeta() { - return ((bitField0_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta() { - return keySharedMeta_; - } - public Builder setKeySharedMeta(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta value) { - if (value == null) { - throw new NullPointerException(); - } - keySharedMeta_ = value; - - bitField0_ |= 0x00010000; - return this; - } - public Builder setKeySharedMeta( - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.Builder builderForValue) { - keySharedMeta_ = builderForValue.build(); - - bitField0_ |= 0x00010000; - return this; - } - public Builder mergeKeySharedMeta(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta value) { - if (((bitField0_ & 0x00010000) == 0x00010000) && - keySharedMeta_ != org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance()) { - keySharedMeta_ = - org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.newBuilder(keySharedMeta_).mergeFrom(value).buildPartial(); - } else { - keySharedMeta_ = value; - } - - bitField0_ |= 0x00010000; - return this; - } - public Builder clearKeySharedMeta() { - keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00010000); - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSubscribe) - } - - static { - defaultInstance = new CommandSubscribe(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSubscribe) - } - - public interface CommandPartitionedTopicMetadataOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string topic = 1; - boolean hasTopic(); - String getTopic(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - - // optional string original_principal = 3; - boolean hasOriginalPrincipal(); - String getOriginalPrincipal(); - - // optional string original_auth_data = 4; - boolean hasOriginalAuthData(); - String getOriginalAuthData(); - - // optional string original_auth_method = 5; - boolean hasOriginalAuthMethod(); - String getOriginalAuthMethod(); - } - public static final class CommandPartitionedTopicMetadata extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandPartitionedTopicMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandPartitionedTopicMetadata.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandPartitionedTopicMetadata(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandPartitionedTopicMetadata newObject(Handle handle) { - return new CommandPartitionedTopicMetadata(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandPartitionedTopicMetadata(boolean noInit) {} - - private static final CommandPartitionedTopicMetadata defaultInstance; - public static CommandPartitionedTopicMetadata getDefaultInstance() { - return defaultInstance; - } - - public CommandPartitionedTopicMetadata getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string topic = 1; - public static final int TOPIC_FIELD_NUMBER = 1; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - // optional string original_principal = 3; - public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 3; - private java.lang.Object originalPrincipal_; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - 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)) { - originalPrincipal_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { - java.lang.Object ref = originalPrincipal_; - 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); - originalPrincipal_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_data = 4; - public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 4; - private java.lang.Object originalAuthData_; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - 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)) { - originalAuthData_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { - java.lang.Object ref = originalAuthData_; - 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); - originalAuthData_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_method = 5; - public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 5; - private java.lang.Object originalAuthMethod_; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - 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)) { - originalAuthMethod_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { - java.lang.Object ref = originalAuthMethod_; - 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); - originalAuthMethod_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - topic_ = ""; - requestId_ = 0L; - originalPrincipal_ = ""; - originalAuthData_ = ""; - originalAuthMethod_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getOriginalAuthMethodBytes()); - } - } - - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getOriginalAuthMethodBytes()); - } - 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.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(); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - originalPrincipal_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - originalAuthData_ = ""; - bitField0_ = (bitField0_ & ~0x00000008); - originalAuthMethod_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.originalPrincipal_ = originalPrincipal_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.originalAuthData_ = originalAuthData_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.originalAuthMethod_ = originalAuthMethod_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance()) return this; - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasOriginalPrincipal()) { - setOriginalPrincipal(other.getOriginalPrincipal()); - } - if (other.hasOriginalAuthData()) { - setOriginalAuthData(other.getOriginalAuthData()); - } - if (other.hasOriginalAuthMethod()) { - setOriginalAuthMethod(other.getOriginalAuthMethod()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasTopic()) { - - return false; - } - if (!hasRequestId()) { - - 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; - topic_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - case 26: { - bitField0_ |= 0x00000004; - originalPrincipal_ = input.readBytes(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - originalAuthData_ = input.readBytes(); - break; - } - case 42: { - bitField0_ |= 0x00000010; - originalAuthMethod_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required string topic = 1; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - topic_ = value; - - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // optional string original_principal = 3; - private java.lang.Object originalPrincipal_ = ""; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalPrincipal_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalPrincipal(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - originalPrincipal_ = value; - - return this; - } - public Builder clearOriginalPrincipal() { - bitField0_ = (bitField0_ & ~0x00000004); - originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); - - return this; - } - void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - originalPrincipal_ = value; - - } - - // optional string original_auth_data = 4; - private java.lang.Object originalAuthData_ = ""; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthData_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthData(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - originalAuthData_ = value; - - return this; - } - public Builder clearOriginalAuthData() { - bitField0_ = (bitField0_ & ~0x00000008); - originalAuthData_ = getDefaultInstance().getOriginalAuthData(); - - return this; - } - void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000008; - originalAuthData_ = value; - - } - - // optional string original_auth_method = 5; - private java.lang.Object originalAuthMethod_ = ""; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthMethod_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthMethod(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - originalAuthMethod_ = value; - - return this; - } - public Builder clearOriginalAuthMethod() { - bitField0_ = (bitField0_ & ~0x00000010); - originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); - - return this; - } - void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - originalAuthMethod_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPartitionedTopicMetadata) - } - - static { - defaultInstance = new CommandPartitionedTopicMetadata(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPartitionedTopicMetadata) - } - - public interface CommandPartitionedTopicMetadataResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional uint32 partitions = 1; - boolean hasPartitions(); - int getPartitions(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; - boolean hasResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandPartitionedTopicMetadataResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandPartitionedTopicMetadataResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandPartitionedTopicMetadataResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandPartitionedTopicMetadataResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandPartitionedTopicMetadataResponse newObject(Handle handle) { - return new CommandPartitionedTopicMetadataResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandPartitionedTopicMetadataResponse(boolean noInit) {} - - private static final CommandPartitionedTopicMetadataResponse defaultInstance; - public static CommandPartitionedTopicMetadataResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandPartitionedTopicMetadataResponse getDefaultInstanceForType() { - return defaultInstance; - } - - public enum LookupType - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - Success(0, 0), - Failed(1, 1), - ; - - public static final int Success_VALUE = 0; - public static final int Failed_VALUE = 1; - - - public final int getNumber() { return value; } - - public static LookupType valueOf(int value) { - switch (value) { - case 0: return Success; - case 1: return Failed; - 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 LookupType findValueByNumber(int number) { - return LookupType.valueOf(number); - } - }; - - private final int value; - - private LookupType(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType) - } - - private int bitField0_; - // optional uint32 partitions = 1; - public static final int PARTITIONS_FIELD_NUMBER = 1; - private int partitions_; - public boolean hasPartitions() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public int getPartitions() { - return partitions_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; - public static final int RESPONSE_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType response_; - public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse() { - return response_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - partitions_ = 0; - requestId_ = 0L; - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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.writeUInt32(1, partitions_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(3, response_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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 - .computeUInt32Size(1, partitions_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(3, response_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.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(); - partitions_ = 0; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.partitions_ = partitions_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.response_ = response_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance()) return this; - if (other.hasPartitions()) { - setPartitions(other.getPartitions()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasResponse()) { - setResponse(other.getResponse()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - partitions_ = input.readUInt32(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - case 24: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - response_ = value; - } - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // optional uint32 partitions = 1; - private int partitions_ ; - public boolean hasPartitions() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public int getPartitions() { - return partitions_; - } - public Builder setPartitions(int value) { - bitField0_ |= 0x00000001; - partitions_ = value; - - return this; - } - public Builder clearPartitions() { - bitField0_ = (bitField0_ & ~0x00000001); - partitions_ = 0; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; - public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse() { - return response_; - } - public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - response_ = value; - - return this; - } - public Builder clearResponse() { - bitField0_ = (bitField0_ & ~0x00000004); - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse) - } - - static { - defaultInstance = new CommandPartitionedTopicMetadataResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse) - } - - public interface CommandLookupTopicOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string topic = 1; - boolean hasTopic(); - String getTopic(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - - // optional bool authoritative = 3 [default = false]; - boolean hasAuthoritative(); - boolean getAuthoritative(); - - // optional string original_principal = 4; - boolean hasOriginalPrincipal(); - String getOriginalPrincipal(); - - // optional string original_auth_data = 5; - boolean hasOriginalAuthData(); - String getOriginalAuthData(); - - // optional string original_auth_method = 6; - boolean hasOriginalAuthMethod(); - String getOriginalAuthMethod(); - } - public static final class CommandLookupTopic extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandLookupTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandLookupTopic.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandLookupTopic(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandLookupTopic newObject(Handle handle) { - return new CommandLookupTopic(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandLookupTopic(boolean noInit) {} - - private static final CommandLookupTopic defaultInstance; - public static CommandLookupTopic getDefaultInstance() { - return defaultInstance; - } - - public CommandLookupTopic getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string topic = 1; - public static final int TOPIC_FIELD_NUMBER = 1; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - // optional bool authoritative = 3 [default = false]; - public static final int AUTHORITATIVE_FIELD_NUMBER = 3; - private boolean authoritative_; - public boolean hasAuthoritative() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public boolean getAuthoritative() { - return authoritative_; - } - - // optional string original_principal = 4; - public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 4; - private java.lang.Object originalPrincipal_; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - 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)) { - originalPrincipal_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { - java.lang.Object ref = originalPrincipal_; - 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); - originalPrincipal_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_data = 5; - public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 5; - private java.lang.Object originalAuthData_; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - 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)) { - originalAuthData_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { - java.lang.Object ref = originalAuthData_; - 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); - originalAuthData_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string original_auth_method = 6; - public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 6; - private java.lang.Object originalAuthMethod_; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - 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)) { - originalAuthMethod_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { - java.lang.Object ref = originalAuthMethod_; - 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); - originalAuthMethod_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - topic_ = ""; - requestId_ = 0L; - authoritative_ = false; - originalPrincipal_ = ""; - originalAuthData_ = ""; - originalAuthMethod_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBool(3, authoritative_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeBytes(6, getOriginalAuthMethodBytes()); - } - } - - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(3, authoritative_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, getOriginalPrincipalBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getOriginalAuthDataBytes()); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(6, getOriginalAuthMethodBytes()); - } - 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.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(); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - authoritative_ = false; - bitField0_ = (bitField0_ & ~0x00000004); - originalPrincipal_ = ""; - bitField0_ = (bitField0_ & ~0x00000008); - originalAuthData_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - originalAuthMethod_ = ""; - bitField0_ = (bitField0_ & ~0x00000020); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.authoritative_ = authoritative_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.originalPrincipal_ = originalPrincipal_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.originalAuthData_ = originalAuthData_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.originalAuthMethod_ = originalAuthMethod_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance()) return this; - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasAuthoritative()) { - setAuthoritative(other.getAuthoritative()); - } - if (other.hasOriginalPrincipal()) { - setOriginalPrincipal(other.getOriginalPrincipal()); - } - if (other.hasOriginalAuthData()) { - setOriginalAuthData(other.getOriginalAuthData()); - } - if (other.hasOriginalAuthMethod()) { - setOriginalAuthMethod(other.getOriginalAuthMethod()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasTopic()) { - - return false; - } - if (!hasRequestId()) { - - 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; - topic_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - authoritative_ = input.readBool(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - originalPrincipal_ = input.readBytes(); - break; - } - case 42: { - bitField0_ |= 0x00000010; - originalAuthData_ = input.readBytes(); - break; - } - case 50: { - bitField0_ |= 0x00000020; - originalAuthMethod_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required string topic = 1; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - topic_ = value; - - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // optional bool authoritative = 3 [default = false]; - private boolean authoritative_ ; - public boolean hasAuthoritative() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public boolean getAuthoritative() { - return authoritative_; - } - public Builder setAuthoritative(boolean value) { - bitField0_ |= 0x00000004; - authoritative_ = value; - - return this; - } - public Builder clearAuthoritative() { - bitField0_ = (bitField0_ & ~0x00000004); - authoritative_ = false; - - return this; - } - - // optional string original_principal = 4; - private java.lang.Object originalPrincipal_ = ""; - public boolean hasOriginalPrincipal() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getOriginalPrincipal() { - java.lang.Object ref = originalPrincipal_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalPrincipal_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalPrincipal(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - originalPrincipal_ = value; - - return this; - } - public Builder clearOriginalPrincipal() { - bitField0_ = (bitField0_ & ~0x00000008); - originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); - - return this; - } - void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000008; - originalPrincipal_ = value; - - } - - // optional string original_auth_data = 5; - private java.lang.Object originalAuthData_ = ""; - public boolean hasOriginalAuthData() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getOriginalAuthData() { - java.lang.Object ref = originalAuthData_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthData_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthData(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - originalAuthData_ = value; - - return this; - } - public Builder clearOriginalAuthData() { - bitField0_ = (bitField0_ & ~0x00000010); - originalAuthData_ = getDefaultInstance().getOriginalAuthData(); - - return this; - } - void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - originalAuthData_ = value; - - } - - // optional string original_auth_method = 6; - private java.lang.Object originalAuthMethod_ = ""; - public boolean hasOriginalAuthMethod() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public String getOriginalAuthMethod() { - java.lang.Object ref = originalAuthMethod_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - originalAuthMethod_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setOriginalAuthMethod(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - originalAuthMethod_ = value; - - return this; - } - public Builder clearOriginalAuthMethod() { - bitField0_ = (bitField0_ & ~0x00000020); - originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); - - return this; - } - void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000020; - originalAuthMethod_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandLookupTopic) - } - - static { - defaultInstance = new CommandLookupTopic(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandLookupTopic) - } - - public interface CommandLookupTopicResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // optional string brokerServiceUrl = 1; - boolean hasBrokerServiceUrl(); - String getBrokerServiceUrl(); - - // optional string brokerServiceUrlTls = 2; - boolean hasBrokerServiceUrlTls(); - String getBrokerServiceUrlTls(); - - // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; - boolean hasResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse(); - - // required uint64 request_id = 4; - boolean hasRequestId(); - long getRequestId(); - - // optional bool authoritative = 5 [default = false]; - boolean hasAuthoritative(); - boolean getAuthoritative(); - - // optional .pulsar.proto.ServerError error = 6; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 7; - boolean hasMessage(); - String getMessage(); - - // optional bool proxy_through_service_url = 8 [default = false]; - boolean hasProxyThroughServiceUrl(); - boolean getProxyThroughServiceUrl(); - } - public static final class CommandLookupTopicResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandLookupTopicResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandLookupTopicResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandLookupTopicResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandLookupTopicResponse newObject(Handle handle) { - return new CommandLookupTopicResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandLookupTopicResponse(boolean noInit) {} - - private static final CommandLookupTopicResponse defaultInstance; - public static CommandLookupTopicResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandLookupTopicResponse getDefaultInstanceForType() { - return defaultInstance; - } - - public enum LookupType - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - Redirect(0, 0), - Connect(1, 1), - Failed(2, 2), - ; - - public static final int Redirect_VALUE = 0; - public static final int Connect_VALUE = 1; - public static final int Failed_VALUE = 2; - - - public final int getNumber() { return value; } - - public static LookupType valueOf(int value) { - switch (value) { - case 0: return Redirect; - case 1: return Connect; - case 2: return Failed; - 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 LookupType findValueByNumber(int number) { - return LookupType.valueOf(number); - } - }; - - private final int value; - - private LookupType(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandLookupTopicResponse.LookupType) - } - - private int bitField0_; - // optional string brokerServiceUrl = 1; - public static final int BROKERSERVICEURL_FIELD_NUMBER = 1; - private java.lang.Object brokerServiceUrl_; - public boolean hasBrokerServiceUrl() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getBrokerServiceUrl() { - java.lang.Object ref = brokerServiceUrl_; - 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)) { - brokerServiceUrl_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getBrokerServiceUrlBytes() { - java.lang.Object ref = brokerServiceUrl_; - 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); - brokerServiceUrl_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string brokerServiceUrlTls = 2; - public static final int BROKERSERVICEURLTLS_FIELD_NUMBER = 2; - private java.lang.Object brokerServiceUrlTls_; - public boolean hasBrokerServiceUrlTls() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getBrokerServiceUrlTls() { - java.lang.Object ref = brokerServiceUrlTls_; - 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)) { - brokerServiceUrlTls_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getBrokerServiceUrlTlsBytes() { - java.lang.Object ref = brokerServiceUrlTls_; - 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); - brokerServiceUrlTls_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; - public static final int RESPONSE_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType response_; - public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse() { - return response_; - } - - // required uint64 request_id = 4; - public static final int REQUEST_ID_FIELD_NUMBER = 4; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getRequestId() { - return requestId_; - } - - // optional bool authoritative = 5 [default = false]; - public static final int AUTHORITATIVE_FIELD_NUMBER = 5; - private boolean authoritative_; - public boolean hasAuthoritative() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getAuthoritative() { - return authoritative_; - } - - // optional .pulsar.proto.ServerError error = 6; - public static final int ERROR_FIELD_NUMBER = 6; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 7; - public static final int MESSAGE_FIELD_NUMBER = 7; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bool proxy_through_service_url = 8 [default = false]; - public static final int PROXY_THROUGH_SERVICE_URL_FIELD_NUMBER = 8; - private boolean proxyThroughServiceUrl_; - public boolean hasProxyThroughServiceUrl() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public boolean getProxyThroughServiceUrl() { - return proxyThroughServiceUrl_; - } - - private void initFields() { - brokerServiceUrl_ = ""; - brokerServiceUrlTls_ = ""; - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; - requestId_ = 0L; - authoritative_ = false; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - proxyThroughServiceUrl_ = false; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, getBrokerServiceUrlBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getBrokerServiceUrlTlsBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(3, response_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(4, requestId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBool(5, authoritative_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeEnum(6, error_.getNumber()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeBytes(7, getMessageBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeBool(8, proxyThroughServiceUrl_); - } - } - - 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, getBrokerServiceUrlBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getBrokerServiceUrlTlsBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(3, response_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, requestId_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(5, authoritative_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(6, error_.getNumber()); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(7, getMessageBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(8, proxyThroughServiceUrl_); - } - 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.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(); - brokerServiceUrl_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - brokerServiceUrlTls_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; - bitField0_ = (bitField0_ & ~0x00000004); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); - authoritative_ = false; - bitField0_ = (bitField0_ & ~0x00000010); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000020); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000040); - proxyThroughServiceUrl_ = false; - bitField0_ = (bitField0_ & ~0x00000080); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.brokerServiceUrl_ = brokerServiceUrl_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.brokerServiceUrlTls_ = brokerServiceUrlTls_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.response_ = response_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.authoritative_ = authoritative_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.message_ = message_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.proxyThroughServiceUrl_ = proxyThroughServiceUrl_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance()) return this; - if (other.hasBrokerServiceUrl()) { - setBrokerServiceUrl(other.getBrokerServiceUrl()); - } - if (other.hasBrokerServiceUrlTls()) { - setBrokerServiceUrlTls(other.getBrokerServiceUrlTls()); - } - if (other.hasResponse()) { - setResponse(other.getResponse()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasAuthoritative()) { - setAuthoritative(other.getAuthoritative()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - if (other.hasProxyThroughServiceUrl()) { - setProxyThroughServiceUrl(other.getProxyThroughServiceUrl()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - brokerServiceUrl_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - brokerServiceUrlTls_ = input.readBytes(); - break; - } - case 24: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - response_ = value; - } - break; - } - case 32: { - bitField0_ |= 0x00000008; - requestId_ = input.readUInt64(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - authoritative_ = input.readBool(); - break; - } - case 48: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000020; - error_ = value; - } - break; - } - case 58: { - bitField0_ |= 0x00000040; - message_ = input.readBytes(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - proxyThroughServiceUrl_ = input.readBool(); - break; - } - } - } - } - - private int bitField0_; - - // optional string brokerServiceUrl = 1; - private java.lang.Object brokerServiceUrl_ = ""; - public boolean hasBrokerServiceUrl() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getBrokerServiceUrl() { - java.lang.Object ref = brokerServiceUrl_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - brokerServiceUrl_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setBrokerServiceUrl(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - brokerServiceUrl_ = value; - - return this; - } - public Builder clearBrokerServiceUrl() { - bitField0_ = (bitField0_ & ~0x00000001); - brokerServiceUrl_ = getDefaultInstance().getBrokerServiceUrl(); - - return this; - } - void setBrokerServiceUrl(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - brokerServiceUrl_ = value; - - } - - // optional string brokerServiceUrlTls = 2; - private java.lang.Object brokerServiceUrlTls_ = ""; - public boolean hasBrokerServiceUrlTls() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getBrokerServiceUrlTls() { - java.lang.Object ref = brokerServiceUrlTls_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - brokerServiceUrlTls_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setBrokerServiceUrlTls(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - brokerServiceUrlTls_ = value; - - return this; - } - public Builder clearBrokerServiceUrlTls() { - bitField0_ = (bitField0_ & ~0x00000002); - brokerServiceUrlTls_ = getDefaultInstance().getBrokerServiceUrlTls(); - - return this; - } - void setBrokerServiceUrlTls(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - brokerServiceUrlTls_ = value; - - } - - // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; - public boolean hasResponse() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse() { - return response_; - } - public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - response_ = value; - - return this; - } - public Builder clearResponse() { - bitField0_ = (bitField0_ & ~0x00000004); - response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; - - return this; - } - - // required uint64 request_id = 4; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000008; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000008); - requestId_ = 0L; - - return this; - } - - // optional bool authoritative = 5 [default = false]; - private boolean authoritative_ ; - public boolean hasAuthoritative() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getAuthoritative() { - return authoritative_; - } - public Builder setAuthoritative(boolean value) { - bitField0_ |= 0x00000010; - authoritative_ = value; - - return this; - } - public Builder clearAuthoritative() { - bitField0_ = (bitField0_ & ~0x00000010); - authoritative_ = false; - - return this; - } - - // optional .pulsar.proto.ServerError error = 6; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000020; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000020); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 7; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000040; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000040); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000040; - message_ = value; - - } - - // optional bool proxy_through_service_url = 8 [default = false]; - private boolean proxyThroughServiceUrl_ ; - public boolean hasProxyThroughServiceUrl() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public boolean getProxyThroughServiceUrl() { - return proxyThroughServiceUrl_; - } - public Builder setProxyThroughServiceUrl(boolean value) { - bitField0_ |= 0x00000080; - proxyThroughServiceUrl_ = value; - - return this; - } - public Builder clearProxyThroughServiceUrl() { - bitField0_ = (bitField0_ & ~0x00000080); - proxyThroughServiceUrl_ = false; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandLookupTopicResponse) - } - - static { - defaultInstance = new CommandLookupTopicResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandLookupTopicResponse) - } - - public interface CommandProducerOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string topic = 1; - boolean hasTopic(); - String getTopic(); - - // required uint64 producer_id = 2; - boolean hasProducerId(); - long getProducerId(); - - // required uint64 request_id = 3; - boolean hasRequestId(); - long getRequestId(); - - // optional string producer_name = 4; - boolean hasProducerName(); - String getProducerName(); - - // optional bool encrypted = 5 [default = false]; - boolean hasEncrypted(); - boolean getEncrypted(); - - // repeated .pulsar.proto.KeyValue metadata = 6; - java.util.List - getMetadataList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); - int getMetadataCount(); - - // optional .pulsar.proto.Schema schema = 7; - boolean hasSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); - - // optional uint64 epoch = 8 [default = 0]; - boolean hasEpoch(); - long getEpoch(); - - // optional bool user_provided_producer_name = 9 [default = true]; - boolean hasUserProvidedProducerName(); - boolean getUserProvidedProducerName(); - } - public static final class CommandProducer extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandProducer.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandProducer(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandProducer newObject(Handle handle) { - return new CommandProducer(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandProducer(boolean noInit) {} - - private static final CommandProducer defaultInstance; - public static CommandProducer getDefaultInstance() { - return defaultInstance; - } - - public CommandProducer getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string topic = 1; - public static final int TOPIC_FIELD_NUMBER = 1; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required uint64 producer_id = 2; - public static final int PRODUCER_ID_FIELD_NUMBER = 2; - private long producerId_; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getProducerId() { - return producerId_; - } - - // required uint64 request_id = 3; - public static final int REQUEST_ID_FIELD_NUMBER = 3; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getRequestId() { - return requestId_; - } - - // optional string producer_name = 4; - public static final int PRODUCER_NAME_FIELD_NUMBER = 4; - private java.lang.Object producerName_; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - 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)) { - producerName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { - java.lang.Object ref = producerName_; - 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); - producerName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bool encrypted = 5 [default = false]; - public static final int ENCRYPTED_FIELD_NUMBER = 5; - private boolean encrypted_; - public boolean hasEncrypted() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getEncrypted() { - return encrypted_; - } - - // repeated .pulsar.proto.KeyValue metadata = 6; - public static final int METADATA_FIELD_NUMBER = 6; - private java.util.List metadata_; - public java.util.List getMetadataList() { - return metadata_; - } - public java.util.List - getMetadataOrBuilderList() { - return metadata_; - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( - int index) { - return metadata_.get(index); - } - - // optional .pulsar.proto.Schema schema = 7; - public static final int SCHEMA_FIELD_NUMBER = 7; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; - public boolean hasSchema() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - - // optional uint64 epoch = 8 [default = 0]; - public static final int EPOCH_FIELD_NUMBER = 8; - private long epoch_; - public boolean hasEpoch() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public long getEpoch() { - return epoch_; - } - - // optional bool user_provided_producer_name = 9 [default = true]; - public static final int USER_PROVIDED_PRODUCER_NAME_FIELD_NUMBER = 9; - private boolean userProvidedProducerName_; - public boolean hasUserProvidedProducerName() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public boolean getUserProvidedProducerName() { - return userProvidedProducerName_; - } - - private void initFields() { - topic_ = ""; - producerId_ = 0L; - requestId_ = 0L; - producerName_ = ""; - encrypted_ = false; - metadata_ = java.util.Collections.emptyList(); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - epoch_ = 0L; - userProvidedProducerName_ = true; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasProducerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSchema()) { - if (!getSchema().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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, producerId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, requestId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBool(5, encrypted_); - } - for (int i = 0; i < metadata_.size(); i++) { - output.writeMessage(6, metadata_.get(i)); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeMessage(7, schema_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeUInt64(8, epoch_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeBool(9, userProvidedProducerName_); - } - } - - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, producerId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, requestId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(5, encrypted_); - } - for (int i = 0; i < metadata_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(6, metadata_.get(i)); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(7, schema_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(8, epoch_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(9, userProvidedProducerName_); - } - 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.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(); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - producerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - producerName_ = ""; - bitField0_ = (bitField0_ & ~0x00000008); - encrypted_ = false; - bitField0_ = (bitField0_ & ~0x00000010); - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000020); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000040); - epoch_ = 0L; - bitField0_ = (bitField0_ & ~0x00000080); - userProvidedProducerName_ = true; - bitField0_ = (bitField0_ & ~0x00000100); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.producerId_ = producerId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.producerName_ = producerName_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.encrypted_ = encrypted_; - if (((bitField0_ & 0x00000020) == 0x00000020)) { - metadata_ = java.util.Collections.unmodifiableList(metadata_); - bitField0_ = (bitField0_ & ~0x00000020); - } - result.metadata_ = metadata_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000020; - } - result.schema_ = schema_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000040; - } - result.epoch_ = epoch_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000080; - } - result.userProvidedProducerName_ = userProvidedProducerName_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance()) return this; - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasProducerId()) { - setProducerId(other.getProducerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasProducerName()) { - setProducerName(other.getProducerName()); - } - if (other.hasEncrypted()) { - setEncrypted(other.getEncrypted()); - } - if (!other.metadata_.isEmpty()) { - if (metadata_.isEmpty()) { - metadata_ = other.metadata_; - bitField0_ = (bitField0_ & ~0x00000020); - } else { - ensureMetadataIsMutable(); - metadata_.addAll(other.metadata_); - } - - } - if (other.hasSchema()) { - mergeSchema(other.getSchema()); - } - if (other.hasEpoch()) { - setEpoch(other.getEpoch()); - } - if (other.hasUserProvidedProducerName()) { - setUserProvidedProducerName(other.getUserProvidedProducerName()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasTopic()) { - - return false; - } - if (!hasProducerId()) { - - return false; - } - if (!hasRequestId()) { - - return false; - } - for (int i = 0; i < getMetadataCount(); i++) { - if (!getMetadata(i).isInitialized()) { - - return false; - } - } - if (hasSchema()) { - if (!getSchema().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; - topic_ = input.readBytes(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - producerId_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - requestId_ = input.readUInt64(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - producerName_ = input.readBytes(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - encrypted_ = input.readBool(); - break; - } - case 50: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMetadata(subBuilder.buildPartial()); - break; - } - case 58: { - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); - if (hasSchema()) { - subBuilder.mergeFrom(getSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - epoch_ = input.readUInt64(); - break; - } - case 72: { - bitField0_ |= 0x00000100; - userProvidedProducerName_ = input.readBool(); - break; - } - } - } - } - - private int bitField0_; - - // required string topic = 1; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - topic_ = value; - - } - - // required uint64 producer_id = 2; - private long producerId_ ; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getProducerId() { - return producerId_; - } - public Builder setProducerId(long value) { - bitField0_ |= 0x00000002; - producerId_ = value; - - return this; - } - public Builder clearProducerId() { - bitField0_ = (bitField0_ & ~0x00000002); - producerId_ = 0L; - - return this; - } - - // required uint64 request_id = 3; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000004; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000004); - requestId_ = 0L; - - return this; - } - - // optional string producer_name = 4; - private java.lang.Object producerName_ = ""; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - producerName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setProducerName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - producerName_ = value; - - return this; - } - public Builder clearProducerName() { - bitField0_ = (bitField0_ & ~0x00000008); - producerName_ = getDefaultInstance().getProducerName(); - - return this; - } - void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000008; - producerName_ = value; - - } - - // optional bool encrypted = 5 [default = false]; - private boolean encrypted_ ; - public boolean hasEncrypted() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public boolean getEncrypted() { - return encrypted_; - } - public Builder setEncrypted(boolean value) { - bitField0_ |= 0x00000010; - encrypted_ = value; - - return this; - } - public Builder clearEncrypted() { - bitField0_ = (bitField0_ & ~0x00000010); - encrypted_ = false; - - return this; - } - - // repeated .pulsar.proto.KeyValue metadata = 6; - private java.util.List metadata_ = - java.util.Collections.emptyList(); - private void ensureMetadataIsMutable() { - if (!((bitField0_ & 0x00000020) == 0x00000020)) { - metadata_ = new java.util.ArrayList(metadata_); - bitField0_ |= 0x00000020; - } - } - - public java.util.List getMetadataList() { - return java.util.Collections.unmodifiableList(metadata_); - } - public int getMetadataCount() { - return metadata_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { - return metadata_.get(index); - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.set(index, value); - - return this; - } - public Builder setMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.set(index, builderForValue.build()); - - return this; - } - public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(value); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMetadataIsMutable(); - metadata_.add(index, value); - - return this; - } - public Builder addMetadata( - org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(builderForValue.build()); - - return this; - } - public Builder addMetadata( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { - ensureMetadataIsMutable(); - metadata_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllMetadata( - java.lang.Iterable values) { - ensureMetadataIsMutable(); - super.addAll(values, metadata_); - - return this; - } - public Builder clearMetadata() { - metadata_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000020); - - return this; - } - public Builder removeMetadata(int index) { - ensureMetadataIsMutable(); - metadata_.remove(index); - - return this; - } - - // optional .pulsar.proto.Schema schema = 7; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - public boolean hasSchema() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (value == null) { - throw new NullPointerException(); - } - schema_ = value; - - bitField0_ |= 0x00000040; - return this; - } - public Builder setSchema( - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { - schema_ = builderForValue.build(); - - bitField0_ |= 0x00000040; - return this; - } - public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (((bitField0_ & 0x00000040) == 0x00000040) && - schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { - schema_ = - org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); - } else { - schema_ = value; - } - - bitField0_ |= 0x00000040; - return this; - } - public Builder clearSchema() { - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000040); - return this; - } - - // optional uint64 epoch = 8 [default = 0]; - private long epoch_ ; - public boolean hasEpoch() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public long getEpoch() { - return epoch_; - } - public Builder setEpoch(long value) { - bitField0_ |= 0x00000080; - epoch_ = value; - - return this; - } - public Builder clearEpoch() { - bitField0_ = (bitField0_ & ~0x00000080); - epoch_ = 0L; - - return this; - } - - // optional bool user_provided_producer_name = 9 [default = true]; - private boolean userProvidedProducerName_ = true; - public boolean hasUserProvidedProducerName() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public boolean getUserProvidedProducerName() { - return userProvidedProducerName_; - } - public Builder setUserProvidedProducerName(boolean value) { - bitField0_ |= 0x00000100; - userProvidedProducerName_ = value; - - return this; - } - public Builder clearUserProvidedProducerName() { - bitField0_ = (bitField0_ & ~0x00000100); - userProvidedProducerName_ = true; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandProducer) - } - - static { - defaultInstance = new CommandProducer(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandProducer) - } - - public interface CommandSendOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 producer_id = 1; - boolean hasProducerId(); - long getProducerId(); - - // required uint64 sequence_id = 2; - boolean hasSequenceId(); - long getSequenceId(); - - // optional int32 num_messages = 3 [default = 1]; - boolean hasNumMessages(); - int getNumMessages(); - - // optional uint64 txnid_least_bits = 4 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 5 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional uint64 highest_sequence_id = 6 [default = 0]; - boolean hasHighestSequenceId(); - long getHighestSequenceId(); - } - public static final class CommandSend extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSendOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSend.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSend(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSend newObject(Handle handle) { - return new CommandSend(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSend(boolean noInit) {} - - private static final CommandSend defaultInstance; - public static CommandSend getDefaultInstance() { - return defaultInstance; - } - - public CommandSend getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 producer_id = 1; - public static final int PRODUCER_ID_FIELD_NUMBER = 1; - private long producerId_; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - - // required uint64 sequence_id = 2; - public static final int SEQUENCE_ID_FIELD_NUMBER = 2; - private long sequenceId_; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - - // optional int32 num_messages = 3 [default = 1]; - public static final int NUM_MESSAGES_FIELD_NUMBER = 3; - private int numMessages_; - public boolean hasNumMessages() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getNumMessages() { - return numMessages_; - } - - // optional uint64 txnid_least_bits = 4 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 4; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 5 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 5; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional uint64 highest_sequence_id = 6 [default = 0]; - public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 6; - private long highestSequenceId_; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - - private void initFields() { - producerId_ = 0L; - sequenceId_ = 0L; - numMessages_ = 1; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - highestSequenceId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasProducerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSequenceId()) { - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt32(3, numMessages_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(4, txnidLeastBits_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeUInt64(5, txnidMostBits_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeUInt64(6, highestSequenceId_); - } - } - - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt32Size(3, numMessages_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, txnidLeastBits_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(5, txnidMostBits_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(6, highestSequenceId_); - } - 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.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(); - producerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - sequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - numMessages_ = 1; - bitField0_ = (bitField0_ & ~0x00000004); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000010); - highestSequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000020); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.producerId_ = producerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.sequenceId_ = sequenceId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.numMessages_ = numMessages_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.highestSequenceId_ = highestSequenceId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance()) return this; - if (other.hasProducerId()) { - setProducerId(other.getProducerId()); - } - if (other.hasSequenceId()) { - setSequenceId(other.getSequenceId()); - } - if (other.hasNumMessages()) { - setNumMessages(other.getNumMessages()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasHighestSequenceId()) { - setHighestSequenceId(other.getHighestSequenceId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasProducerId()) { - - return false; - } - if (!hasSequenceId()) { - - 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; - producerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - sequenceId_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - numMessages_ = input.readInt32(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 40: { - bitField0_ |= 0x00000010; - txnidMostBits_ = input.readUInt64(); - break; - } - case 48: { - bitField0_ |= 0x00000020; - highestSequenceId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 producer_id = 1; - private long producerId_ ; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - public Builder setProducerId(long value) { - bitField0_ |= 0x00000001; - producerId_ = value; - - return this; - } - public Builder clearProducerId() { - bitField0_ = (bitField0_ & ~0x00000001); - producerId_ = 0L; - - return this; - } - - // required uint64 sequence_id = 2; - private long sequenceId_ ; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - public Builder setSequenceId(long value) { - bitField0_ |= 0x00000002; - sequenceId_ = value; - - return this; - } - public Builder clearSequenceId() { - bitField0_ = (bitField0_ & ~0x00000002); - sequenceId_ = 0L; - - return this; - } - - // optional int32 num_messages = 3 [default = 1]; - private int numMessages_ = 1; - public boolean hasNumMessages() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getNumMessages() { - return numMessages_; - } - public Builder setNumMessages(int value) { - bitField0_ |= 0x00000004; - numMessages_ = value; - - return this; - } - public Builder clearNumMessages() { - bitField0_ = (bitField0_ & ~0x00000004); - numMessages_ = 1; - - return this; - } - - // optional uint64 txnid_least_bits = 4 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000008; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000008); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 5 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000010; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000010); - txnidMostBits_ = 0L; - - return this; - } - - // optional uint64 highest_sequence_id = 6 [default = 0]; - private long highestSequenceId_ ; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - public Builder setHighestSequenceId(long value) { - bitField0_ |= 0x00000020; - highestSequenceId_ = value; - - return this; - } - public Builder clearHighestSequenceId() { - bitField0_ = (bitField0_ & ~0x00000020); - highestSequenceId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSend) - } - - static { - defaultInstance = new CommandSend(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSend) - } - - public interface CommandSendReceiptOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 producer_id = 1; - boolean hasProducerId(); - long getProducerId(); - - // required uint64 sequence_id = 2; - boolean hasSequenceId(); - long getSequenceId(); - - // optional .pulsar.proto.MessageIdData message_id = 3; - boolean hasMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); - - // optional uint64 highest_sequence_id = 4 [default = 0]; - boolean hasHighestSequenceId(); - long getHighestSequenceId(); - } - public static final class CommandSendReceipt extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSendReceiptOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSendReceipt.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSendReceipt(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSendReceipt newObject(Handle handle) { - return new CommandSendReceipt(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSendReceipt(boolean noInit) {} - - private static final CommandSendReceipt defaultInstance; - public static CommandSendReceipt getDefaultInstance() { - return defaultInstance; - } - - public CommandSendReceipt getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 producer_id = 1; - public static final int PRODUCER_ID_FIELD_NUMBER = 1; - private long producerId_; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - - // required uint64 sequence_id = 2; - public static final int SEQUENCE_ID_FIELD_NUMBER = 2; - private long sequenceId_; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - - // optional .pulsar.proto.MessageIdData message_id = 3; - public static final int MESSAGE_ID_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; - public boolean hasMessageId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - - // optional uint64 highest_sequence_id = 4 [default = 0]; - public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 4; - private long highestSequenceId_; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - - private void initFields() { - producerId_ = 0L; - sequenceId_ = 0L; - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - highestSequenceId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasProducerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSequenceId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasMessageId()) { - 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.writeUInt64(1, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeMessage(3, messageId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(4, highestSequenceId_); - } - } - - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, messageId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, highestSequenceId_); - } - 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceiptOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.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(); - producerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - sequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000004); - highestSequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.producerId_ = producerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.sequenceId_ = sequenceId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.messageId_ = messageId_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.highestSequenceId_ = highestSequenceId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance()) return this; - if (other.hasProducerId()) { - setProducerId(other.getProducerId()); - } - if (other.hasSequenceId()) { - setSequenceId(other.getSequenceId()); - } - if (other.hasMessageId()) { - mergeMessageId(other.getMessageId()); - } - if (other.hasHighestSequenceId()) { - setHighestSequenceId(other.getHighestSequenceId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasProducerId()) { - - return false; - } - if (!hasSequenceId()) { - - return false; - } - if (hasMessageId()) { - 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 8: { - bitField0_ |= 0x00000001; - producerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - sequenceId_ = input.readUInt64(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasMessageId()) { - subBuilder.mergeFrom(getMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - highestSequenceId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 producer_id = 1; - private long producerId_ ; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - public Builder setProducerId(long value) { - bitField0_ |= 0x00000001; - producerId_ = value; - - return this; - } - public Builder clearProducerId() { - bitField0_ = (bitField0_ & ~0x00000001); - producerId_ = 0L; - - return this; - } - - // required uint64 sequence_id = 2; - private long sequenceId_ ; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - public Builder setSequenceId(long value) { - bitField0_ |= 0x00000002; - sequenceId_ = value; - - return this; - } - public Builder clearSequenceId() { - bitField0_ = (bitField0_ & ~0x00000002); - sequenceId_ = 0L; - - return this; - } - - // optional .pulsar.proto.MessageIdData message_id = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasMessageId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - messageId_ = value; - - bitField0_ |= 0x00000004; - return this; - } - public Builder setMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - messageId_ = builderForValue.build(); - - bitField0_ |= 0x00000004; - return this; - } - public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000004) == 0x00000004) && - messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - messageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); - } else { - messageId_ = value; - } - - bitField0_ |= 0x00000004; - return this; - } - public Builder clearMessageId() { - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - // optional uint64 highest_sequence_id = 4 [default = 0]; - private long highestSequenceId_ ; - public boolean hasHighestSequenceId() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getHighestSequenceId() { - return highestSequenceId_; - } - public Builder setHighestSequenceId(long value) { - bitField0_ |= 0x00000008; - highestSequenceId_ = value; - - return this; - } - public Builder clearHighestSequenceId() { - bitField0_ = (bitField0_ & ~0x00000008); - highestSequenceId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSendReceipt) - } - - static { - defaultInstance = new CommandSendReceipt(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSendReceipt) - } - - public interface CommandSendErrorOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 producer_id = 1; - boolean hasProducerId(); - long getProducerId(); - - // required uint64 sequence_id = 2; - boolean hasSequenceId(); - long getSequenceId(); - - // required .pulsar.proto.ServerError error = 3; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // required string message = 4; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandSendError extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSendErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSendError.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSendError(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSendError newObject(Handle handle) { - return new CommandSendError(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSendError(boolean noInit) {} - - private static final CommandSendError defaultInstance; - public static CommandSendError getDefaultInstance() { - return defaultInstance; - } - - public CommandSendError getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 producer_id = 1; - public static final int PRODUCER_ID_FIELD_NUMBER = 1; - private long producerId_; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - - // required uint64 sequence_id = 2; - public static final int SEQUENCE_ID_FIELD_NUMBER = 2; - private long sequenceId_; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - - // required .pulsar.proto.ServerError error = 3; - public static final int ERROR_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // required string message = 4; - public static final int MESSAGE_FIELD_NUMBER = 4; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - producerId_ = 0L; - sequenceId_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasProducerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSequenceId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasError()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasMessage()) { - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(3, error_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getMessageBytes()); - } - } - - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, sequenceId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(3, error_.getNumber()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, getMessageBytes()); - } - 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.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(); - producerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - sequenceId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000004); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.producerId_ = producerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.sequenceId_ = sequenceId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance()) return this; - if (other.hasProducerId()) { - setProducerId(other.getProducerId()); - } - if (other.hasSequenceId()) { - setSequenceId(other.getSequenceId()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasProducerId()) { - - return false; - } - if (!hasSequenceId()) { - - return false; - } - if (!hasError()) { - - return false; - } - if (!hasMessage()) { - - 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; - producerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - sequenceId_ = input.readUInt64(); - break; - } - case 24: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - error_ = value; - } - break; - } - case 34: { - bitField0_ |= 0x00000008; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 producer_id = 1; - private long producerId_ ; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - public Builder setProducerId(long value) { - bitField0_ |= 0x00000001; - producerId_ = value; - - return this; - } - public Builder clearProducerId() { - bitField0_ = (bitField0_ & ~0x00000001); - producerId_ = 0L; - - return this; - } - - // required uint64 sequence_id = 2; - private long sequenceId_ ; - public boolean hasSequenceId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getSequenceId() { - return sequenceId_; - } - public Builder setSequenceId(long value) { - bitField0_ |= 0x00000002; - sequenceId_ = value; - - return this; - } - public Builder clearSequenceId() { - bitField0_ = (bitField0_ & ~0x00000002); - sequenceId_ = 0L; - - return this; - } - - // required .pulsar.proto.ServerError error = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // required string message = 4; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000008); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000008; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSendError) - } - - static { - defaultInstance = new CommandSendError(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSendError) - } - - public interface CommandMessageOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required .pulsar.proto.MessageIdData message_id = 2; - boolean hasMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); - - // optional uint32 redelivery_count = 3 [default = 0]; - boolean hasRedeliveryCount(); - int getRedeliveryCount(); - - // repeated .pulsar.proto.IntRange acked_indexes = 4; - java.util.List - getAckedIndexesList(); - org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index); - int getAckedIndexesCount(); - } - public static final class CommandMessage extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandMessageOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandMessage.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandMessage(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandMessage newObject(Handle handle) { - return new CommandMessage(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandMessage(boolean noInit) {} - - private static final CommandMessage defaultInstance; - public static CommandMessage getDefaultInstance() { - return defaultInstance; - } - - public CommandMessage getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required .pulsar.proto.MessageIdData message_id = 2; - public static final int MESSAGE_ID_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; - public boolean hasMessageId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - - // optional uint32 redelivery_count = 3 [default = 0]; - public static final int REDELIVERY_COUNT_FIELD_NUMBER = 3; - private int redeliveryCount_; - public boolean hasRedeliveryCount() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getRedeliveryCount() { - return redeliveryCount_; - } - - // repeated .pulsar.proto.IntRange acked_indexes = 4; - public static final int ACKED_INDEXES_FIELD_NUMBER = 4; - private java.util.List ackedIndexes_; - public java.util.List getAckedIndexesList() { - return ackedIndexes_; - } - public java.util.List - getAckedIndexesOrBuilderList() { - return ackedIndexes_; - } - public int getAckedIndexesCount() { - return ackedIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index) { - return ackedIndexes_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getAckedIndexesOrBuilder( - int index) { - return ackedIndexes_.get(index); - } - - private void initFields() { - consumerId_ = 0L; - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - redeliveryCount_ = 0; - ackedIndexes_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasMessageId()) { - memoizedIsInitialized = 0; - return false; - } - if (!getMessageId().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getAckedIndexesCount(); i++) { - if (!getAckedIndexes(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.writeUInt64(1, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, messageId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt32(3, redeliveryCount_); - } - for (int i = 0; i < ackedIndexes_.size(); i++) { - output.writeMessage(4, ackedIndexes_.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 - .computeUInt64Size(1, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, messageId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt32Size(3, redeliveryCount_); - } - for (int i = 0; i < ackedIndexes_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, ackedIndexes_.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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandMessageOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000002); - redeliveryCount_ = 0; - bitField0_ = (bitField0_ & ~0x00000004); - ackedIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.messageId_ = messageId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.redeliveryCount_ = redeliveryCount_; - if (((bitField0_ & 0x00000008) == 0x00000008)) { - ackedIndexes_ = java.util.Collections.unmodifiableList(ackedIndexes_); - bitField0_ = (bitField0_ & ~0x00000008); - } - result.ackedIndexes_ = ackedIndexes_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasMessageId()) { - mergeMessageId(other.getMessageId()); - } - if (other.hasRedeliveryCount()) { - setRedeliveryCount(other.getRedeliveryCount()); - } - if (!other.ackedIndexes_.isEmpty()) { - if (ackedIndexes_.isEmpty()) { - ackedIndexes_ = other.ackedIndexes_; - bitField0_ = (bitField0_ & ~0x00000008); - } else { - ensureAckedIndexesIsMutable(); - ackedIndexes_.addAll(other.ackedIndexes_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasMessageId()) { - - return false; - } - if (!getMessageId().isInitialized()) { - - return false; - } - for (int i = 0; i < getAckedIndexesCount(); i++) { - if (!getAckedIndexes(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 8: { - bitField0_ |= 0x00000001; - consumerId_ = input.readUInt64(); - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasMessageId()) { - subBuilder.mergeFrom(getMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - redeliveryCount_ = input.readUInt32(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addAckedIndexes(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required .pulsar.proto.MessageIdData message_id = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasMessageId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - messageId_ = value; - - bitField0_ |= 0x00000002; - return this; - } - public Builder setMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - messageId_ = builderForValue.build(); - - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - messageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); - } else { - messageId_ = value; - } - - bitField0_ |= 0x00000002; - return this; - } - public Builder clearMessageId() { - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - // optional uint32 redelivery_count = 3 [default = 0]; - private int redeliveryCount_ ; - public boolean hasRedeliveryCount() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public int getRedeliveryCount() { - return redeliveryCount_; - } - public Builder setRedeliveryCount(int value) { - bitField0_ |= 0x00000004; - redeliveryCount_ = value; - - return this; - } - public Builder clearRedeliveryCount() { - bitField0_ = (bitField0_ & ~0x00000004); - redeliveryCount_ = 0; - - return this; - } - - // repeated .pulsar.proto.IntRange acked_indexes = 4; - private java.util.List ackedIndexes_ = - java.util.Collections.emptyList(); - private void ensureAckedIndexesIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { - ackedIndexes_ = new java.util.ArrayList(ackedIndexes_); - bitField0_ |= 0x00000008; - } - } - - public java.util.List getAckedIndexesList() { - return java.util.Collections.unmodifiableList(ackedIndexes_); - } - public int getAckedIndexesCount() { - return ackedIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getAckedIndexes(int index) { - return ackedIndexes_.get(index); - } - public Builder setAckedIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckedIndexesIsMutable(); - ackedIndexes_.set(index, value); - - return this; - } - public Builder setAckedIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckedIndexesIsMutable(); - ackedIndexes_.set(index, builderForValue.build()); - - return this; - } - public Builder addAckedIndexes(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckedIndexesIsMutable(); - ackedIndexes_.add(value); - - return this; - } - public Builder addAckedIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { - if (value == null) { - throw new NullPointerException(); - } - ensureAckedIndexesIsMutable(); - ackedIndexes_.add(index, value); - - return this; - } - public Builder addAckedIndexes( - org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckedIndexesIsMutable(); - ackedIndexes_.add(builderForValue.build()); - - return this; - } - public Builder addAckedIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { - ensureAckedIndexesIsMutable(); - ackedIndexes_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllAckedIndexes( - java.lang.Iterable values) { - ensureAckedIndexesIsMutable(); - super.addAll(values, ackedIndexes_); - - return this; - } - public Builder clearAckedIndexes() { - ackedIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - - return this; - } - public Builder removeAckedIndexes(int index) { - ensureAckedIndexesIsMutable(); - ackedIndexes_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandMessage) - } - - static { - defaultInstance = new CommandMessage(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandMessage) - } - - public interface CommandAckOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required .pulsar.proto.CommandAck.AckType ack_type = 2; - boolean hasAckType(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType(); - - // repeated .pulsar.proto.MessageIdData message_id = 3; - java.util.List - getMessageIdList(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index); - int getMessageIdCount(); - - // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; - boolean hasValidationError(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError(); - - // repeated .pulsar.proto.KeyLongValue properties = 5; - java.util.List - getPropertiesList(); - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index); - int getPropertiesCount(); - - // optional uint64 txnid_least_bits = 6 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 7 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; - java.util.List - getBatchMessageAckIndexesList(); - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index); - int getBatchMessageAckIndexesCount(); - } - public static final class CommandAck extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAckOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAck.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAck(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAck newObject(Handle handle) { - return new CommandAck(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAck(boolean noInit) {} - - private static final CommandAck defaultInstance; - public static CommandAck getDefaultInstance() { - return defaultInstance; - } - - public CommandAck getDefaultInstanceForType() { - return defaultInstance; - } - - public enum AckType - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - Individual(0, 0), - Cumulative(1, 1), - ; - - public static final int Individual_VALUE = 0; - public static final int Cumulative_VALUE = 1; - - - public final int getNumber() { return value; } - - public static AckType valueOf(int value) { - switch (value) { - case 0: return Individual; - case 1: return Cumulative; - 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 AckType findValueByNumber(int number) { - return AckType.valueOf(number); - } - }; - - private final int value; - - private AckType(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandAck.AckType) - } - - public enum ValidationError - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - UncompressedSizeCorruption(0, 0), - DecompressionError(1, 1), - ChecksumMismatch(2, 2), - BatchDeSerializeError(3, 3), - DecryptionError(4, 4), - ; - - public static final int UncompressedSizeCorruption_VALUE = 0; - public static final int DecompressionError_VALUE = 1; - public static final int ChecksumMismatch_VALUE = 2; - public static final int BatchDeSerializeError_VALUE = 3; - public static final int DecryptionError_VALUE = 4; - - - public final int getNumber() { return value; } - - public static ValidationError valueOf(int value) { - switch (value) { - case 0: return UncompressedSizeCorruption; - case 1: return DecompressionError; - case 2: return ChecksumMismatch; - case 3: return BatchDeSerializeError; - case 4: return DecryptionError; - 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 ValidationError findValueByNumber(int number) { - return ValidationError.valueOf(number); - } - }; - - private final int value; - - private ValidationError(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandAck.ValidationError) - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required .pulsar.proto.CommandAck.AckType ack_type = 2; - public static final int ACK_TYPE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType ackType_; - public boolean hasAckType() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType() { - return ackType_; - } - - // repeated .pulsar.proto.MessageIdData message_id = 3; - public static final int MESSAGE_ID_FIELD_NUMBER = 3; - private java.util.List messageId_; - public java.util.List getMessageIdList() { - return messageId_; - } - public java.util.List - getMessageIdOrBuilderList() { - return messageId_; - } - public int getMessageIdCount() { - return messageId_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index) { - return messageId_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder getMessageIdOrBuilder( - int index) { - return messageId_.get(index); - } - - // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; - public static final int VALIDATION_ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError validationError_; - public boolean hasValidationError() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError() { - return validationError_; - } - - // repeated .pulsar.proto.KeyLongValue properties = 5; - public static final int PROPERTIES_FIELD_NUMBER = 5; - private java.util.List properties_; - public java.util.List getPropertiesList() { - return properties_; - } - public java.util.List - getPropertiesOrBuilderList() { - return properties_; - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index) { - return properties_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValueOrBuilder getPropertiesOrBuilder( - int index) { - return properties_.get(index); - } - - // optional uint64 txnid_least_bits = 6 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 6; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 7 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 7; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; - public static final int BATCH_MESSAGE_ACK_INDEXES_FIELD_NUMBER = 8; - private java.util.List batchMessageAckIndexes_; - public java.util.List getBatchMessageAckIndexesList() { - return batchMessageAckIndexes_; - } - public java.util.List - getBatchMessageAckIndexesOrBuilderList() { - return batchMessageAckIndexes_; - } - public int getBatchMessageAckIndexesCount() { - return batchMessageAckIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index) { - return batchMessageAckIndexes_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckDataOrBuilder getBatchMessageAckIndexesOrBuilder( - int index) { - return batchMessageAckIndexes_.get(index); - } - - private void initFields() { - consumerId_ = 0L; - ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; - messageId_ = java.util.Collections.emptyList(); - validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; - properties_ = java.util.Collections.emptyList(); - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - batchMessageAckIndexes_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasAckType()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getMessageIdCount(); i++) { - if (!getMessageId(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(i).isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - for (int i = 0; i < getBatchMessageAckIndexesCount(); i++) { - if (!getBatchMessageAckIndexes(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.writeUInt64(1, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, ackType_.getNumber()); - } - for (int i = 0; i < messageId_.size(); i++) { - output.writeMessage(3, messageId_.get(i)); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(4, validationError_.getNumber()); - } - for (int i = 0; i < properties_.size(); i++) { - output.writeMessage(5, properties_.get(i)); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(6, txnidLeastBits_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeUInt64(7, txnidMostBits_); - } - for (int i = 0; i < batchMessageAckIndexes_.size(); i++) { - output.writeMessage(8, batchMessageAckIndexes_.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 - .computeUInt64Size(1, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, ackType_.getNumber()); - } - for (int i = 0; i < messageId_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, messageId_.get(i)); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, validationError_.getNumber()); - } - for (int i = 0; i < properties_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(5, properties_.get(i)); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(6, txnidLeastBits_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(7, txnidMostBits_); - } - for (int i = 0; i < batchMessageAckIndexes_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(8, batchMessageAckIndexes_.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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAckOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; - bitField0_ = (bitField0_ & ~0x00000002); - messageId_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; - bitField0_ = (bitField0_ & ~0x00000008); - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000010); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000020); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000040); - batchMessageAckIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000080); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.ackType_ = ackType_; - if (((bitField0_ & 0x00000004) == 0x00000004)) { - messageId_ = java.util.Collections.unmodifiableList(messageId_); - bitField0_ = (bitField0_ & ~0x00000004); - } - result.messageId_ = messageId_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000004; - } - result.validationError_ = validationError_; - if (((bitField0_ & 0x00000010) == 0x00000010)) { - properties_ = java.util.Collections.unmodifiableList(properties_); - bitField0_ = (bitField0_ & ~0x00000010); - } - result.properties_ = properties_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000008; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000010; - } - result.txnidMostBits_ = txnidMostBits_; - if (((bitField0_ & 0x00000080) == 0x00000080)) { - batchMessageAckIndexes_ = java.util.Collections.unmodifiableList(batchMessageAckIndexes_); - bitField0_ = (bitField0_ & ~0x00000080); - } - result.batchMessageAckIndexes_ = batchMessageAckIndexes_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasAckType()) { - setAckType(other.getAckType()); - } - if (!other.messageId_.isEmpty()) { - if (messageId_.isEmpty()) { - messageId_ = other.messageId_; - bitField0_ = (bitField0_ & ~0x00000004); - } else { - ensureMessageIdIsMutable(); - messageId_.addAll(other.messageId_); - } - - } - if (other.hasValidationError()) { - setValidationError(other.getValidationError()); - } - if (!other.properties_.isEmpty()) { - if (properties_.isEmpty()) { - properties_ = other.properties_; - bitField0_ = (bitField0_ & ~0x00000010); - } else { - ensurePropertiesIsMutable(); - properties_.addAll(other.properties_); - } - - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (!other.batchMessageAckIndexes_.isEmpty()) { - if (batchMessageAckIndexes_.isEmpty()) { - batchMessageAckIndexes_ = other.batchMessageAckIndexes_; - bitField0_ = (bitField0_ & ~0x00000080); - } else { - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.addAll(other.batchMessageAckIndexes_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasAckType()) { - - return false; - } - for (int i = 0; i < getMessageIdCount(); i++) { - if (!getMessageId(i).isInitialized()) { - - return false; - } - } - for (int i = 0; i < getPropertiesCount(); i++) { - if (!getProperties(i).isInitialized()) { - - return false; - } - } - for (int i = 0; i < getBatchMessageAckIndexesCount(); i++) { - if (!getBatchMessageAckIndexes(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 8: { - bitField0_ |= 0x00000001; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - ackType_ = value; - } - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMessageId(subBuilder.buildPartial()); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError value = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - validationError_ = value; - } - break; - } - case 42: { - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addProperties(subBuilder.buildPartial()); - break; - } - case 48: { - bitField0_ |= 0x00000020; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 56: { - bitField0_ |= 0x00000040; - txnidMostBits_ = input.readUInt64(); - break; - } - case 66: { - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addBatchMessageAckIndexes(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required .pulsar.proto.CommandAck.AckType ack_type = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; - public boolean hasAckType() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType() { - return ackType_; - } - public Builder setAckType(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - ackType_ = value; - - return this; - } - public Builder clearAckType() { - bitField0_ = (bitField0_ & ~0x00000002); - ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; - - return this; - } - - // repeated .pulsar.proto.MessageIdData message_id = 3; - private java.util.List messageId_ = - java.util.Collections.emptyList(); - private void ensureMessageIdIsMutable() { - if (!((bitField0_ & 0x00000004) == 0x00000004)) { - messageId_ = new java.util.ArrayList(messageId_); - bitField0_ |= 0x00000004; - } - } - - public java.util.List getMessageIdList() { - return java.util.Collections.unmodifiableList(messageId_); - } - public int getMessageIdCount() { - return messageId_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index) { - return messageId_.get(index); - } - public Builder setMessageId( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdIsMutable(); - messageId_.set(index, value); - - return this; - } - public Builder setMessageId( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdIsMutable(); - messageId_.set(index, builderForValue.build()); - - return this; - } - public Builder addMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdIsMutable(); - messageId_.add(value); - - return this; - } - public Builder addMessageId( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdIsMutable(); - messageId_.add(index, value); - - return this; - } - public Builder addMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdIsMutable(); - messageId_.add(builderForValue.build()); - - return this; - } - public Builder addMessageId( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdIsMutable(); - messageId_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllMessageId( - java.lang.Iterable values) { - ensureMessageIdIsMutable(); - super.addAll(values, messageId_); - - return this; - } - public Builder clearMessageId() { - messageId_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000004); - - return this; - } - public Builder removeMessageId(int index) { - ensureMessageIdIsMutable(); - messageId_.remove(index); - - return this; - } - - // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; - public boolean hasValidationError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError() { - return validationError_; - } - public Builder setValidationError(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - validationError_ = value; - - return this; - } - public Builder clearValidationError() { - bitField0_ = (bitField0_ & ~0x00000008); - validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; - - return this; - } - - // repeated .pulsar.proto.KeyLongValue properties = 5; - private java.util.List properties_ = - java.util.Collections.emptyList(); - private void ensurePropertiesIsMutable() { - if (!((bitField0_ & 0x00000010) == 0x00000010)) { - properties_ = new java.util.ArrayList(properties_); - bitField0_ |= 0x00000010; - } - } - - public java.util.List getPropertiesList() { - return java.util.Collections.unmodifiableList(properties_); - } - public int getPropertiesCount() { - return properties_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index) { - return properties_.get(index); - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.set(index, value); - - return this; - } - public Builder setProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.set(index, builderForValue.build()); - - return this; - } - public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(value); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePropertiesIsMutable(); - properties_.add(index, value); - - return this; - } - public Builder addProperties( - org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(builderForValue.build()); - - return this; - } - public Builder addProperties( - int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { - ensurePropertiesIsMutable(); - properties_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllProperties( - java.lang.Iterable values) { - ensurePropertiesIsMutable(); - super.addAll(values, properties_); - - return this; - } - public Builder clearProperties() { - properties_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000010); - - return this; - } - public Builder removeProperties(int index) { - ensurePropertiesIsMutable(); - properties_.remove(index); - - return this; - } - - // optional uint64 txnid_least_bits = 6 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000020; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000020); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 7 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000040; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000040); - txnidMostBits_ = 0L; - - return this; - } - - // repeated .pulsar.proto.BatchMessageIndexesAckData batch_message_ack_indexes = 8; - private java.util.List batchMessageAckIndexes_ = - java.util.Collections.emptyList(); - private void ensureBatchMessageAckIndexesIsMutable() { - if (!((bitField0_ & 0x00000080) == 0x00000080)) { - batchMessageAckIndexes_ = new java.util.ArrayList(batchMessageAckIndexes_); - bitField0_ |= 0x00000080; - } - } - - public java.util.List getBatchMessageAckIndexesList() { - return java.util.Collections.unmodifiableList(batchMessageAckIndexes_); - } - public int getBatchMessageAckIndexesCount() { - return batchMessageAckIndexes_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData getBatchMessageAckIndexes(int index) { - return batchMessageAckIndexes_.get(index); - } - public Builder setBatchMessageAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.set(index, value); - - return this; - } - public Builder setBatchMessageAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.set(index, builderForValue.build()); - - return this; - } - public Builder addBatchMessageAckIndexes(org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.add(value); - - return this; - } - public Builder addBatchMessageAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.add(index, value); - - return this; - } - public Builder addBatchMessageAckIndexes( - org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.add(builderForValue.build()); - - return this; - } - public Builder addBatchMessageAckIndexes( - int index, org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData.Builder builderForValue) { - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllBatchMessageAckIndexes( - java.lang.Iterable values) { - ensureBatchMessageAckIndexesIsMutable(); - super.addAll(values, batchMessageAckIndexes_); - - return this; - } - public Builder clearBatchMessageAckIndexes() { - batchMessageAckIndexes_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000080); - - return this; - } - public Builder removeBatchMessageAckIndexes(int index) { - ensureBatchMessageAckIndexesIsMutable(); - batchMessageAckIndexes_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAck) - } - - static { - defaultInstance = new CommandAck(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAck) - } - - public interface CommandAckResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandAckResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAckResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAckResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAckResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAckResponse newObject(Handle handle) { - return new CommandAckResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAckResponse(boolean noInit) {} - - private static final CommandAckResponse defaultInstance; - public static CommandAckResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandAckResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - consumerId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAckResponse) - } - - static { - defaultInstance = new CommandAckResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAckResponse) - } - - public interface CommandActiveConsumerChangeOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // optional bool is_active = 2 [default = false]; - boolean hasIsActive(); - boolean getIsActive(); - } - public static final class CommandActiveConsumerChange extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandActiveConsumerChangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandActiveConsumerChange.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandActiveConsumerChange(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandActiveConsumerChange newObject(Handle handle) { - return new CommandActiveConsumerChange(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandActiveConsumerChange(boolean noInit) {} - - private static final CommandActiveConsumerChange defaultInstance; - public static CommandActiveConsumerChange getDefaultInstance() { - return defaultInstance; - } - - public CommandActiveConsumerChange getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // optional bool is_active = 2 [default = false]; - public static final int IS_ACTIVE_FIELD_NUMBER = 2; - private boolean isActive_; - public boolean hasIsActive() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public boolean getIsActive() { - return isActive_; - } - - private void initFields() { - consumerId_ = 0L; - isActive_ = false; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBool(2, isActive_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(2, isActive_); - } - 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - isActive_ = false; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.isActive_ = isActive_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasIsActive()) { - setIsActive(other.getIsActive()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - isActive_ = input.readBool(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // optional bool is_active = 2 [default = false]; - private boolean isActive_ ; - public boolean hasIsActive() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public boolean getIsActive() { - return isActive_; - } - public Builder setIsActive(boolean value) { - bitField0_ |= 0x00000002; - isActive_ = value; - - return this; - } - public Builder clearIsActive() { - bitField0_ = (bitField0_ & ~0x00000002); - isActive_ = false; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandActiveConsumerChange) - } - - static { - defaultInstance = new CommandActiveConsumerChange(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandActiveConsumerChange) - } - - public interface CommandFlowOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint32 messagePermits = 2; - boolean hasMessagePermits(); - int getMessagePermits(); - } - public static final class CommandFlow extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandFlowOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandFlow.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandFlow(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandFlow newObject(Handle handle) { - return new CommandFlow(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandFlow(boolean noInit) {} - - private static final CommandFlow defaultInstance; - public static CommandFlow getDefaultInstance() { - return defaultInstance; - } - - public CommandFlow getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint32 messagePermits = 2; - public static final int MESSAGEPERMITS_FIELD_NUMBER = 2; - private int messagePermits_; - public boolean hasMessagePermits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getMessagePermits() { - return messagePermits_; - } - - private void initFields() { - consumerId_ = 0L; - messagePermits_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasMessagePermits()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt32(2, messagePermits_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt32Size(2, messagePermits_); - } - 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandFlowOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - messagePermits_ = 0; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.messagePermits_ = messagePermits_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasMessagePermits()) { - setMessagePermits(other.getMessagePermits()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasMessagePermits()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - messagePermits_ = input.readUInt32(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required uint32 messagePermits = 2; - private int messagePermits_ ; - public boolean hasMessagePermits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public int getMessagePermits() { - return messagePermits_; - } - public Builder setMessagePermits(int value) { - bitField0_ |= 0x00000002; - messagePermits_ = value; - - return this; - } - public Builder clearMessagePermits() { - bitField0_ = (bitField0_ & ~0x00000002); - messagePermits_ = 0; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandFlow) - } - - static { - defaultInstance = new CommandFlow(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandFlow) - } - - public interface CommandUnsubscribeOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - } - public static final class CommandUnsubscribe extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandUnsubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandUnsubscribe.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandUnsubscribe(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandUnsubscribe newObject(Handle handle) { - return new CommandUnsubscribe(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandUnsubscribe(boolean noInit) {} - - private static final CommandUnsubscribe defaultInstance; - public static CommandUnsubscribe getDefaultInstance() { - return defaultInstance; - } - - public CommandUnsubscribe getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - private void initFields() { - consumerId_ = 0L; - requestId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasRequestId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandUnsubscribe) - } - - static { - defaultInstance = new CommandUnsubscribe(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandUnsubscribe) - } - - public interface CommandSeekOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.MessageIdData message_id = 3; - boolean hasMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); - - // optional uint64 message_publish_time = 4; - boolean hasMessagePublishTime(); - long getMessagePublishTime(); - } - public static final class CommandSeek extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSeekOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSeek.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSeek(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSeek newObject(Handle handle) { - return new CommandSeek(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSeek(boolean noInit) {} - - private static final CommandSeek defaultInstance; - public static CommandSeek getDefaultInstance() { - return defaultInstance; - } - - public CommandSeek getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.MessageIdData message_id = 3; - public static final int MESSAGE_ID_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; - public boolean hasMessageId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - - // optional uint64 message_publish_time = 4; - public static final int MESSAGE_PUBLISH_TIME_FIELD_NUMBER = 4; - private long messagePublishTime_; - public boolean hasMessagePublishTime() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getMessagePublishTime() { - return messagePublishTime_; - } - - private void initFields() { - consumerId_ = 0L; - requestId_ = 0L; - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - messagePublishTime_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasMessageId()) { - 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.writeUInt64(1, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeMessage(3, messageId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt64(4, messagePublishTime_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, messageId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, messagePublishTime_); - } - 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSeekOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000004); - messagePublishTime_ = 0L; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.messageId_ = messageId_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.messagePublishTime_ = messagePublishTime_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasMessageId()) { - mergeMessageId(other.getMessageId()); - } - if (other.hasMessagePublishTime()) { - setMessagePublishTime(other.getMessagePublishTime()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasRequestId()) { - - return false; - } - if (hasMessageId()) { - 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 8: { - bitField0_ |= 0x00000001; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasMessageId()) { - subBuilder.mergeFrom(getMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - messagePublishTime_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.MessageIdData message_id = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasMessageId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { - return messageId_; - } - public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - messageId_ = value; - - bitField0_ |= 0x00000004; - return this; - } - public Builder setMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - messageId_ = builderForValue.build(); - - bitField0_ |= 0x00000004; - return this; - } - public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000004) == 0x00000004) && - messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - messageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); - } else { - messageId_ = value; - } - - bitField0_ |= 0x00000004; - return this; - } - public Builder clearMessageId() { - messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - // optional uint64 message_publish_time = 4; - private long messagePublishTime_ ; - public boolean hasMessagePublishTime() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public long getMessagePublishTime() { - return messagePublishTime_; - } - public Builder setMessagePublishTime(long value) { - bitField0_ |= 0x00000008; - messagePublishTime_ = value; - - return this; - } - public Builder clearMessagePublishTime() { - bitField0_ = (bitField0_ & ~0x00000008); - messagePublishTime_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSeek) - } - - static { - defaultInstance = new CommandSeek(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSeek) - } - - public interface CommandReachedEndOfTopicOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - } - public static final class CommandReachedEndOfTopic extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandReachedEndOfTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandReachedEndOfTopic.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandReachedEndOfTopic(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandReachedEndOfTopic newObject(Handle handle) { - return new CommandReachedEndOfTopic(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandReachedEndOfTopic(boolean noInit) {} - - private static final CommandReachedEndOfTopic defaultInstance; - public static CommandReachedEndOfTopic getDefaultInstance() { - return defaultInstance; - } - - public CommandReachedEndOfTopic getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - private void initFields() { - consumerId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - 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, consumerId_); - } - } - - 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, consumerId_); - } - 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandReachedEndOfTopic) - } - - static { - defaultInstance = new CommandReachedEndOfTopic(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandReachedEndOfTopic) - } - - public interface CommandCloseProducerOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 producer_id = 1; - boolean hasProducerId(); - long getProducerId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - } - public static final class CommandCloseProducer extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandCloseProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandCloseProducer.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandCloseProducer(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandCloseProducer newObject(Handle handle) { - return new CommandCloseProducer(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandCloseProducer(boolean noInit) {} - - private static final CommandCloseProducer defaultInstance; - public static CommandCloseProducer getDefaultInstance() { - return defaultInstance; - } - - public CommandCloseProducer getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 producer_id = 1; - public static final int PRODUCER_ID_FIELD_NUMBER = 1; - private long producerId_; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - private void initFields() { - producerId_ = 0L; - requestId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasProducerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - } - - 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, producerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.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(); - producerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.producerId_ = producerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance()) return this; - if (other.hasProducerId()) { - setProducerId(other.getProducerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasProducerId()) { - - return false; - } - if (!hasRequestId()) { - - 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; - producerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 producer_id = 1; - private long producerId_ ; - public boolean hasProducerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getProducerId() { - return producerId_; - } - public Builder setProducerId(long value) { - bitField0_ |= 0x00000001; - producerId_ = value; - - return this; - } - public Builder clearProducerId() { - bitField0_ = (bitField0_ & ~0x00000001); - producerId_ = 0L; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandCloseProducer) - } - - static { - defaultInstance = new CommandCloseProducer(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandCloseProducer) - } - - public interface CommandCloseConsumerOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - } - public static final class CommandCloseConsumer extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandCloseConsumerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandCloseConsumer.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandCloseConsumer(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandCloseConsumer newObject(Handle handle) { - return new CommandCloseConsumer(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandCloseConsumer(boolean noInit) {} - - private static final CommandCloseConsumer defaultInstance; - public static CommandCloseConsumer getDefaultInstance() { - return defaultInstance; - } - - public CommandCloseConsumer getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - private void initFields() { - consumerId_ = 0L; - requestId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasRequestId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandCloseConsumer) - } - - static { - defaultInstance = new CommandCloseConsumer(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandCloseConsumer) - } - - public interface CommandRedeliverUnacknowledgedMessagesOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // repeated .pulsar.proto.MessageIdData message_ids = 2; - java.util.List - getMessageIdsList(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index); - int getMessageIdsCount(); - } - public static final class CommandRedeliverUnacknowledgedMessages extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandRedeliverUnacknowledgedMessagesOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandRedeliverUnacknowledgedMessages.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandRedeliverUnacknowledgedMessages(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandRedeliverUnacknowledgedMessages newObject(Handle handle) { - return new CommandRedeliverUnacknowledgedMessages(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandRedeliverUnacknowledgedMessages(boolean noInit) {} - - private static final CommandRedeliverUnacknowledgedMessages defaultInstance; - public static CommandRedeliverUnacknowledgedMessages getDefaultInstance() { - return defaultInstance; - } - - public CommandRedeliverUnacknowledgedMessages getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // repeated .pulsar.proto.MessageIdData message_ids = 2; - public static final int MESSAGE_IDS_FIELD_NUMBER = 2; - private java.util.List messageIds_; - public java.util.List getMessageIdsList() { - return messageIds_; - } - public java.util.List - getMessageIdsOrBuilderList() { - return messageIds_; - } - public int getMessageIdsCount() { - return messageIds_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index) { - return messageIds_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder getMessageIdsOrBuilder( - int index) { - return messageIds_.get(index); - } - - private void initFields() { - consumerId_ = 0L; - messageIds_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getMessageIdsCount(); i++) { - if (!getMessageIds(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.writeUInt64(1, consumerId_); - } - for (int i = 0; i < messageIds_.size(); i++) { - output.writeMessage(2, messageIds_.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 - .computeUInt64Size(1, consumerId_); - } - for (int i = 0; i < messageIds_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, messageIds_.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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessagesOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - messageIds_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((bitField0_ & 0x00000002) == 0x00000002)) { - messageIds_ = java.util.Collections.unmodifiableList(messageIds_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.messageIds_ = messageIds_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (!other.messageIds_.isEmpty()) { - if (messageIds_.isEmpty()) { - messageIds_ = other.messageIds_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureMessageIdsIsMutable(); - messageIds_.addAll(other.messageIds_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - for (int i = 0; i < getMessageIdsCount(); i++) { - if (!getMessageIds(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 8: { - bitField0_ |= 0x00000001; - consumerId_ = input.readUInt64(); - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addMessageIds(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // repeated .pulsar.proto.MessageIdData message_ids = 2; - private java.util.List messageIds_ = - java.util.Collections.emptyList(); - private void ensureMessageIdsIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - messageIds_ = new java.util.ArrayList(messageIds_); - bitField0_ |= 0x00000002; - } - } - - public java.util.List getMessageIdsList() { - return java.util.Collections.unmodifiableList(messageIds_); - } - public int getMessageIdsCount() { - return messageIds_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index) { - return messageIds_.get(index); - } - public Builder setMessageIds( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdsIsMutable(); - messageIds_.set(index, value); - - return this; - } - public Builder setMessageIds( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdsIsMutable(); - messageIds_.set(index, builderForValue.build()); - - return this; - } - public Builder addMessageIds(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdsIsMutable(); - messageIds_.add(value); - - return this; - } - public Builder addMessageIds( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - ensureMessageIdsIsMutable(); - messageIds_.add(index, value); - - return this; - } - public Builder addMessageIds( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdsIsMutable(); - messageIds_.add(builderForValue.build()); - - return this; - } - public Builder addMessageIds( - int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - ensureMessageIdsIsMutable(); - messageIds_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllMessageIds( - java.lang.Iterable values) { - ensureMessageIdsIsMutable(); - super.addAll(values, messageIds_); - - return this; - } - public Builder clearMessageIds() { - messageIds_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000002); - - return this; - } - public Builder removeMessageIds(int index) { - ensureMessageIdsIsMutable(); - messageIds_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandRedeliverUnacknowledgedMessages) - } - - static { - defaultInstance = new CommandRedeliverUnacknowledgedMessages(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandRedeliverUnacknowledgedMessages) - } - - public interface CommandSuccessOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.Schema schema = 2; - boolean hasSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); - } - public static final class CommandSuccess extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandSuccess.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandSuccess(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandSuccess newObject(Handle handle) { - return new CommandSuccess(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandSuccess(boolean noInit) {} - - private static final CommandSuccess defaultInstance; - public static CommandSuccess getDefaultInstance() { - return defaultInstance; - } - - public CommandSuccess getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.Schema schema = 2; - public static final int SCHEMA_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; - public boolean hasSchema() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - - private void initFields() { - requestId_ = 0L; - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasSchema()) { - if (!getSchema().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.writeUInt64(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, schema_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, schema_); - } - 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.schema_ = schema_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasSchema()) { - mergeSchema(other.getSchema()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (hasSchema()) { - if (!getSchema().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 8: { - bitField0_ |= 0x00000001; - requestId_ = input.readUInt64(); - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); - if (hasSchema()) { - subBuilder.mergeFrom(getSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.Schema schema = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - public boolean hasSchema() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (value == null) { - throw new NullPointerException(); - } - schema_ = value; - - bitField0_ |= 0x00000002; - return this; - } - public Builder setSchema( - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { - schema_ = builderForValue.build(); - - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { - schema_ = - org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); - } else { - schema_ = value; - } - - bitField0_ |= 0x00000002; - return this; - } - public Builder clearSchema() { - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSuccess) - } - - static { - defaultInstance = new CommandSuccess(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSuccess) - } - - public interface CommandProducerSuccessOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required string producer_name = 2; - boolean hasProducerName(); - String getProducerName(); - - // optional int64 last_sequence_id = 3 [default = -1]; - boolean hasLastSequenceId(); - long getLastSequenceId(); - - // optional bytes schema_version = 4; - boolean hasSchemaVersion(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); - } - public static final class CommandProducerSuccess extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandProducerSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandProducerSuccess.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandProducerSuccess(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandProducerSuccess newObject(Handle handle) { - return new CommandProducerSuccess(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandProducerSuccess(boolean noInit) {} - - private static final CommandProducerSuccess defaultInstance; - public static CommandProducerSuccess getDefaultInstance() { - return defaultInstance; - } - - public CommandProducerSuccess getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required string producer_name = 2; - public static final int PRODUCER_NAME_FIELD_NUMBER = 2; - private java.lang.Object producerName_; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - 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)) { - producerName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { - java.lang.Object ref = producerName_; - 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); - producerName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional int64 last_sequence_id = 3 [default = -1]; - public static final int LAST_SEQUENCE_ID_FIELD_NUMBER = 3; - private long lastSequenceId_; - public boolean hasLastSequenceId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getLastSequenceId() { - return lastSequenceId_; - } - - // optional bytes schema_version = 4; - public static final int SCHEMA_VERSION_FIELD_NUMBER = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - - private void initFields() { - requestId_ = 0L; - producerName_ = ""; - lastSequenceId_ = -1L; - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasProducerName()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeInt64(3, lastSequenceId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, schemaVersion_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getProducerNameBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeInt64Size(3, lastSequenceId_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, schemaVersion_); - } - 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - producerName_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - lastSequenceId_ = -1L; - bitField0_ = (bitField0_ & ~0x00000004); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.producerName_ = producerName_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.lastSequenceId_ = lastSequenceId_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.schemaVersion_ = schemaVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasProducerName()) { - setProducerName(other.getProducerName()); - } - if (other.hasLastSequenceId()) { - setLastSequenceId(other.getLastSequenceId()); - } - if (other.hasSchemaVersion()) { - setSchemaVersion(other.getSchemaVersion()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasProducerName()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - producerName_ = input.readBytes(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - lastSequenceId_ = input.readInt64(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - schemaVersion_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required string producer_name = 2; - private java.lang.Object producerName_ = ""; - public boolean hasProducerName() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getProducerName() { - java.lang.Object ref = producerName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - producerName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setProducerName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - producerName_ = value; - - return this; - } - public Builder clearProducerName() { - bitField0_ = (bitField0_ & ~0x00000002); - producerName_ = getDefaultInstance().getProducerName(); - - return this; - } - void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - producerName_ = value; - - } - - // optional int64 last_sequence_id = 3 [default = -1]; - private long lastSequenceId_ = -1L; - public boolean hasLastSequenceId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getLastSequenceId() { - return lastSequenceId_; - } - public Builder setLastSequenceId(long value) { - bitField0_ |= 0x00000004; - lastSequenceId_ = value; - - return this; - } - public Builder clearLastSequenceId() { - bitField0_ = (bitField0_ & ~0x00000004); - lastSequenceId_ = -1L; - - return this; - } - - // optional bytes schema_version = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - schemaVersion_ = value; - - return this; - } - public Builder clearSchemaVersion() { - bitField0_ = (bitField0_ & ~0x00000008); - schemaVersion_ = getDefaultInstance().getSchemaVersion(); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandProducerSuccess) - } - - static { - defaultInstance = new CommandProducerSuccess(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandProducerSuccess) - } - - public interface CommandErrorOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required .pulsar.proto.ServerError error = 2; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // required string message = 3; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandError extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandError.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandError(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandError newObject(Handle handle) { - return new CommandError(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandError(boolean noInit) {} - - private static final CommandError defaultInstance; - public static CommandError getDefaultInstance() { - return defaultInstance; - } - - public CommandError getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required .pulsar.proto.ServerError error = 2; - public static final int ERROR_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // required string message = 3; - public static final int MESSAGE_FIELD_NUMBER = 3; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasError()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasMessage()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, error_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, error_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, getMessageBytes()); - } - 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandError.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000002); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandError build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandError buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandError buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandError other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasError()) { - - return false; - } - if (!hasMessage()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - error_ = value; - } - break; - } - case 26: { - bitField0_ |= 0x00000004; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required .pulsar.proto.ServerError error = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000002); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // required string message = 3; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000004); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandError) - } - - static { - defaultInstance = new CommandError(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandError) - } - - public interface CommandPingOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - } - public static final class CommandPing extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandPingOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandPing.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandPing(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandPing newObject(Handle handle) { - return new CommandPing(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandPing(boolean noInit) {} - - private static final CommandPing defaultInstance; - public static CommandPing getDefaultInstance() { - return defaultInstance; - } - - public CommandPing getDefaultInstanceForType() { - return defaultInstance; - } - - private void initFields() { - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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(); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPingOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.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(); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.RECYCLER.get(); - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance()) return this; - return this; - } - - public final boolean isInitialized() { - 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; - } - } - } - } - - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPing) - } - - static { - defaultInstance = new CommandPing(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPing) - } - - public interface CommandPongOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - } - public static final class CommandPong extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandPongOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandPong.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandPong(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandPong newObject(Handle handle) { - return new CommandPong(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandPong(boolean noInit) {} - - private static final CommandPong defaultInstance; - public static CommandPong getDefaultInstance() { - return defaultInstance; - } - - public CommandPong getDefaultInstanceForType() { - return defaultInstance; - } - - private void initFields() { - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - 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(); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPongOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.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(); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.RECYCLER.get(); - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance()) return this; - return this; - } - - public final boolean isInitialized() { - 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; - } - } - } - } - - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPong) - } - - static { - defaultInstance = new CommandPong(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPong) - } - - public interface CommandConsumerStatsOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required uint64 consumer_id = 4; - boolean hasConsumerId(); - long getConsumerId(); - } - public static final class CommandConsumerStats extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandConsumerStatsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandConsumerStats.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandConsumerStats(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandConsumerStats newObject(Handle handle) { - return new CommandConsumerStats(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandConsumerStats(boolean noInit) {} - - private static final CommandConsumerStats defaultInstance; - public static CommandConsumerStats getDefaultInstance() { - return defaultInstance; - } - - public CommandConsumerStats getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required uint64 consumer_id = 4; - public static final int CONSUMER_ID_FIELD_NUMBER = 4; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getConsumerId() { - return consumerId_; - } - - private void initFields() { - requestId_ = 0L; - consumerId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasConsumerId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(4, consumerId_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(4, consumerId_); - } - 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.consumerId_ = consumerId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasConsumerId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 32: { - bitField0_ |= 0x00000002; - consumerId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required uint64 consumer_id = 4; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000002; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000002); - consumerId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStats) - } - - static { - defaultInstance = new CommandConsumerStats(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStats) - } - - public interface CommandConsumerStatsResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.ServerError error_code = 2; - boolean hasErrorCode(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); - - // optional string error_message = 3; - boolean hasErrorMessage(); - String getErrorMessage(); - - // optional double msgRateOut = 4; - boolean hasMsgRateOut(); - double getMsgRateOut(); - - // optional double msgThroughputOut = 5; - boolean hasMsgThroughputOut(); - double getMsgThroughputOut(); - - // optional double msgRateRedeliver = 6; - boolean hasMsgRateRedeliver(); - double getMsgRateRedeliver(); - - // optional string consumerName = 7; - boolean hasConsumerName(); - String getConsumerName(); - - // optional uint64 availablePermits = 8; - boolean hasAvailablePermits(); - long getAvailablePermits(); - - // optional uint64 unackedMessages = 9; - boolean hasUnackedMessages(); - long getUnackedMessages(); - - // optional bool blockedConsumerOnUnackedMsgs = 10; - boolean hasBlockedConsumerOnUnackedMsgs(); - boolean getBlockedConsumerOnUnackedMsgs(); - - // optional string address = 11; - boolean hasAddress(); - String getAddress(); - - // optional string connectedSince = 12; - boolean hasConnectedSince(); - String getConnectedSince(); - - // optional string type = 13; - boolean hasType(); - String getType(); - - // optional double msgRateExpired = 14; - boolean hasMsgRateExpired(); - double getMsgRateExpired(); - - // optional uint64 msgBacklog = 15; - boolean hasMsgBacklog(); - long getMsgBacklog(); - } - public static final class CommandConsumerStatsResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandConsumerStatsResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandConsumerStatsResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandConsumerStatsResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandConsumerStatsResponse newObject(Handle handle) { - return new CommandConsumerStatsResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandConsumerStatsResponse(boolean noInit) {} - - private static final CommandConsumerStatsResponse defaultInstance; - public static CommandConsumerStatsResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandConsumerStatsResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.ServerError error_code = 2; - public static final int ERROR_CODE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - - // optional string error_message = 3; - public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; - private java.lang.Object errorMessage_; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - 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)) { - errorMessage_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { - java.lang.Object ref = errorMessage_; - 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); - errorMessage_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional double msgRateOut = 4; - public static final int MSGRATEOUT_FIELD_NUMBER = 4; - private double msgRateOut_; - public boolean hasMsgRateOut() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public double getMsgRateOut() { - return msgRateOut_; - } - - // optional double msgThroughputOut = 5; - public static final int MSGTHROUGHPUTOUT_FIELD_NUMBER = 5; - private double msgThroughputOut_; - public boolean hasMsgThroughputOut() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public double getMsgThroughputOut() { - return msgThroughputOut_; - } - - // optional double msgRateRedeliver = 6; - public static final int MSGRATEREDELIVER_FIELD_NUMBER = 6; - private double msgRateRedeliver_; - public boolean hasMsgRateRedeliver() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public double getMsgRateRedeliver() { - return msgRateRedeliver_; - } - - // optional string consumerName = 7; - public static final int CONSUMERNAME_FIELD_NUMBER = 7; - private java.lang.Object consumerName_; - public boolean hasConsumerName() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getConsumerName() { - java.lang.Object ref = consumerName_; - 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)) { - consumerName_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConsumerNameBytes() { - java.lang.Object ref = consumerName_; - 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); - consumerName_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional uint64 availablePermits = 8; - public static final int AVAILABLEPERMITS_FIELD_NUMBER = 8; - private long availablePermits_; - public boolean hasAvailablePermits() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public long getAvailablePermits() { - return availablePermits_; - } - - // optional uint64 unackedMessages = 9; - public static final int UNACKEDMESSAGES_FIELD_NUMBER = 9; - private long unackedMessages_; - public boolean hasUnackedMessages() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public long getUnackedMessages() { - return unackedMessages_; - } - - // optional bool blockedConsumerOnUnackedMsgs = 10; - public static final int BLOCKEDCONSUMERONUNACKEDMSGS_FIELD_NUMBER = 10; - private boolean blockedConsumerOnUnackedMsgs_; - public boolean hasBlockedConsumerOnUnackedMsgs() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public boolean getBlockedConsumerOnUnackedMsgs() { - return blockedConsumerOnUnackedMsgs_; - } - - // optional string address = 11; - public static final int ADDRESS_FIELD_NUMBER = 11; - private java.lang.Object address_; - public boolean hasAddress() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public String getAddress() { - java.lang.Object ref = address_; - 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)) { - address_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAddressBytes() { - java.lang.Object ref = address_; - 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); - address_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string connectedSince = 12; - public static final int CONNECTEDSINCE_FIELD_NUMBER = 12; - private java.lang.Object connectedSince_; - public boolean hasConnectedSince() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public String getConnectedSince() { - java.lang.Object ref = connectedSince_; - 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)) { - connectedSince_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConnectedSinceBytes() { - java.lang.Object ref = connectedSince_; - 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); - connectedSince_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional string type = 13; - public static final int TYPE_FIELD_NUMBER = 13; - private java.lang.Object type_; - public boolean hasType() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public String getType() { - java.lang.Object ref = type_; - 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)) { - type_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTypeBytes() { - java.lang.Object ref = type_; - 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); - type_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional double msgRateExpired = 14; - public static final int MSGRATEEXPIRED_FIELD_NUMBER = 14; - private double msgRateExpired_; - public boolean hasMsgRateExpired() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public double getMsgRateExpired() { - return msgRateExpired_; - } - - // optional uint64 msgBacklog = 15; - public static final int MSGBACKLOG_FIELD_NUMBER = 15; - private long msgBacklog_; - public boolean hasMsgBacklog() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public long getMsgBacklog() { - return msgBacklog_; - } - - private void initFields() { - requestId_ = 0L; - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - errorMessage_ = ""; - msgRateOut_ = 0D; - msgThroughputOut_ = 0D; - msgRateRedeliver_ = 0D; - consumerName_ = ""; - availablePermits_ = 0L; - unackedMessages_ = 0L; - blockedConsumerOnUnackedMsgs_ = false; - address_ = ""; - connectedSince_ = ""; - type_ = ""; - msgRateExpired_ = 0D; - msgBacklog_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeDouble(4, msgRateOut_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeDouble(5, msgThroughputOut_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeDouble(6, msgRateRedeliver_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeBytes(7, getConsumerNameBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeUInt64(8, availablePermits_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeUInt64(9, unackedMessages_); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - output.writeBool(10, blockedConsumerOnUnackedMsgs_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - output.writeBytes(11, getAddressBytes()); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - output.writeBytes(12, getConnectedSinceBytes()); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - output.writeBytes(13, getTypeBytes()); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - output.writeDouble(14, msgRateExpired_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - output.writeUInt64(15, msgBacklog_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeDoubleSize(4, msgRateOut_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeDoubleSize(5, msgThroughputOut_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeDoubleSize(6, msgRateRedeliver_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(7, getConsumerNameBytes()); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(8, availablePermits_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(9, unackedMessages_); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBoolSize(10, blockedConsumerOnUnackedMsgs_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(11, getAddressBytes()); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(12, getConnectedSinceBytes()); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(13, getTypeBytes()); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeDoubleSize(14, msgRateExpired_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(15, msgBacklog_); - } - 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000002); - errorMessage_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - msgRateOut_ = 0D; - bitField0_ = (bitField0_ & ~0x00000008); - msgThroughputOut_ = 0D; - bitField0_ = (bitField0_ & ~0x00000010); - msgRateRedeliver_ = 0D; - bitField0_ = (bitField0_ & ~0x00000020); - consumerName_ = ""; - bitField0_ = (bitField0_ & ~0x00000040); - availablePermits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000080); - unackedMessages_ = 0L; - bitField0_ = (bitField0_ & ~0x00000100); - blockedConsumerOnUnackedMsgs_ = false; - bitField0_ = (bitField0_ & ~0x00000200); - address_ = ""; - bitField0_ = (bitField0_ & ~0x00000400); - connectedSince_ = ""; - bitField0_ = (bitField0_ & ~0x00000800); - type_ = ""; - bitField0_ = (bitField0_ & ~0x00001000); - msgRateExpired_ = 0D; - bitField0_ = (bitField0_ & ~0x00002000); - msgBacklog_ = 0L; - bitField0_ = (bitField0_ & ~0x00004000); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.errorCode_ = errorCode_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.errorMessage_ = errorMessage_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.msgRateOut_ = msgRateOut_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.msgThroughputOut_ = msgThroughputOut_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.msgRateRedeliver_ = msgRateRedeliver_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.consumerName_ = consumerName_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.availablePermits_ = availablePermits_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.unackedMessages_ = unackedMessages_; - if (((from_bitField0_ & 0x00000200) == 0x00000200)) { - to_bitField0_ |= 0x00000200; - } - result.blockedConsumerOnUnackedMsgs_ = blockedConsumerOnUnackedMsgs_; - if (((from_bitField0_ & 0x00000400) == 0x00000400)) { - to_bitField0_ |= 0x00000400; - } - result.address_ = address_; - if (((from_bitField0_ & 0x00000800) == 0x00000800)) { - to_bitField0_ |= 0x00000800; - } - result.connectedSince_ = connectedSince_; - if (((from_bitField0_ & 0x00001000) == 0x00001000)) { - to_bitField0_ |= 0x00001000; - } - result.type_ = type_; - if (((from_bitField0_ & 0x00002000) == 0x00002000)) { - to_bitField0_ |= 0x00002000; - } - result.msgRateExpired_ = msgRateExpired_; - if (((from_bitField0_ & 0x00004000) == 0x00004000)) { - to_bitField0_ |= 0x00004000; - } - result.msgBacklog_ = msgBacklog_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasErrorCode()) { - setErrorCode(other.getErrorCode()); - } - if (other.hasErrorMessage()) { - setErrorMessage(other.getErrorMessage()); - } - if (other.hasMsgRateOut()) { - setMsgRateOut(other.getMsgRateOut()); - } - if (other.hasMsgThroughputOut()) { - setMsgThroughputOut(other.getMsgThroughputOut()); - } - if (other.hasMsgRateRedeliver()) { - setMsgRateRedeliver(other.getMsgRateRedeliver()); - } - if (other.hasConsumerName()) { - setConsumerName(other.getConsumerName()); - } - if (other.hasAvailablePermits()) { - setAvailablePermits(other.getAvailablePermits()); - } - if (other.hasUnackedMessages()) { - setUnackedMessages(other.getUnackedMessages()); - } - if (other.hasBlockedConsumerOnUnackedMsgs()) { - setBlockedConsumerOnUnackedMsgs(other.getBlockedConsumerOnUnackedMsgs()); - } - if (other.hasAddress()) { - setAddress(other.getAddress()); - } - if (other.hasConnectedSince()) { - setConnectedSince(other.getConnectedSince()); - } - if (other.hasType()) { - setType(other.getType()); - } - if (other.hasMsgRateExpired()) { - setMsgRateExpired(other.getMsgRateExpired()); - } - if (other.hasMsgBacklog()) { - setMsgBacklog(other.getMsgBacklog()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - errorCode_ = value; - } - break; - } - case 26: { - bitField0_ |= 0x00000004; - errorMessage_ = input.readBytes(); - break; - } - case 33: { - bitField0_ |= 0x00000008; - msgRateOut_ = input.readDouble(); - break; - } - case 41: { - bitField0_ |= 0x00000010; - msgThroughputOut_ = input.readDouble(); - break; - } - case 49: { - bitField0_ |= 0x00000020; - msgRateRedeliver_ = input.readDouble(); - break; - } - case 58: { - bitField0_ |= 0x00000040; - consumerName_ = input.readBytes(); - break; - } - case 64: { - bitField0_ |= 0x00000080; - availablePermits_ = input.readUInt64(); - break; - } - case 72: { - bitField0_ |= 0x00000100; - unackedMessages_ = input.readUInt64(); - break; - } - case 80: { - bitField0_ |= 0x00000200; - blockedConsumerOnUnackedMsgs_ = input.readBool(); - break; - } - case 90: { - bitField0_ |= 0x00000400; - address_ = input.readBytes(); - break; - } - case 98: { - bitField0_ |= 0x00000800; - connectedSince_ = input.readBytes(); - break; - } - case 106: { - bitField0_ |= 0x00001000; - type_ = input.readBytes(); - break; - } - case 113: { - bitField0_ |= 0x00002000; - msgRateExpired_ = input.readDouble(); - break; - } - case 120: { - bitField0_ |= 0x00004000; - msgBacklog_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error_code = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - errorCode_ = value; - - return this; - } - public Builder clearErrorCode() { - bitField0_ = (bitField0_ & ~0x00000002); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string error_message = 3; - private java.lang.Object errorMessage_ = ""; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - errorMessage_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setErrorMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - errorMessage_ = value; - - return this; - } - public Builder clearErrorMessage() { - bitField0_ = (bitField0_ & ~0x00000004); - errorMessage_ = getDefaultInstance().getErrorMessage(); - - return this; - } - void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - errorMessage_ = value; - - } - - // optional double msgRateOut = 4; - private double msgRateOut_ ; - public boolean hasMsgRateOut() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public double getMsgRateOut() { - return msgRateOut_; - } - public Builder setMsgRateOut(double value) { - bitField0_ |= 0x00000008; - msgRateOut_ = value; - - return this; - } - public Builder clearMsgRateOut() { - bitField0_ = (bitField0_ & ~0x00000008); - msgRateOut_ = 0D; - - return this; - } - - // optional double msgThroughputOut = 5; - private double msgThroughputOut_ ; - public boolean hasMsgThroughputOut() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public double getMsgThroughputOut() { - return msgThroughputOut_; - } - public Builder setMsgThroughputOut(double value) { - bitField0_ |= 0x00000010; - msgThroughputOut_ = value; - - return this; - } - public Builder clearMsgThroughputOut() { - bitField0_ = (bitField0_ & ~0x00000010); - msgThroughputOut_ = 0D; - - return this; - } - - // optional double msgRateRedeliver = 6; - private double msgRateRedeliver_ ; - public boolean hasMsgRateRedeliver() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public double getMsgRateRedeliver() { - return msgRateRedeliver_; - } - public Builder setMsgRateRedeliver(double value) { - bitField0_ |= 0x00000020; - msgRateRedeliver_ = value; - - return this; - } - public Builder clearMsgRateRedeliver() { - bitField0_ = (bitField0_ & ~0x00000020); - msgRateRedeliver_ = 0D; - - return this; - } - - // optional string consumerName = 7; - private java.lang.Object consumerName_ = ""; - public boolean hasConsumerName() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public String getConsumerName() { - java.lang.Object ref = consumerName_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - consumerName_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setConsumerName(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000040; - consumerName_ = value; - - return this; - } - public Builder clearConsumerName() { - bitField0_ = (bitField0_ & ~0x00000040); - consumerName_ = getDefaultInstance().getConsumerName(); - - return this; - } - void setConsumerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000040; - consumerName_ = value; - - } - - // optional uint64 availablePermits = 8; - private long availablePermits_ ; - public boolean hasAvailablePermits() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public long getAvailablePermits() { - return availablePermits_; - } - public Builder setAvailablePermits(long value) { - bitField0_ |= 0x00000080; - availablePermits_ = value; - - return this; - } - public Builder clearAvailablePermits() { - bitField0_ = (bitField0_ & ~0x00000080); - availablePermits_ = 0L; - - return this; - } - - // optional uint64 unackedMessages = 9; - private long unackedMessages_ ; - public boolean hasUnackedMessages() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public long getUnackedMessages() { - return unackedMessages_; - } - public Builder setUnackedMessages(long value) { - bitField0_ |= 0x00000100; - unackedMessages_ = value; - - return this; - } - public Builder clearUnackedMessages() { - bitField0_ = (bitField0_ & ~0x00000100); - unackedMessages_ = 0L; - - return this; - } - - // optional bool blockedConsumerOnUnackedMsgs = 10; - private boolean blockedConsumerOnUnackedMsgs_ ; - public boolean hasBlockedConsumerOnUnackedMsgs() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public boolean getBlockedConsumerOnUnackedMsgs() { - return blockedConsumerOnUnackedMsgs_; - } - public Builder setBlockedConsumerOnUnackedMsgs(boolean value) { - bitField0_ |= 0x00000200; - blockedConsumerOnUnackedMsgs_ = value; - - return this; - } - public Builder clearBlockedConsumerOnUnackedMsgs() { - bitField0_ = (bitField0_ & ~0x00000200); - blockedConsumerOnUnackedMsgs_ = false; - - return this; - } - - // optional string address = 11; - private java.lang.Object address_ = ""; - public boolean hasAddress() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public String getAddress() { - java.lang.Object ref = address_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - address_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setAddress(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000400; - address_ = value; - - return this; - } - public Builder clearAddress() { - bitField0_ = (bitField0_ & ~0x00000400); - address_ = getDefaultInstance().getAddress(); - - return this; - } - void setAddress(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000400; - address_ = value; - - } - - // optional string connectedSince = 12; - private java.lang.Object connectedSince_ = ""; - public boolean hasConnectedSince() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public String getConnectedSince() { - java.lang.Object ref = connectedSince_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - connectedSince_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setConnectedSince(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000800; - connectedSince_ = value; - - return this; - } - public Builder clearConnectedSince() { - bitField0_ = (bitField0_ & ~0x00000800); - connectedSince_ = getDefaultInstance().getConnectedSince(); - - return this; - } - void setConnectedSince(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000800; - connectedSince_ = value; - - } - - // optional string type = 13; - private java.lang.Object type_ = ""; - public boolean hasType() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public String getType() { - java.lang.Object ref = type_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - type_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setType(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00001000; - type_ = value; - - return this; - } - public Builder clearType() { - bitField0_ = (bitField0_ & ~0x00001000); - type_ = getDefaultInstance().getType(); - - return this; - } - void setType(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00001000; - type_ = value; - - } - - // optional double msgRateExpired = 14; - private double msgRateExpired_ ; - public boolean hasMsgRateExpired() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public double getMsgRateExpired() { - return msgRateExpired_; - } - public Builder setMsgRateExpired(double value) { - bitField0_ |= 0x00002000; - msgRateExpired_ = value; - - return this; - } - public Builder clearMsgRateExpired() { - bitField0_ = (bitField0_ & ~0x00002000); - msgRateExpired_ = 0D; - - return this; - } - - // optional uint64 msgBacklog = 15; - private long msgBacklog_ ; - public boolean hasMsgBacklog() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public long getMsgBacklog() { - return msgBacklog_; - } - public Builder setMsgBacklog(long value) { - bitField0_ |= 0x00004000; - msgBacklog_ = value; - - return this; - } - public Builder clearMsgBacklog() { - bitField0_ = (bitField0_ & ~0x00004000); - msgBacklog_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStatsResponse) - } - - static { - defaultInstance = new CommandConsumerStatsResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStatsResponse) - } - - public interface CommandGetLastMessageIdOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 consumer_id = 1; - boolean hasConsumerId(); - long getConsumerId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - } - public static final class CommandGetLastMessageId extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetLastMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetLastMessageId.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetLastMessageId(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetLastMessageId newObject(Handle handle) { - return new CommandGetLastMessageId(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetLastMessageId(boolean noInit) {} - - private static final CommandGetLastMessageId defaultInstance; - public static CommandGetLastMessageId getDefaultInstance() { - return defaultInstance; - } - - public CommandGetLastMessageId getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 consumer_id = 1; - public static final int CONSUMER_ID_FIELD_NUMBER = 1; - private long consumerId_; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - private void initFields() { - consumerId_ = 0L; - requestId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasConsumerId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - } - - 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, consumerId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.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(); - consumerId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.consumerId_ = consumerId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance()) return this; - if (other.hasConsumerId()) { - setConsumerId(other.getConsumerId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasConsumerId()) { - - return false; - } - if (!hasRequestId()) { - - 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; - consumerId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 consumer_id = 1; - private long consumerId_ ; - public boolean hasConsumerId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getConsumerId() { - return consumerId_; - } - public Builder setConsumerId(long value) { - bitField0_ |= 0x00000001; - consumerId_ = value; - - return this; - } - public Builder clearConsumerId() { - bitField0_ = (bitField0_ & ~0x00000001); - consumerId_ = 0L; - - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetLastMessageId) - } - - static { - defaultInstance = new CommandGetLastMessageId(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetLastMessageId) - } - - public interface CommandGetLastMessageIdResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required .pulsar.proto.MessageIdData last_message_id = 1; - boolean hasLastMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId(); - - // required uint64 request_id = 2; - boolean hasRequestId(); - long getRequestId(); - } - public static final class CommandGetLastMessageIdResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetLastMessageIdResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetLastMessageIdResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetLastMessageIdResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetLastMessageIdResponse newObject(Handle handle) { - return new CommandGetLastMessageIdResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetLastMessageIdResponse(boolean noInit) {} - - private static final CommandGetLastMessageIdResponse defaultInstance; - public static CommandGetLastMessageIdResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandGetLastMessageIdResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required .pulsar.proto.MessageIdData last_message_id = 1; - public static final int LAST_MESSAGE_ID_FIELD_NUMBER = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData lastMessageId_; - public boolean hasLastMessageId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId() { - return lastMessageId_; - } - - // required uint64 request_id = 2; - public static final int REQUEST_ID_FIELD_NUMBER = 2; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - - private void initFields() { - lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - requestId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasLastMessageId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!getLastMessageId().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.writeMessage(1, lastMessageId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, requestId_); - } - } - - 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 - .computeMessageSize(1, lastMessageId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, requestId_); - } - 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.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(); - lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.lastMessageId_ = lastMessageId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.requestId_ = requestId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance()) return this; - if (other.hasLastMessageId()) { - mergeLastMessageId(other.getLastMessageId()); - } - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasLastMessageId()) { - - return false; - } - if (!hasRequestId()) { - - return false; - } - if (!getLastMessageId().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: { - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); - if (hasLastMessageId()) { - subBuilder.mergeFrom(getLastMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setLastMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - requestId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required .pulsar.proto.MessageIdData last_message_id = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - public boolean hasLastMessageId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId() { - return lastMessageId_; - } - public Builder setLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (value == null) { - throw new NullPointerException(); - } - lastMessageId_ = value; - - bitField0_ |= 0x00000001; - return this; - } - public Builder setLastMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { - lastMessageId_ = builderForValue.build(); - - bitField0_ |= 0x00000001; - return this; - } - public Builder mergeLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { - if (((bitField0_ & 0x00000001) == 0x00000001) && - lastMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { - lastMessageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(lastMessageId_).mergeFrom(value).buildPartial(); - } else { - lastMessageId_ = value; - } - - bitField0_ |= 0x00000001; - return this; - } - public Builder clearLastMessageId() { - lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - // required uint64 request_id = 2; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000002; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000002); - requestId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetLastMessageIdResponse) - } - - static { - defaultInstance = new CommandGetLastMessageIdResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetLastMessageIdResponse) - } - - public interface CommandGetTopicsOfNamespaceOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required string namespace = 2; - boolean hasNamespace(); - String getNamespace(); - - // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; - boolean hasMode(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode(); - } - public static final class CommandGetTopicsOfNamespace extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetTopicsOfNamespaceOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetTopicsOfNamespace.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetTopicsOfNamespace(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetTopicsOfNamespace newObject(Handle handle) { - return new CommandGetTopicsOfNamespace(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetTopicsOfNamespace(boolean noInit) {} - - private static final CommandGetTopicsOfNamespace defaultInstance; - public static CommandGetTopicsOfNamespace getDefaultInstance() { - return defaultInstance; - } - - public CommandGetTopicsOfNamespace getDefaultInstanceForType() { - return defaultInstance; - } - - public enum Mode - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - PERSISTENT(0, 0), - NON_PERSISTENT(1, 1), - ALL(2, 2), - ; - - public static final int PERSISTENT_VALUE = 0; - public static final int NON_PERSISTENT_VALUE = 1; - public static final int ALL_VALUE = 2; - - - public final int getNumber() { return value; } - - public static Mode valueOf(int value) { - switch (value) { - case 0: return PERSISTENT; - case 1: return NON_PERSISTENT; - case 2: return ALL; - 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 Mode findValueByNumber(int number) { - return Mode.valueOf(number); - } - }; - - private final int value; - - private Mode(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandGetTopicsOfNamespace.Mode) - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required string namespace = 2; - public static final int NAMESPACE_FIELD_NUMBER = 2; - private java.lang.Object namespace_; - public boolean hasNamespace() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getNamespace() { - java.lang.Object ref = namespace_; - 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)) { - namespace_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getNamespaceBytes() { - java.lang.Object ref = namespace_; - 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); - namespace_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; - public static final int MODE_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode mode_; - public boolean hasMode() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode() { - return mode_; - } - - private void initFields() { - requestId_ = 0L; - namespace_ = ""; - mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasNamespace()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getNamespaceBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeEnum(3, mode_.getNumber()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getNamespaceBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(3, mode_.getNumber()); - } - 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - namespace_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.namespace_ = namespace_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.mode_ = mode_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasNamespace()) { - setNamespace(other.getNamespace()); - } - if (other.hasMode()) { - setMode(other.getMode()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasNamespace()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - namespace_ = input.readBytes(); - break; - } - case 24: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode value = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000004; - mode_ = value; - } - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required string namespace = 2; - private java.lang.Object namespace_ = ""; - public boolean hasNamespace() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getNamespace() { - java.lang.Object ref = namespace_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - namespace_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setNamespace(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - namespace_ = value; - - return this; - } - public Builder clearNamespace() { - bitField0_ = (bitField0_ & ~0x00000002); - namespace_ = getDefaultInstance().getNamespace(); - - return this; - } - void setNamespace(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - namespace_ = value; - - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; - public boolean hasMode() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode() { - return mode_; - } - public Builder setMode(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - mode_ = value; - - return this; - } - public Builder clearMode() { - bitField0_ = (bitField0_ & ~0x00000004); - mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetTopicsOfNamespace) - } - - static { - defaultInstance = new CommandGetTopicsOfNamespace(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetTopicsOfNamespace) - } - - public interface CommandGetTopicsOfNamespaceResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // repeated string topics = 2; - java.util.List getTopicsList(); - int getTopicsCount(); - String getTopics(int index); - } - public static final class CommandGetTopicsOfNamespaceResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetTopicsOfNamespaceResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetTopicsOfNamespaceResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetTopicsOfNamespaceResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetTopicsOfNamespaceResponse newObject(Handle handle) { - return new CommandGetTopicsOfNamespaceResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetTopicsOfNamespaceResponse(boolean noInit) {} - - private static final CommandGetTopicsOfNamespaceResponse defaultInstance; - public static CommandGetTopicsOfNamespaceResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandGetTopicsOfNamespaceResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // repeated string topics = 2; - public static final int TOPICS_FIELD_NUMBER = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList topics_; - public java.util.List - getTopicsList() { - return topics_; - } - public int getTopicsCount() { - return topics_.size(); - } - public String getTopics(int index) { - return topics_.get(index); - } - - private void initFields() { - requestId_ = 0L; - topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - for (int i = 0; i < topics_.size(); i++) { - output.writeBytes(2, topics_.getByteString(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 - .computeUInt64Size(1, requestId_); - } - { - int dataSize = 0; - for (int i = 0; i < topics_.size(); i++) { - dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSizeNoTag(topics_.getByteString(i)); - } - size += dataSize; - size += 1 * getTopicsList().size(); - } - 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((bitField0_ & 0x00000002) == 0x00000002)) { - topics_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( - topics_); - bitField0_ = (bitField0_ & ~0x00000002); - } - result.topics_ = topics_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (!other.topics_.isEmpty()) { - if (topics_.isEmpty()) { - topics_ = other.topics_; - bitField0_ = (bitField0_ & ~0x00000002); - } else { - ensureTopicsIsMutable(); - topics_.addAll(other.topics_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 18: { - ensureTopicsIsMutable(); - topics_.add(input.readBytes()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // repeated string topics = 2; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - private void ensureTopicsIsMutable() { - if (!((bitField0_ & 0x00000002) == 0x00000002)) { - topics_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(topics_); - bitField0_ |= 0x00000002; - } - } - public java.util.List - getTopicsList() { - return java.util.Collections.unmodifiableList(topics_); - } - public int getTopicsCount() { - return topics_.size(); - } - public String getTopics(int index) { - return topics_.get(index); - } - public Builder setTopics( - int index, String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureTopicsIsMutable(); - topics_.set(index, value); - - return this; - } - public Builder addTopics(String value) { - if (value == null) { - throw new NullPointerException(); - } - ensureTopicsIsMutable(); - topics_.add(value); - - return this; - } - public Builder addAllTopics( - java.lang.Iterable values) { - ensureTopicsIsMutable(); - super.addAll(values, topics_); - - return this; - } - public Builder clearTopics() { - topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000002); - - return this; - } - void addTopics(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - ensureTopicsIsMutable(); - topics_.add(value); - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetTopicsOfNamespaceResponse) - } - - static { - defaultInstance = new CommandGetTopicsOfNamespaceResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetTopicsOfNamespaceResponse) - } - - public interface CommandGetSchemaOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required string topic = 2; - boolean hasTopic(); - String getTopic(); - - // optional bytes schema_version = 3; - boolean hasSchemaVersion(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); - } - public static final class CommandGetSchema extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetSchema.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetSchema(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetSchema newObject(Handle handle) { - return new CommandGetSchema(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetSchema(boolean noInit) {} - - private static final CommandGetSchema defaultInstance; - public static CommandGetSchema getDefaultInstance() { - return defaultInstance; - } - - public CommandGetSchema getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required string topic = 2; - public static final int TOPIC_FIELD_NUMBER = 2; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bytes schema_version = 3; - public static final int SCHEMA_VERSION_FIELD_NUMBER = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - - private void initFields() { - requestId_ = 0L; - topic_ = ""; - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasTopic()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getTopicBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, schemaVersion_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getTopicBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, schemaVersion_); - } - 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.schemaVersion_ = schemaVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasSchemaVersion()) { - setSchemaVersion(other.getSchemaVersion()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasTopic()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - topic_ = input.readBytes(); - break; - } - case 26: { - bitField0_ |= 0x00000004; - schemaVersion_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required string topic = 2; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000002); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - topic_ = value; - - } - - // optional bytes schema_version = 3; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - schemaVersion_ = value; - - return this; - } - public Builder clearSchemaVersion() { - bitField0_ = (bitField0_ & ~0x00000004); - schemaVersion_ = getDefaultInstance().getSchemaVersion(); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetSchema) - } - - static { - defaultInstance = new CommandGetSchema(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetSchema) - } - - public interface CommandGetSchemaResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.ServerError error_code = 2; - boolean hasErrorCode(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); - - // optional string error_message = 3; - boolean hasErrorMessage(); - String getErrorMessage(); - - // optional .pulsar.proto.Schema schema = 4; - boolean hasSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); - - // optional bytes schema_version = 5; - boolean hasSchemaVersion(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); - } - public static final class CommandGetSchemaResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetSchemaResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetSchemaResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetSchemaResponse newObject(Handle handle) { - return new CommandGetSchemaResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetSchemaResponse(boolean noInit) {} - - private static final CommandGetSchemaResponse defaultInstance; - public static CommandGetSchemaResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandGetSchemaResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.ServerError error_code = 2; - public static final int ERROR_CODE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - - // optional string error_message = 3; - public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; - private java.lang.Object errorMessage_; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - 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)) { - errorMessage_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { - java.lang.Object ref = errorMessage_; - 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); - errorMessage_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.Schema schema = 4; - public static final int SCHEMA_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; - public boolean hasSchema() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - - // optional bytes schema_version = 5; - public static final int SCHEMA_VERSION_FIELD_NUMBER = 5; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - - private void initFields() { - requestId_ = 0L; - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - errorMessage_ = ""; - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasSchema()) { - if (!getSchema().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.writeUInt64(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeMessage(4, schema_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, schemaVersion_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, schema_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, schemaVersion_); - } - 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000002); - errorMessage_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000008); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.errorCode_ = errorCode_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.errorMessage_ = errorMessage_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.schema_ = schema_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.schemaVersion_ = schemaVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasErrorCode()) { - setErrorCode(other.getErrorCode()); - } - if (other.hasErrorMessage()) { - setErrorMessage(other.getErrorMessage()); - } - if (other.hasSchema()) { - mergeSchema(other.getSchema()); - } - if (other.hasSchemaVersion()) { - setSchemaVersion(other.getSchemaVersion()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (hasSchema()) { - if (!getSchema().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 8: { - bitField0_ |= 0x00000001; - requestId_ = input.readUInt64(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - errorCode_ = value; - } - break; - } - case 26: { - bitField0_ |= 0x00000004; - errorMessage_ = input.readBytes(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); - if (hasSchema()) { - subBuilder.mergeFrom(getSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 42: { - bitField0_ |= 0x00000010; - schemaVersion_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error_code = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - errorCode_ = value; - - return this; - } - public Builder clearErrorCode() { - bitField0_ = (bitField0_ & ~0x00000002); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string error_message = 3; - private java.lang.Object errorMessage_ = ""; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - errorMessage_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setErrorMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - errorMessage_ = value; - - return this; - } - public Builder clearErrorMessage() { - bitField0_ = (bitField0_ & ~0x00000004); - errorMessage_ = getDefaultInstance().getErrorMessage(); - - return this; - } - void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - errorMessage_ = value; - - } - - // optional .pulsar.proto.Schema schema = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - public boolean hasSchema() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (value == null) { - throw new NullPointerException(); - } - schema_ = value; - - bitField0_ |= 0x00000008; - return this; - } - public Builder setSchema( - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { - schema_ = builderForValue.build(); - - bitField0_ |= 0x00000008; - return this; - } - public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (((bitField0_ & 0x00000008) == 0x00000008) && - schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { - schema_ = - org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); - } else { - schema_ = value; - } - - bitField0_ |= 0x00000008; - return this; - } - public Builder clearSchema() { - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - // optional bytes schema_version = 5; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - schemaVersion_ = value; - - return this; - } - public Builder clearSchemaVersion() { - bitField0_ = (bitField0_ & ~0x00000010); - schemaVersion_ = getDefaultInstance().getSchemaVersion(); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetSchemaResponse) - } - - static { - defaultInstance = new CommandGetSchemaResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetSchemaResponse) - } - - public interface CommandGetOrCreateSchemaOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // required string topic = 2; - boolean hasTopic(); - String getTopic(); - - // required .pulsar.proto.Schema schema = 3; - boolean hasSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); - } - public static final class CommandGetOrCreateSchema extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetOrCreateSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetOrCreateSchema.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetOrCreateSchema(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetOrCreateSchema newObject(Handle handle) { - return new CommandGetOrCreateSchema(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetOrCreateSchema(boolean noInit) {} - - private static final CommandGetOrCreateSchema defaultInstance; - public static CommandGetOrCreateSchema getDefaultInstance() { - return defaultInstance; - } - - public CommandGetOrCreateSchema getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // required string topic = 2; - public static final int TOPIC_FIELD_NUMBER = 2; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required .pulsar.proto.Schema schema = 3; - public static final int SCHEMA_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; - public boolean hasSchema() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - - private void initFields() { - requestId_ = 0L; - topic_ = ""; - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSchema()) { - memoizedIsInitialized = 0; - return false; - } - if (!getSchema().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.writeUInt64(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getTopicBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeMessage(3, schema_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getTopicBytes()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, schema_); - } - 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.schema_ = schema_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasSchema()) { - mergeSchema(other.getSchema()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (!hasTopic()) { - - return false; - } - if (!hasSchema()) { - - return false; - } - if (!getSchema().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 8: { - bitField0_ |= 0x00000001; - requestId_ = input.readUInt64(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - topic_ = input.readBytes(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); - if (hasSchema()) { - subBuilder.mergeFrom(getSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // required string topic = 2; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000002); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - topic_ = value; - - } - - // required .pulsar.proto.Schema schema = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - public boolean hasSchema() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { - return schema_; - } - public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (value == null) { - throw new NullPointerException(); - } - schema_ = value; - - bitField0_ |= 0x00000004; - return this; - } - public Builder setSchema( - org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { - schema_ = builderForValue.build(); - - bitField0_ |= 0x00000004; - return this; - } - public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { - if (((bitField0_ & 0x00000004) == 0x00000004) && - schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { - schema_ = - org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); - } else { - schema_ = value; - } - - bitField0_ |= 0x00000004; - return this; - } - public Builder clearSchema() { - schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetOrCreateSchema) - } - - static { - defaultInstance = new CommandGetOrCreateSchema(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetOrCreateSchema) - } - - public interface CommandGetOrCreateSchemaResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional .pulsar.proto.ServerError error_code = 2; - boolean hasErrorCode(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); - - // optional string error_message = 3; - boolean hasErrorMessage(); - String getErrorMessage(); - - // optional bytes schema_version = 4; - boolean hasSchemaVersion(); - org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); - } - public static final class CommandGetOrCreateSchemaResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandGetOrCreateSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandGetOrCreateSchemaResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandGetOrCreateSchemaResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandGetOrCreateSchemaResponse newObject(Handle handle) { - return new CommandGetOrCreateSchemaResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandGetOrCreateSchemaResponse(boolean noInit) {} - - private static final CommandGetOrCreateSchemaResponse defaultInstance; - public static CommandGetOrCreateSchemaResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandGetOrCreateSchemaResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional .pulsar.proto.ServerError error_code = 2; - public static final int ERROR_CODE_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - - // optional string error_message = 3; - public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; - private java.lang.Object errorMessage_; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - 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)) { - errorMessage_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { - java.lang.Object ref = errorMessage_; - 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); - errorMessage_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional bytes schema_version = 4; - public static final int SCHEMA_VERSION_FIELD_NUMBER = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - - private void initFields() { - requestId_ = 0L; - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - errorMessage_ = ""; - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeBytes(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, schemaVersion_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(2, errorCode_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(3, getErrorMessageBytes()); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, schemaVersion_); - } - 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000002); - errorMessage_ = ""; - bitField0_ = (bitField0_ & ~0x00000004); - schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.errorCode_ = errorCode_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.errorMessage_ = errorMessage_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.schemaVersion_ = schemaVersion_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasErrorCode()) { - setErrorCode(other.getErrorCode()); - } - if (other.hasErrorMessage()) { - setErrorMessage(other.getErrorMessage()); - } - if (other.hasSchemaVersion()) { - setSchemaVersion(other.getSchemaVersion()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000002; - errorCode_ = value; - } - break; - } - case 26: { - bitField0_ |= 0x00000004; - errorMessage_ = input.readBytes(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - schemaVersion_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error_code = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasErrorCode() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { - return errorCode_; - } - public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - errorCode_ = value; - - return this; - } - public Builder clearErrorCode() { - bitField0_ = (bitField0_ & ~0x00000002); - errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string error_message = 3; - private java.lang.Object errorMessage_ = ""; - public boolean hasErrorMessage() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public String getErrorMessage() { - java.lang.Object ref = errorMessage_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - errorMessage_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setErrorMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000004; - errorMessage_ = value; - - return this; - } - public Builder clearErrorMessage() { - bitField0_ = (bitField0_ & ~0x00000004); - errorMessage_ = getDefaultInstance().getErrorMessage(); - - return this; - } - void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000004; - errorMessage_ = value; - - } - - // optional bytes schema_version = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; - public boolean hasSchemaVersion() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { - return schemaVersion_; - } - public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - schemaVersion_ = value; - - return this; - } - public Builder clearSchemaVersion() { - bitField0_ = (bitField0_ & ~0x00000008); - schemaVersion_ = getDefaultInstance().getSchemaVersion(); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetOrCreateSchemaResponse) - } - - static { - defaultInstance = new CommandGetOrCreateSchemaResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetOrCreateSchemaResponse) - } - - public interface CommandNewTxnOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txn_ttl_seconds = 2 [default = 0]; - boolean hasTxnTtlSeconds(); - long getTxnTtlSeconds(); - - // optional uint64 tc_id = 3 [default = 0]; - boolean hasTcId(); - long getTcId(); - } - public static final class CommandNewTxn extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandNewTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandNewTxn.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandNewTxn(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandNewTxn newObject(Handle handle) { - return new CommandNewTxn(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandNewTxn(boolean noInit) {} - - private static final CommandNewTxn defaultInstance; - public static CommandNewTxn getDefaultInstance() { - return defaultInstance; - } - - public CommandNewTxn getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txn_ttl_seconds = 2 [default = 0]; - public static final int TXN_TTL_SECONDS_FIELD_NUMBER = 2; - private long txnTtlSeconds_; - public boolean hasTxnTtlSeconds() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnTtlSeconds() { - return txnTtlSeconds_; - } - - // optional uint64 tc_id = 3 [default = 0]; - public static final int TC_ID_FIELD_NUMBER = 3; - private long tcId_; - public boolean hasTcId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTcId() { - return tcId_; - } - - private void initFields() { - requestId_ = 0L; - txnTtlSeconds_ = 0L; - tcId_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnTtlSeconds_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, tcId_); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnTtlSeconds_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, tcId_); - } - 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnTtlSeconds_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - tcId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnTtlSeconds_ = txnTtlSeconds_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.tcId_ = tcId_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnTtlSeconds()) { - setTxnTtlSeconds(other.getTxnTtlSeconds()); - } - if (other.hasTcId()) { - setTcId(other.getTcId()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnTtlSeconds_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - tcId_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txn_ttl_seconds = 2 [default = 0]; - private long txnTtlSeconds_ ; - public boolean hasTxnTtlSeconds() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnTtlSeconds() { - return txnTtlSeconds_; - } - public Builder setTxnTtlSeconds(long value) { - bitField0_ |= 0x00000002; - txnTtlSeconds_ = value; - - return this; - } - public Builder clearTxnTtlSeconds() { - bitField0_ = (bitField0_ & ~0x00000002); - txnTtlSeconds_ = 0L; - - return this; - } - - // optional uint64 tc_id = 3 [default = 0]; - private long tcId_ ; - public boolean hasTcId() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTcId() { - return tcId_; - } - public Builder setTcId(long value) { - bitField0_ |= 0x00000004; - tcId_ = value; - - return this; - } - public Builder clearTcId() { - bitField0_ = (bitField0_ & ~0x00000004); - tcId_ = 0L; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandNewTxn) - } - - static { - defaultInstance = new CommandNewTxn(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandNewTxn) - } - - public interface CommandNewTxnResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandNewTxnResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandNewTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandNewTxnResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandNewTxnResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandNewTxnResponse newObject(Handle handle) { - return new CommandNewTxnResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandNewTxnResponse(boolean noInit) {} - - private static final CommandNewTxnResponse defaultInstance; - public static CommandNewTxnResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandNewTxnResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandNewTxnResponse) - } - - static { - defaultInstance = new CommandNewTxnResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandNewTxnResponse) - } - - public interface CommandAddPartitionToTxnOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // repeated string partitions = 4; - java.util.List getPartitionsList(); - int getPartitionsCount(); - String getPartitions(int index); - } - public static final class CommandAddPartitionToTxn extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAddPartitionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAddPartitionToTxn.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAddPartitionToTxn(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAddPartitionToTxn newObject(Handle handle) { - return new CommandAddPartitionToTxn(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAddPartitionToTxn(boolean noInit) {} - - private static final CommandAddPartitionToTxn defaultInstance; - public static CommandAddPartitionToTxn getDefaultInstance() { - return defaultInstance; - } - - public CommandAddPartitionToTxn getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // repeated string partitions = 4; - public static final int PARTITIONS_FIELD_NUMBER = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList partitions_; - public java.util.List - getPartitionsList() { - return partitions_; - } - public int getPartitionsCount() { - return partitions_.size(); - } - public String getPartitions(int index) { - return partitions_.get(index); - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - for (int i = 0; i < partitions_.size(); i++) { - output.writeBytes(4, partitions_.getByteString(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 - .computeUInt64Size(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - { - int dataSize = 0; - for (int i = 0; i < partitions_.size(); i++) { - dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSizeNoTag(partitions_.getByteString(i)); - } - size += dataSize; - size += 1 * getPartitionsList().size(); - } - 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((bitField0_ & 0x00000008) == 0x00000008)) { - partitions_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( - partitions_); - bitField0_ = (bitField0_ & ~0x00000008); - } - result.partitions_ = partitions_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (!other.partitions_.isEmpty()) { - if (partitions_.isEmpty()) { - partitions_ = other.partitions_; - bitField0_ = (bitField0_ & ~0x00000008); - } else { - ensurePartitionsIsMutable(); - partitions_.addAll(other.partitions_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 34: { - ensurePartitionsIsMutable(); - partitions_.add(input.readBytes()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // repeated string partitions = 4; - private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - private void ensurePartitionsIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { - partitions_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(partitions_); - bitField0_ |= 0x00000008; - } - } - public java.util.List - getPartitionsList() { - return java.util.Collections.unmodifiableList(partitions_); - } - public int getPartitionsCount() { - return partitions_.size(); - } - public String getPartitions(int index) { - return partitions_.get(index); - } - public Builder setPartitions( - int index, String value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePartitionsIsMutable(); - partitions_.set(index, value); - - return this; - } - public Builder addPartitions(String value) { - if (value == null) { - throw new NullPointerException(); - } - ensurePartitionsIsMutable(); - partitions_.add(value); - - return this; - } - public Builder addAllPartitions( - java.lang.Iterable values) { - ensurePartitionsIsMutable(); - super.addAll(values, partitions_); - - return this; - } - public Builder clearPartitions() { - partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; - bitField0_ = (bitField0_ & ~0x00000008); - - return this; - } - void addPartitions(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - ensurePartitionsIsMutable(); - partitions_.add(value); - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddPartitionToTxn) - } - - static { - defaultInstance = new CommandAddPartitionToTxn(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddPartitionToTxn) - } - - public interface CommandAddPartitionToTxnResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandAddPartitionToTxnResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAddPartitionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAddPartitionToTxnResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAddPartitionToTxnResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAddPartitionToTxnResponse newObject(Handle handle) { - return new CommandAddPartitionToTxnResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAddPartitionToTxnResponse(boolean noInit) {} - - private static final CommandAddPartitionToTxnResponse defaultInstance; - public static CommandAddPartitionToTxnResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandAddPartitionToTxnResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddPartitionToTxnResponse) - } - - static { - defaultInstance = new CommandAddPartitionToTxnResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddPartitionToTxnResponse) - } - - public interface SubscriptionOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required string topic = 1; - boolean hasTopic(); - String getTopic(); - - // required string subscription = 2; - boolean hasSubscription(); - String getSubscription(); - } - public static final class Subscription extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements SubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use Subscription.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private Subscription(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected Subscription newObject(Handle handle) { - return new Subscription(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private Subscription(boolean noInit) {} - - private static final Subscription defaultInstance; - public static Subscription getDefaultInstance() { - return defaultInstance; - } - - public Subscription getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required string topic = 1; - public static final int TOPIC_FIELD_NUMBER = 1; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // required string subscription = 2; - public static final int SUBSCRIPTION_FIELD_NUMBER = 2; - private java.lang.Object subscription_; - public boolean hasSubscription() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getSubscription() { - java.lang.Object ref = subscription_; - 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)) { - subscription_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSubscriptionBytes() { - java.lang.Object ref = subscription_; - 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); - subscription_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - topic_ = ""; - subscription_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasTopic()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasSubscription()) { - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeBytes(2, getSubscriptionBytes()); - } - } - - 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, getTopicBytes()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(2, getSubscriptionBytes()); - } - 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.SubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.Subscription.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(); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000001); - subscription_ = ""; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription build() { - org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.Subscription buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.subscription_ = subscription_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.Subscription other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance()) return this; - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasSubscription()) { - setSubscription(other.getSubscription()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasTopic()) { - - return false; - } - if (!hasSubscription()) { - - 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; - topic_ = input.readBytes(); - break; - } - case 18: { - bitField0_ |= 0x00000002; - subscription_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required string topic = 1; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000001); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000001; - topic_ = value; - - } - - // required string subscription = 2; - private java.lang.Object subscription_ = ""; - public boolean hasSubscription() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public String getSubscription() { - java.lang.Object ref = subscription_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - subscription_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setSubscription(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - subscription_ = value; - - return this; - } - public Builder clearSubscription() { - bitField0_ = (bitField0_ & ~0x00000002); - subscription_ = getDefaultInstance().getSubscription(); - - return this; - } - void setSubscription(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000002; - subscription_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.Subscription) - } - - static { - defaultInstance = new Subscription(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.Subscription) - } - - public interface CommandAddSubscriptionToTxnOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // repeated .pulsar.proto.Subscription subscription = 4; - java.util.List - getSubscriptionList(); - org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index); - int getSubscriptionCount(); - } - public static final class CommandAddSubscriptionToTxn extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAddSubscriptionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAddSubscriptionToTxn.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAddSubscriptionToTxn(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAddSubscriptionToTxn newObject(Handle handle) { - return new CommandAddSubscriptionToTxn(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAddSubscriptionToTxn(boolean noInit) {} - - private static final CommandAddSubscriptionToTxn defaultInstance; - public static CommandAddSubscriptionToTxn getDefaultInstance() { - return defaultInstance; - } - - public CommandAddSubscriptionToTxn getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // repeated .pulsar.proto.Subscription subscription = 4; - public static final int SUBSCRIPTION_FIELD_NUMBER = 4; - private java.util.List subscription_; - public java.util.List getSubscriptionList() { - return subscription_; - } - public java.util.List - getSubscriptionOrBuilderList() { - return subscription_; - } - public int getSubscriptionCount() { - return subscription_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index) { - return subscription_.get(index); - } - public org.apache.pulsar.common.api.proto.PulsarApi.SubscriptionOrBuilder getSubscriptionOrBuilder( - int index) { - return subscription_.get(index); - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - subscription_ = java.util.Collections.emptyList(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - for (int i = 0; i < getSubscriptionCount(); i++) { - if (!getSubscription(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.writeUInt64(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - for (int i = 0; i < subscription_.size(); i++) { - output.writeMessage(4, subscription_.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 - .computeUInt64Size(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - for (int i = 0; i < subscription_.size(); i++) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, subscription_.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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - subscription_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((bitField0_ & 0x00000008) == 0x00000008)) { - subscription_ = java.util.Collections.unmodifiableList(subscription_); - bitField0_ = (bitField0_ & ~0x00000008); - } - result.subscription_ = subscription_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (!other.subscription_.isEmpty()) { - if (subscription_.isEmpty()) { - subscription_ = other.subscription_; - bitField0_ = (bitField0_ & ~0x00000008); - } else { - ensureSubscriptionIsMutable(); - subscription_.addAll(other.subscription_); - } - - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - for (int i = 0; i < getSubscriptionCount(); i++) { - if (!getSubscription(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 8: { - bitField0_ |= 0x00000001; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(); - input.readMessage(subBuilder, extensionRegistry); - addSubscription(subBuilder.buildPartial()); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // repeated .pulsar.proto.Subscription subscription = 4; - private java.util.List subscription_ = - java.util.Collections.emptyList(); - private void ensureSubscriptionIsMutable() { - if (!((bitField0_ & 0x00000008) == 0x00000008)) { - subscription_ = new java.util.ArrayList(subscription_); - bitField0_ |= 0x00000008; - } - } - - public java.util.List getSubscriptionList() { - return java.util.Collections.unmodifiableList(subscription_); - } - public int getSubscriptionCount() { - return subscription_.size(); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index) { - return subscription_.get(index); - } - public Builder setSubscription( - int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { - if (value == null) { - throw new NullPointerException(); - } - ensureSubscriptionIsMutable(); - subscription_.set(index, value); - - return this; - } - public Builder setSubscription( - int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { - ensureSubscriptionIsMutable(); - subscription_.set(index, builderForValue.build()); - - return this; - } - public Builder addSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { - if (value == null) { - throw new NullPointerException(); - } - ensureSubscriptionIsMutable(); - subscription_.add(value); - - return this; - } - public Builder addSubscription( - int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { - if (value == null) { - throw new NullPointerException(); - } - ensureSubscriptionIsMutable(); - subscription_.add(index, value); - - return this; - } - public Builder addSubscription( - org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { - ensureSubscriptionIsMutable(); - subscription_.add(builderForValue.build()); - - return this; - } - public Builder addSubscription( - int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { - ensureSubscriptionIsMutable(); - subscription_.add(index, builderForValue.build()); - - return this; - } - public Builder addAllSubscription( - java.lang.Iterable values) { - ensureSubscriptionIsMutable(); - super.addAll(values, subscription_); - - return this; - } - public Builder clearSubscription() { - subscription_ = java.util.Collections.emptyList(); - bitField0_ = (bitField0_ & ~0x00000008); - - return this; - } - public Builder removeSubscription(int index) { - ensureSubscriptionIsMutable(); - subscription_.remove(index); - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddSubscriptionToTxn) - } - - static { - defaultInstance = new CommandAddSubscriptionToTxn(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddSubscriptionToTxn) - } - - public interface CommandAddSubscriptionToTxnResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandAddSubscriptionToTxnResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandAddSubscriptionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandAddSubscriptionToTxnResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandAddSubscriptionToTxnResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandAddSubscriptionToTxnResponse newObject(Handle handle) { - return new CommandAddSubscriptionToTxnResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandAddSubscriptionToTxnResponse(boolean noInit) {} - - private static final CommandAddSubscriptionToTxnResponse defaultInstance; - public static CommandAddSubscriptionToTxnResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandAddSubscriptionToTxnResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddSubscriptionToTxnResponse) - } - - static { - defaultInstance = new CommandAddSubscriptionToTxnResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddSubscriptionToTxnResponse) - } - - public interface CommandEndTxnOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.TxnAction txn_action = 4; - boolean hasTxnAction(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); - } - public static final class CommandEndTxn extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxn.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxn(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxn newObject(Handle handle) { - return new CommandEndTxn(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxn(boolean noInit) {} - - private static final CommandEndTxn defaultInstance; - public static CommandEndTxn getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxn getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.TxnAction txn_action = 4; - public static final int TXN_ACTION_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, txnAction_.getNumber()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, txnAction_.getNumber()); - } - 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.txnAction_ = txnAction_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasTxnAction()) { - setTxnAction(other.getTxnAction()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - txnAction_ = value; - } - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.TxnAction txn_action = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - txnAction_ = value; - - return this; - } - public Builder clearTxnAction() { - bitField0_ = (bitField0_ & ~0x00000008); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxn) - } - - static { - defaultInstance = new CommandEndTxn(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxn) - } - - public interface CommandEndTxnResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandEndTxnResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxnResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxnResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxnResponse newObject(Handle handle) { - return new CommandEndTxnResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxnResponse(boolean noInit) {} - - private static final CommandEndTxnResponse defaultInstance; - public static CommandEndTxnResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxnResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnResponse) - } - - static { - defaultInstance = new CommandEndTxnResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnResponse) - } - - public interface CommandEndTxnOnPartitionOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional string topic = 4; - boolean hasTopic(); - String getTopic(); - - // optional .pulsar.proto.TxnAction txn_action = 5; - boolean hasTxnAction(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); - } - public static final class CommandEndTxnOnPartition extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnOnPartitionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxnOnPartition.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxnOnPartition(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxnOnPartition newObject(Handle handle) { - return new CommandEndTxnOnPartition(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxnOnPartition(boolean noInit) {} - - private static final CommandEndTxnOnPartition defaultInstance; - public static CommandEndTxnOnPartition getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxnOnPartition getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional string topic = 4; - public static final int TOPIC_FIELD_NUMBER = 4; - private java.lang.Object topic_; - public boolean hasTopic() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getTopic() { - java.lang.Object ref = topic_; - 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)) { - topic_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { - java.lang.Object ref = topic_; - 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); - topic_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - // optional .pulsar.proto.TxnAction txn_action = 5; - public static final int TXN_ACTION_FIELD_NUMBER = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - topic_ = ""; - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeBytes(4, getTopicBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeEnum(5, txnAction_.getNumber()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(4, getTopicBytes()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(5, txnAction_.getNumber()); - } - 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - topic_ = ""; - bitField0_ = (bitField0_ & ~0x00000008); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.topic_ = topic_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.txnAction_ = txnAction_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasTopic()) { - setTopic(other.getTopic()); - } - if (other.hasTxnAction()) { - setTxnAction(other.getTxnAction()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 34: { - bitField0_ |= 0x00000008; - topic_ = input.readBytes(); - break; - } - case 40: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000010; - txnAction_ = value; - } - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional string topic = 4; - private java.lang.Object topic_ = ""; - public boolean hasTopic() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public String getTopic() { - java.lang.Object ref = topic_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - topic_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setTopic(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - topic_ = value; - - return this; - } - public Builder clearTopic() { - bitField0_ = (bitField0_ & ~0x00000008); - topic_ = getDefaultInstance().getTopic(); - - return this; - } - void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000008; - topic_ = value; - - } - - // optional .pulsar.proto.TxnAction txn_action = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - txnAction_ = value; - - return this; - } - public Builder clearTxnAction() { - bitField0_ = (bitField0_ & ~0x00000010); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnPartition) - } - - static { - defaultInstance = new CommandEndTxnOnPartition(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnPartition) - } - - public interface CommandEndTxnOnPartitionResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandEndTxnOnPartitionResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnOnPartitionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxnOnPartitionResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxnOnPartitionResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxnOnPartitionResponse newObject(Handle handle) { - return new CommandEndTxnOnPartitionResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxnOnPartitionResponse(boolean noInit) {} - - private static final CommandEndTxnOnPartitionResponse defaultInstance; - public static CommandEndTxnOnPartitionResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxnOnPartitionResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnPartitionResponse) - } - - static { - defaultInstance = new CommandEndTxnOnPartitionResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnPartitionResponse) - } - - public interface CommandEndTxnOnSubscriptionOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.Subscription subscription = 4; - boolean hasSubscription(); - org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(); - - // optional .pulsar.proto.TxnAction txn_action = 5; - boolean hasTxnAction(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); - } - public static final class CommandEndTxnOnSubscription extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnOnSubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxnOnSubscription.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxnOnSubscription(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxnOnSubscription newObject(Handle handle) { - return new CommandEndTxnOnSubscription(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxnOnSubscription(boolean noInit) {} - - private static final CommandEndTxnOnSubscription defaultInstance; - public static CommandEndTxnOnSubscription getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxnOnSubscription getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.Subscription subscription = 4; - public static final int SUBSCRIPTION_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Subscription subscription_; - public boolean hasSubscription() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription() { - return subscription_; - } - - // optional .pulsar.proto.TxnAction txn_action = 5; - public static final int TXN_ACTION_FIELD_NUMBER = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - memoizedIsInitialized = 0; - return false; - } - if (hasSubscription()) { - if (!getSubscription().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.writeUInt64(1, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeMessage(4, subscription_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeEnum(5, txnAction_.getNumber()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, subscription_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(5, txnAction_.getNumber()); - } - 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000008); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.subscription_ = subscription_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.txnAction_ = txnAction_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasSubscription()) { - mergeSubscription(other.getSubscription()); - } - if (other.hasTxnAction()) { - setTxnAction(other.getTxnAction()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - return false; - } - if (hasSubscription()) { - if (!getSubscription().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 8: { - bitField0_ |= 0x00000001; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(); - if (hasSubscription()) { - subBuilder.mergeFrom(getSubscription()); - } - input.readMessage(subBuilder, extensionRegistry); - setSubscription(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 40: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000010; - txnAction_ = value; - } - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.Subscription subscription = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.Subscription subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); - public boolean hasSubscription() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription() { - return subscription_; - } - public Builder setSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { - if (value == null) { - throw new NullPointerException(); - } - subscription_ = value; - - bitField0_ |= 0x00000008; - return this; - } - public Builder setSubscription( - org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { - subscription_ = builderForValue.build(); - - bitField0_ |= 0x00000008; - return this; - } - public Builder mergeSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { - if (((bitField0_ & 0x00000008) == 0x00000008) && - subscription_ != org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance()) { - subscription_ = - org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(subscription_).mergeFrom(value).buildPartial(); - } else { - subscription_ = value; - } - - bitField0_ |= 0x00000008; - return this; - } - public Builder clearSubscription() { - subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - // optional .pulsar.proto.TxnAction txn_action = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - public boolean hasTxnAction() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { - return txnAction_; - } - public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - txnAction_ = value; - - return this; - } - public Builder clearTxnAction() { - bitField0_ = (bitField0_ & ~0x00000010); - txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; - - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnSubscription) - } - - static { - defaultInstance = new CommandEndTxnOnSubscription(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnSubscription) - } - - public interface CommandEndTxnOnSubscriptionResponseOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required uint64 request_id = 1; - boolean hasRequestId(); - long getRequestId(); - - // optional uint64 txnid_least_bits = 2 [default = 0]; - boolean hasTxnidLeastBits(); - long getTxnidLeastBits(); - - // optional uint64 txnid_most_bits = 3 [default = 0]; - boolean hasTxnidMostBits(); - long getTxnidMostBits(); - - // optional .pulsar.proto.ServerError error = 4; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); - - // optional string message = 5; - boolean hasMessage(); - String getMessage(); - } - public static final class CommandEndTxnOnSubscriptionResponse extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements CommandEndTxnOnSubscriptionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use CommandEndTxnOnSubscriptionResponse.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private CommandEndTxnOnSubscriptionResponse(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected CommandEndTxnOnSubscriptionResponse newObject(Handle handle) { - return new CommandEndTxnOnSubscriptionResponse(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private CommandEndTxnOnSubscriptionResponse(boolean noInit) {} - - private static final CommandEndTxnOnSubscriptionResponse defaultInstance; - public static CommandEndTxnOnSubscriptionResponse getDefaultInstance() { - return defaultInstance; - } - - public CommandEndTxnOnSubscriptionResponse getDefaultInstanceForType() { - return defaultInstance; - } - - private int bitField0_; - // required uint64 request_id = 1; - public static final int REQUEST_ID_FIELD_NUMBER = 1; - private long requestId_; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; - private long txnidLeastBits_; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; - private long txnidMostBits_; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - - // optional .pulsar.proto.ServerError error = 4; - public static final int ERROR_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - - // optional string message = 5; - public static final int MESSAGE_FIELD_NUMBER = 5; - private java.lang.Object message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - 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)) { - message_ = s; - } - return s; - } - } - private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { - java.lang.Object ref = message_; - 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); - message_ = b; - return b; - } else { - return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; - } - } - - private void initFields() { - requestId_ = 0L; - txnidLeastBits_ = 0L; - txnidMostBits_ = 0L; - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - message_ = ""; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRequestId()) { - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeEnum(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeBytes(5, getMessageBytes()); - } - } - - 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, requestId_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(2, txnidLeastBits_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeUInt64Size(3, txnidMostBits_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeEnumSize(4, error_.getNumber()); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeBytesSize(5, getMessageBytes()); - } - 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.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(); - requestId_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - txnidLeastBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - txnidMostBits_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - bitField0_ = (bitField0_ & ~0x00000008); - message_ = ""; - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse build() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.requestId_ = requestId_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.txnidLeastBits_ = txnidLeastBits_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.txnidMostBits_ = txnidMostBits_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.message_ = message_; - result.bitField0_ = to_bitField0_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance()) return this; - if (other.hasRequestId()) { - setRequestId(other.getRequestId()); - } - if (other.hasTxnidLeastBits()) { - setTxnidLeastBits(other.getTxnidLeastBits()); - } - if (other.hasTxnidMostBits()) { - setTxnidMostBits(other.getTxnidMostBits()); - } - if (other.hasError()) { - setError(other.getError()); - } - if (other.hasMessage()) { - setMessage(other.getMessage()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasRequestId()) { - - 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; - requestId_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - txnidLeastBits_ = input.readUInt64(); - break; - } - case 24: { - bitField0_ |= 0x00000004; - txnidMostBits_ = input.readUInt64(); - break; - } - case 32: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000008; - error_ = value; - } - break; - } - case 42: { - bitField0_ |= 0x00000010; - message_ = input.readBytes(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 request_id = 1; - private long requestId_ ; - public boolean hasRequestId() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRequestId() { - return requestId_; - } - public Builder setRequestId(long value) { - bitField0_ |= 0x00000001; - requestId_ = value; - - return this; - } - public Builder clearRequestId() { - bitField0_ = (bitField0_ & ~0x00000001); - requestId_ = 0L; - - return this; - } - - // optional uint64 txnid_least_bits = 2 [default = 0]; - private long txnidLeastBits_ ; - public boolean hasTxnidLeastBits() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getTxnidLeastBits() { - return txnidLeastBits_; - } - public Builder setTxnidLeastBits(long value) { - bitField0_ |= 0x00000002; - txnidLeastBits_ = value; - - return this; - } - public Builder clearTxnidLeastBits() { - bitField0_ = (bitField0_ & ~0x00000002); - txnidLeastBits_ = 0L; - - return this; - } - - // optional uint64 txnid_most_bits = 3 [default = 0]; - private long txnidMostBits_ ; - public boolean hasTxnidMostBits() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTxnidMostBits() { - return txnidMostBits_; - } - public Builder setTxnidMostBits(long value) { - bitField0_ |= 0x00000004; - txnidMostBits_ = value; - - return this; - } - public Builder clearTxnidMostBits() { - bitField0_ = (bitField0_ & ~0x00000004); - txnidMostBits_ = 0L; - - return this; - } - - // optional .pulsar.proto.ServerError error = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - public boolean hasError() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000008; - error_ = value; - - return this; - } - public Builder clearError() { - bitField0_ = (bitField0_ & ~0x00000008); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; - - return this; - } - - // optional string message = 5; - private java.lang.Object message_ = ""; - public boolean hasMessage() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public String getMessage() { - java.lang.Object ref = message_; - if (!(ref instanceof String)) { - String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); - message_ = s; - return s; - } else { - return (String) ref; - } - } - public Builder setMessage(String value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000010; - message_ = value; - - return this; - } - public Builder clearMessage() { - bitField0_ = (bitField0_ & ~0x00000010); - message_ = getDefaultInstance().getMessage(); - - return this; - } - void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { - bitField0_ |= 0x00000010; - message_ = value; - - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnSubscriptionResponse) - } - - static { - defaultInstance = new CommandEndTxnOnSubscriptionResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnSubscriptionResponse) - } - - public interface BaseCommandOrBuilder - extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { - - // required .pulsar.proto.BaseCommand.Type type = 1; - boolean hasType(); - org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType(); - - // optional .pulsar.proto.CommandConnect connect = 2; - boolean hasConnect(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect(); - - // optional .pulsar.proto.CommandConnected connected = 3; - boolean hasConnected(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected(); - - // optional .pulsar.proto.CommandSubscribe subscribe = 4; - boolean hasSubscribe(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe(); - - // optional .pulsar.proto.CommandProducer producer = 5; - boolean hasProducer(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer(); - - // optional .pulsar.proto.CommandSend send = 6; - boolean hasSend(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend(); - - // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; - boolean hasSendReceipt(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt(); - - // optional .pulsar.proto.CommandSendError send_error = 8; - boolean hasSendError(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError(); - - // optional .pulsar.proto.CommandMessage message = 9; - boolean hasMessage(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage(); - - // optional .pulsar.proto.CommandAck ack = 10; - boolean hasAck(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck(); - - // optional .pulsar.proto.CommandFlow flow = 11; - boolean hasFlow(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow(); - - // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; - boolean hasUnsubscribe(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe(); - - // optional .pulsar.proto.CommandSuccess success = 13; - boolean hasSuccess(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess(); - - // optional .pulsar.proto.CommandError error = 14; - boolean hasError(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError(); - - // optional .pulsar.proto.CommandCloseProducer close_producer = 15; - boolean hasCloseProducer(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer(); - - // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; - boolean hasCloseConsumer(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer(); - - // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; - boolean hasProducerSuccess(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess(); - - // optional .pulsar.proto.CommandPing ping = 18; - boolean hasPing(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing(); - - // optional .pulsar.proto.CommandPong pong = 19; - boolean hasPong(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong(); - - // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; - boolean hasRedeliverUnacknowledgedMessages(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages(); - - // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; - boolean hasPartitionMetadata(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata(); - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; - boolean hasPartitionMetadataResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse(); - - // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; - boolean hasLookupTopic(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic(); - - // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; - boolean hasLookupTopicResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse(); - - // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; - boolean hasConsumerStats(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats(); - - // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; - boolean hasConsumerStatsResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse(); - - // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; - boolean hasReachedEndOfTopic(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic(); - - // optional .pulsar.proto.CommandSeek seek = 28; - boolean hasSeek(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek(); - - // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; - boolean hasGetLastMessageId(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId(); - - // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; - boolean hasGetLastMessageIdResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse(); - - // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; - boolean hasActiveConsumerChange(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange(); - - // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; - boolean hasGetTopicsOfNamespace(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace(); - - // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; - boolean hasGetTopicsOfNamespaceResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse(); - - // optional .pulsar.proto.CommandGetSchema getSchema = 34; - boolean hasGetSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema(); - - // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; - boolean hasGetSchemaResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse(); - - // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; - boolean hasAuthChallenge(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge(); - - // optional .pulsar.proto.CommandAuthResponse authResponse = 37; - boolean hasAuthResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse(); - - // optional .pulsar.proto.CommandAckResponse ackResponse = 38; - boolean hasAckResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse(); - - // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; - boolean hasGetOrCreateSchema(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema(); - - // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; - boolean hasGetOrCreateSchemaResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse(); - - // optional .pulsar.proto.CommandNewTxn newTxn = 50; - boolean hasNewTxn(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn(); - - // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; - boolean hasNewTxnResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse(); - - // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; - boolean hasAddPartitionToTxn(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn(); - - // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; - boolean hasAddPartitionToTxnResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse(); - - // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; - boolean hasAddSubscriptionToTxn(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn(); - - // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; - boolean hasAddSubscriptionToTxnResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse(); - - // optional .pulsar.proto.CommandEndTxn endTxn = 56; - boolean hasEndTxn(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn(); - - // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; - boolean hasEndTxnResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse(); - - // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; - boolean hasEndTxnOnPartition(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition(); - - // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; - boolean hasEndTxnOnPartitionResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse(); - - // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; - boolean hasEndTxnOnSubscription(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription(); - - // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; - boolean hasEndTxnOnSubscriptionResponse(); - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse(); - } - public static final class BaseCommand extends - org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite - implements BaseCommandOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { - // Use BaseCommand.newBuilder() to construct. - private io.netty.util.Recycler.Handle handle; - private BaseCommand(io.netty.util.Recycler.Handle handle) { - this.handle = handle; - } - - private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { - protected BaseCommand newObject(Handle handle) { - return new BaseCommand(handle); - } - }; - - public void recycle() { - this.initFields(); - this.memoizedIsInitialized = -1; - this.bitField0_ = 0; - this.bitField1_ = 0; - this.memoizedSerializedSize = -1; - if (handle != null) { RECYCLER.recycle(this, handle); } - } - - private BaseCommand(boolean noInit) {} - - private static final BaseCommand defaultInstance; - public static BaseCommand getDefaultInstance() { - return defaultInstance; - } - - public BaseCommand getDefaultInstanceForType() { - return defaultInstance; - } - - public enum Type - implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { - CONNECT(0, 2), - CONNECTED(1, 3), - SUBSCRIBE(2, 4), - PRODUCER(3, 5), - SEND(4, 6), - SEND_RECEIPT(5, 7), - SEND_ERROR(6, 8), - MESSAGE(7, 9), - ACK(8, 10), - FLOW(9, 11), - UNSUBSCRIBE(10, 12), - SUCCESS(11, 13), - ERROR(12, 14), - CLOSE_PRODUCER(13, 15), - CLOSE_CONSUMER(14, 16), - PRODUCER_SUCCESS(15, 17), - PING(16, 18), - PONG(17, 19), - REDELIVER_UNACKNOWLEDGED_MESSAGES(18, 20), - PARTITIONED_METADATA(19, 21), - PARTITIONED_METADATA_RESPONSE(20, 22), - LOOKUP(21, 23), - LOOKUP_RESPONSE(22, 24), - CONSUMER_STATS(23, 25), - CONSUMER_STATS_RESPONSE(24, 26), - REACHED_END_OF_TOPIC(25, 27), - SEEK(26, 28), - GET_LAST_MESSAGE_ID(27, 29), - GET_LAST_MESSAGE_ID_RESPONSE(28, 30), - ACTIVE_CONSUMER_CHANGE(29, 31), - GET_TOPICS_OF_NAMESPACE(30, 32), - GET_TOPICS_OF_NAMESPACE_RESPONSE(31, 33), - GET_SCHEMA(32, 34), - GET_SCHEMA_RESPONSE(33, 35), - AUTH_CHALLENGE(34, 36), - AUTH_RESPONSE(35, 37), - ACK_RESPONSE(36, 38), - GET_OR_CREATE_SCHEMA(37, 39), - GET_OR_CREATE_SCHEMA_RESPONSE(38, 40), - NEW_TXN(39, 50), - NEW_TXN_RESPONSE(40, 51), - ADD_PARTITION_TO_TXN(41, 52), - ADD_PARTITION_TO_TXN_RESPONSE(42, 53), - ADD_SUBSCRIPTION_TO_TXN(43, 54), - ADD_SUBSCRIPTION_TO_TXN_RESPONSE(44, 55), - END_TXN(45, 56), - END_TXN_RESPONSE(46, 57), - END_TXN_ON_PARTITION(47, 58), - END_TXN_ON_PARTITION_RESPONSE(48, 59), - END_TXN_ON_SUBSCRIPTION(49, 60), - END_TXN_ON_SUBSCRIPTION_RESPONSE(50, 61), - ; - - public static final int CONNECT_VALUE = 2; - public static final int CONNECTED_VALUE = 3; - public static final int SUBSCRIBE_VALUE = 4; - public static final int PRODUCER_VALUE = 5; - public static final int SEND_VALUE = 6; - public static final int SEND_RECEIPT_VALUE = 7; - public static final int SEND_ERROR_VALUE = 8; - public static final int MESSAGE_VALUE = 9; - public static final int ACK_VALUE = 10; - public static final int FLOW_VALUE = 11; - public static final int UNSUBSCRIBE_VALUE = 12; - public static final int SUCCESS_VALUE = 13; - public static final int ERROR_VALUE = 14; - public static final int CLOSE_PRODUCER_VALUE = 15; - public static final int CLOSE_CONSUMER_VALUE = 16; - public static final int PRODUCER_SUCCESS_VALUE = 17; - public static final int PING_VALUE = 18; - public static final int PONG_VALUE = 19; - public static final int REDELIVER_UNACKNOWLEDGED_MESSAGES_VALUE = 20; - public static final int PARTITIONED_METADATA_VALUE = 21; - public static final int PARTITIONED_METADATA_RESPONSE_VALUE = 22; - public static final int LOOKUP_VALUE = 23; - public static final int LOOKUP_RESPONSE_VALUE = 24; - public static final int CONSUMER_STATS_VALUE = 25; - public static final int CONSUMER_STATS_RESPONSE_VALUE = 26; - public static final int REACHED_END_OF_TOPIC_VALUE = 27; - public static final int SEEK_VALUE = 28; - public static final int GET_LAST_MESSAGE_ID_VALUE = 29; - public static final int GET_LAST_MESSAGE_ID_RESPONSE_VALUE = 30; - public static final int ACTIVE_CONSUMER_CHANGE_VALUE = 31; - public static final int GET_TOPICS_OF_NAMESPACE_VALUE = 32; - public static final int GET_TOPICS_OF_NAMESPACE_RESPONSE_VALUE = 33; - public static final int GET_SCHEMA_VALUE = 34; - public static final int GET_SCHEMA_RESPONSE_VALUE = 35; - public static final int AUTH_CHALLENGE_VALUE = 36; - public static final int AUTH_RESPONSE_VALUE = 37; - public static final int ACK_RESPONSE_VALUE = 38; - public static final int GET_OR_CREATE_SCHEMA_VALUE = 39; - public static final int GET_OR_CREATE_SCHEMA_RESPONSE_VALUE = 40; - public static final int NEW_TXN_VALUE = 50; - public static final int NEW_TXN_RESPONSE_VALUE = 51; - public static final int ADD_PARTITION_TO_TXN_VALUE = 52; - public static final int ADD_PARTITION_TO_TXN_RESPONSE_VALUE = 53; - public static final int ADD_SUBSCRIPTION_TO_TXN_VALUE = 54; - public static final int ADD_SUBSCRIPTION_TO_TXN_RESPONSE_VALUE = 55; - public static final int END_TXN_VALUE = 56; - public static final int END_TXN_RESPONSE_VALUE = 57; - public static final int END_TXN_ON_PARTITION_VALUE = 58; - public static final int END_TXN_ON_PARTITION_RESPONSE_VALUE = 59; - public static final int END_TXN_ON_SUBSCRIPTION_VALUE = 60; - public static final int END_TXN_ON_SUBSCRIPTION_RESPONSE_VALUE = 61; - - - public final int getNumber() { return value; } - - public static Type valueOf(int value) { - switch (value) { - case 2: return CONNECT; - case 3: return CONNECTED; - case 4: return SUBSCRIBE; - case 5: return PRODUCER; - case 6: return SEND; - case 7: return SEND_RECEIPT; - case 8: return SEND_ERROR; - case 9: return MESSAGE; - case 10: return ACK; - case 11: return FLOW; - case 12: return UNSUBSCRIBE; - case 13: return SUCCESS; - case 14: return ERROR; - case 15: return CLOSE_PRODUCER; - case 16: return CLOSE_CONSUMER; - case 17: return PRODUCER_SUCCESS; - case 18: return PING; - case 19: return PONG; - case 20: return REDELIVER_UNACKNOWLEDGED_MESSAGES; - case 21: return PARTITIONED_METADATA; - case 22: return PARTITIONED_METADATA_RESPONSE; - case 23: return LOOKUP; - case 24: return LOOKUP_RESPONSE; - case 25: return CONSUMER_STATS; - case 26: return CONSUMER_STATS_RESPONSE; - case 27: return REACHED_END_OF_TOPIC; - case 28: return SEEK; - case 29: return GET_LAST_MESSAGE_ID; - case 30: return GET_LAST_MESSAGE_ID_RESPONSE; - case 31: return ACTIVE_CONSUMER_CHANGE; - case 32: return GET_TOPICS_OF_NAMESPACE; - case 33: return GET_TOPICS_OF_NAMESPACE_RESPONSE; - case 34: return GET_SCHEMA; - case 35: return GET_SCHEMA_RESPONSE; - case 36: return AUTH_CHALLENGE; - case 37: return AUTH_RESPONSE; - case 38: return ACK_RESPONSE; - case 39: return GET_OR_CREATE_SCHEMA; - case 40: return GET_OR_CREATE_SCHEMA_RESPONSE; - case 50: return NEW_TXN; - case 51: return NEW_TXN_RESPONSE; - case 52: return ADD_PARTITION_TO_TXN; - case 53: return ADD_PARTITION_TO_TXN_RESPONSE; - case 54: return ADD_SUBSCRIPTION_TO_TXN; - case 55: return ADD_SUBSCRIPTION_TO_TXN_RESPONSE; - case 56: return END_TXN; - case 57: return END_TXN_RESPONSE; - case 58: return END_TXN_ON_PARTITION; - case 59: return END_TXN_ON_PARTITION_RESPONSE; - case 60: return END_TXN_ON_SUBSCRIPTION; - case 61: return END_TXN_ON_SUBSCRIPTION_RESPONSE; - 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 Type findValueByNumber(int number) { - return Type.valueOf(number); - } - }; - - private final int value; - - private Type(int index, int value) { - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:pulsar.proto.BaseCommand.Type) - } - - private int bitField0_; - private int bitField1_; - // required .pulsar.proto.BaseCommand.Type type = 1; - public static final int TYPE_FIELD_NUMBER = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type type_; - public boolean hasType() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType() { - return type_; - } - - // optional .pulsar.proto.CommandConnect connect = 2; - public static final int CONNECT_FIELD_NUMBER = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect connect_; - public boolean hasConnect() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect() { - return connect_; - } - - // optional .pulsar.proto.CommandConnected connected = 3; - public static final int CONNECTED_FIELD_NUMBER = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected connected_; - public boolean hasConnected() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected() { - return connected_; - } - - // optional .pulsar.proto.CommandSubscribe subscribe = 4; - public static final int SUBSCRIBE_FIELD_NUMBER = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe subscribe_; - public boolean hasSubscribe() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe() { - return subscribe_; - } - - // optional .pulsar.proto.CommandProducer producer = 5; - public static final int PRODUCER_FIELD_NUMBER = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer producer_; - public boolean hasProducer() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer() { - return producer_; - } - - // optional .pulsar.proto.CommandSend send = 6; - public static final int SEND_FIELD_NUMBER = 6; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend send_; - public boolean hasSend() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend() { - return send_; - } - - // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; - public static final int SEND_RECEIPT_FIELD_NUMBER = 7; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt sendReceipt_; - public boolean hasSendReceipt() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt() { - return sendReceipt_; - } - - // optional .pulsar.proto.CommandSendError send_error = 8; - public static final int SEND_ERROR_FIELD_NUMBER = 8; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError sendError_; - public boolean hasSendError() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError() { - return sendError_; - } - - // optional .pulsar.proto.CommandMessage message = 9; - public static final int MESSAGE_FIELD_NUMBER = 9; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage message_; - public boolean hasMessage() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage() { - return message_; - } - - // optional .pulsar.proto.CommandAck ack = 10; - public static final int ACK_FIELD_NUMBER = 10; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck ack_; - public boolean hasAck() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck() { - return ack_; - } - - // optional .pulsar.proto.CommandFlow flow = 11; - public static final int FLOW_FIELD_NUMBER = 11; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow flow_; - public boolean hasFlow() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow() { - return flow_; - } - - // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; - public static final int UNSUBSCRIBE_FIELD_NUMBER = 12; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe unsubscribe_; - public boolean hasUnsubscribe() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe() { - return unsubscribe_; - } - - // optional .pulsar.proto.CommandSuccess success = 13; - public static final int SUCCESS_FIELD_NUMBER = 13; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess success_; - public boolean hasSuccess() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess() { - return success_; - } - - // optional .pulsar.proto.CommandError error = 14; - public static final int ERROR_FIELD_NUMBER = 14; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandError error_; - public boolean hasError() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError() { - return error_; - } - - // optional .pulsar.proto.CommandCloseProducer close_producer = 15; - public static final int CLOSE_PRODUCER_FIELD_NUMBER = 15; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer closeProducer_; - public boolean hasCloseProducer() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer() { - return closeProducer_; - } - - // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; - public static final int CLOSE_CONSUMER_FIELD_NUMBER = 16; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer closeConsumer_; - public boolean hasCloseConsumer() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer() { - return closeConsumer_; - } - - // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; - public static final int PRODUCER_SUCCESS_FIELD_NUMBER = 17; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess producerSuccess_; - public boolean hasProducerSuccess() { - return ((bitField0_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess() { - return producerSuccess_; - } - - // optional .pulsar.proto.CommandPing ping = 18; - public static final int PING_FIELD_NUMBER = 18; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing ping_; - public boolean hasPing() { - return ((bitField0_ & 0x00020000) == 0x00020000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing() { - return ping_; - } - - // optional .pulsar.proto.CommandPong pong = 19; - public static final int PONG_FIELD_NUMBER = 19; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong pong_; - public boolean hasPong() { - return ((bitField0_ & 0x00040000) == 0x00040000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong() { - return pong_; - } - - // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; - public static final int REDELIVERUNACKNOWLEDGEDMESSAGES_FIELD_NUMBER = 20; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages_; - public boolean hasRedeliverUnacknowledgedMessages() { - return ((bitField0_ & 0x00080000) == 0x00080000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages() { - return redeliverUnacknowledgedMessages_; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; - public static final int PARTITIONMETADATA_FIELD_NUMBER = 21; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata partitionMetadata_; - public boolean hasPartitionMetadata() { - return ((bitField0_ & 0x00100000) == 0x00100000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata() { - return partitionMetadata_; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; - public static final int PARTITIONMETADATARESPONSE_FIELD_NUMBER = 22; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse partitionMetadataResponse_; - public boolean hasPartitionMetadataResponse() { - return ((bitField0_ & 0x00200000) == 0x00200000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse() { - return partitionMetadataResponse_; - } - - // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; - public static final int LOOKUPTOPIC_FIELD_NUMBER = 23; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic lookupTopic_; - public boolean hasLookupTopic() { - return ((bitField0_ & 0x00400000) == 0x00400000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic() { - return lookupTopic_; - } - - // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; - public static final int LOOKUPTOPICRESPONSE_FIELD_NUMBER = 24; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse lookupTopicResponse_; - public boolean hasLookupTopicResponse() { - return ((bitField0_ & 0x00800000) == 0x00800000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse() { - return lookupTopicResponse_; - } - - // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; - public static final int CONSUMERSTATS_FIELD_NUMBER = 25; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_; - public boolean hasConsumerStats() { - return ((bitField0_ & 0x01000000) == 0x01000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() { - return consumerStats_; - } - - // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; - public static final int CONSUMERSTATSRESPONSE_FIELD_NUMBER = 26; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_; - public boolean hasConsumerStatsResponse() { - return ((bitField0_ & 0x02000000) == 0x02000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() { - return consumerStatsResponse_; - } - - // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; - public static final int REACHEDENDOFTOPIC_FIELD_NUMBER = 27; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic reachedEndOfTopic_; - public boolean hasReachedEndOfTopic() { - return ((bitField0_ & 0x04000000) == 0x04000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic() { - return reachedEndOfTopic_; - } - - // optional .pulsar.proto.CommandSeek seek = 28; - public static final int SEEK_FIELD_NUMBER = 28; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek seek_; - public boolean hasSeek() { - return ((bitField0_ & 0x08000000) == 0x08000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek() { - return seek_; - } - - // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; - public static final int GETLASTMESSAGEID_FIELD_NUMBER = 29; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getLastMessageId_; - public boolean hasGetLastMessageId() { - return ((bitField0_ & 0x10000000) == 0x10000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId() { - return getLastMessageId_; - } - - // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; - public static final int GETLASTMESSAGEIDRESPONSE_FIELD_NUMBER = 30; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getLastMessageIdResponse_; - public boolean hasGetLastMessageIdResponse() { - return ((bitField0_ & 0x20000000) == 0x20000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse() { - return getLastMessageIdResponse_; - } - - // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; - public static final int ACTIVE_CONSUMER_CHANGE_FIELD_NUMBER = 31; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange activeConsumerChange_; - public boolean hasActiveConsumerChange() { - return ((bitField0_ & 0x40000000) == 0x40000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange() { - return activeConsumerChange_; - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; - public static final int GETTOPICSOFNAMESPACE_FIELD_NUMBER = 32; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getTopicsOfNamespace_; - public boolean hasGetTopicsOfNamespace() { - return ((bitField0_ & 0x80000000) == 0x80000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace() { - return getTopicsOfNamespace_; - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; - public static final int GETTOPICSOFNAMESPACERESPONSE_FIELD_NUMBER = 33; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse_; - public boolean hasGetTopicsOfNamespaceResponse() { - return ((bitField1_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse() { - return getTopicsOfNamespaceResponse_; - } - - // optional .pulsar.proto.CommandGetSchema getSchema = 34; - public static final int GETSCHEMA_FIELD_NUMBER = 34; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getSchema_; - public boolean hasGetSchema() { - return ((bitField1_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema() { - return getSchema_; - } - - // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; - public static final int GETSCHEMARESPONSE_FIELD_NUMBER = 35; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getSchemaResponse_; - public boolean hasGetSchemaResponse() { - return ((bitField1_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse() { - return getSchemaResponse_; - } - - // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; - public static final int AUTHCHALLENGE_FIELD_NUMBER = 36; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge authChallenge_; - public boolean hasAuthChallenge() { - return ((bitField1_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge() { - return authChallenge_; - } - - // optional .pulsar.proto.CommandAuthResponse authResponse = 37; - public static final int AUTHRESPONSE_FIELD_NUMBER = 37; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse authResponse_; - public boolean hasAuthResponse() { - return ((bitField1_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse() { - return authResponse_; - } - - // optional .pulsar.proto.CommandAckResponse ackResponse = 38; - public static final int ACKRESPONSE_FIELD_NUMBER = 38; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse ackResponse_; - public boolean hasAckResponse() { - return ((bitField1_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse() { - return ackResponse_; - } - - // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; - public static final int GETORCREATESCHEMA_FIELD_NUMBER = 39; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getOrCreateSchema_; - public boolean hasGetOrCreateSchema() { - return ((bitField1_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema() { - return getOrCreateSchema_; - } - - // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; - public static final int GETORCREATESCHEMARESPONSE_FIELD_NUMBER = 40; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse_; - public boolean hasGetOrCreateSchemaResponse() { - return ((bitField1_ & 0x00000080) == 0x00000080); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse() { - return getOrCreateSchemaResponse_; - } - - // optional .pulsar.proto.CommandNewTxn newTxn = 50; - public static final int NEWTXN_FIELD_NUMBER = 50; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn newTxn_; - public boolean hasNewTxn() { - return ((bitField1_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn() { - return newTxn_; - } - - // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; - public static final int NEWTXNRESPONSE_FIELD_NUMBER = 51; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse newTxnResponse_; - public boolean hasNewTxnResponse() { - return ((bitField1_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse() { - return newTxnResponse_; - } - - // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; - public static final int ADDPARTITIONTOTXN_FIELD_NUMBER = 52; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn addPartitionToTxn_; - public boolean hasAddPartitionToTxn() { - return ((bitField1_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn() { - return addPartitionToTxn_; - } - - // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; - public static final int ADDPARTITIONTOTXNRESPONSE_FIELD_NUMBER = 53; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse addPartitionToTxnResponse_; - public boolean hasAddPartitionToTxnResponse() { - return ((bitField1_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse() { - return addPartitionToTxnResponse_; - } - - // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; - public static final int ADDSUBSCRIPTIONTOTXN_FIELD_NUMBER = 54; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn addSubscriptionToTxn_; - public boolean hasAddSubscriptionToTxn() { - return ((bitField1_ & 0x00001000) == 0x00001000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn() { - return addSubscriptionToTxn_; - } - - // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; - public static final int ADDSUBSCRIPTIONTOTXNRESPONSE_FIELD_NUMBER = 55; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse_; - public boolean hasAddSubscriptionToTxnResponse() { - return ((bitField1_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse() { - return addSubscriptionToTxnResponse_; - } - - // optional .pulsar.proto.CommandEndTxn endTxn = 56; - public static final int ENDTXN_FIELD_NUMBER = 56; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn endTxn_; - public boolean hasEndTxn() { - return ((bitField1_ & 0x00004000) == 0x00004000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn() { - return endTxn_; - } - - // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; - public static final int ENDTXNRESPONSE_FIELD_NUMBER = 57; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse endTxnResponse_; - public boolean hasEndTxnResponse() { - return ((bitField1_ & 0x00008000) == 0x00008000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse() { - return endTxnResponse_; - } - - // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; - public static final int ENDTXNONPARTITION_FIELD_NUMBER = 58; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition endTxnOnPartition_; - public boolean hasEndTxnOnPartition() { - return ((bitField1_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition() { - return endTxnOnPartition_; - } - - // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; - public static final int ENDTXNONPARTITIONRESPONSE_FIELD_NUMBER = 59; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse_; - public boolean hasEndTxnOnPartitionResponse() { - return ((bitField1_ & 0x00020000) == 0x00020000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse() { - return endTxnOnPartitionResponse_; - } - - // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; - public static final int ENDTXNONSUBSCRIPTION_FIELD_NUMBER = 60; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription endTxnOnSubscription_; - public boolean hasEndTxnOnSubscription() { - return ((bitField1_ & 0x00040000) == 0x00040000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription() { - return endTxnOnSubscription_; - } - - // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; - public static final int ENDTXNONSUBSCRIPTIONRESPONSE_FIELD_NUMBER = 61; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse_; - public boolean hasEndTxnOnSubscriptionResponse() { - return ((bitField1_ & 0x00080000) == 0x00080000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse() { - return endTxnOnSubscriptionResponse_; - } - - private void initFields() { - type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; - connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); - connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); - subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); - producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); - send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); - sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); - sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); - message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); - ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); - flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); - unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); - success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); - closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); - closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); - producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); - ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); - pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); - redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); - partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); - partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); - lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); - lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); - consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); - consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); - reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); - seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); - getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); - getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); - activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); - getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); - getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); - getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); - getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); - authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); - authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); - ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); - getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); - getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); - newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); - newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); - addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); - addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); - addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); - addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); - endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); - endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); - endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); - endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); - endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); - endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasType()) { - memoizedIsInitialized = 0; - return false; - } - if (hasConnect()) { - if (!getConnect().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasConnected()) { - if (!getConnected().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSubscribe()) { - if (!getSubscribe().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasProducer()) { - if (!getProducer().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSend()) { - if (!getSend().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSendReceipt()) { - if (!getSendReceipt().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSendError()) { - if (!getSendError().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasMessage()) { - if (!getMessage().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAck()) { - if (!getAck().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasFlow()) { - if (!getFlow().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasUnsubscribe()) { - if (!getUnsubscribe().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSuccess()) { - if (!getSuccess().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasError()) { - if (!getError().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasCloseProducer()) { - if (!getCloseProducer().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasCloseConsumer()) { - if (!getCloseConsumer().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasProducerSuccess()) { - if (!getProducerSuccess().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasRedeliverUnacknowledgedMessages()) { - if (!getRedeliverUnacknowledgedMessages().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasPartitionMetadata()) { - if (!getPartitionMetadata().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasPartitionMetadataResponse()) { - if (!getPartitionMetadataResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasLookupTopic()) { - if (!getLookupTopic().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasLookupTopicResponse()) { - if (!getLookupTopicResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasConsumerStats()) { - if (!getConsumerStats().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasConsumerStatsResponse()) { - if (!getConsumerStatsResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasReachedEndOfTopic()) { - if (!getReachedEndOfTopic().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasSeek()) { - if (!getSeek().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetLastMessageId()) { - if (!getGetLastMessageId().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetLastMessageIdResponse()) { - if (!getGetLastMessageIdResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasActiveConsumerChange()) { - if (!getActiveConsumerChange().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetTopicsOfNamespace()) { - if (!getGetTopicsOfNamespace().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetTopicsOfNamespaceResponse()) { - if (!getGetTopicsOfNamespaceResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetSchema()) { - if (!getGetSchema().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetSchemaResponse()) { - if (!getGetSchemaResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAckResponse()) { - if (!getAckResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetOrCreateSchema()) { - if (!getGetOrCreateSchema().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasGetOrCreateSchemaResponse()) { - if (!getGetOrCreateSchemaResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasNewTxn()) { - if (!getNewTxn().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasNewTxnResponse()) { - if (!getNewTxnResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAddPartitionToTxn()) { - if (!getAddPartitionToTxn().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAddPartitionToTxnResponse()) { - if (!getAddPartitionToTxnResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAddSubscriptionToTxn()) { - if (!getAddSubscriptionToTxn().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasAddSubscriptionToTxnResponse()) { - if (!getAddSubscriptionToTxnResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxn()) { - if (!getEndTxn().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxnResponse()) { - if (!getEndTxnResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxnOnPartition()) { - if (!getEndTxnOnPartition().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxnOnPartitionResponse()) { - if (!getEndTxnOnPartitionResponse().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxnOnSubscription()) { - if (!getEndTxnOnSubscription().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - } - if (hasEndTxnOnSubscriptionResponse()) { - if (!getEndTxnOnSubscriptionResponse().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.writeEnum(1, type_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeMessage(2, connect_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeMessage(3, connected_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeMessage(4, subscribe_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - output.writeMessage(5, producer_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - output.writeMessage(6, send_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - output.writeMessage(7, sendReceipt_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - output.writeMessage(8, sendError_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - output.writeMessage(9, message_); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - output.writeMessage(10, ack_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - output.writeMessage(11, flow_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - output.writeMessage(12, unsubscribe_); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - output.writeMessage(13, success_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - output.writeMessage(14, error_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - output.writeMessage(15, closeProducer_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - output.writeMessage(16, closeConsumer_); - } - if (((bitField0_ & 0x00010000) == 0x00010000)) { - output.writeMessage(17, producerSuccess_); - } - if (((bitField0_ & 0x00020000) == 0x00020000)) { - output.writeMessage(18, ping_); - } - if (((bitField0_ & 0x00040000) == 0x00040000)) { - output.writeMessage(19, pong_); - } - if (((bitField0_ & 0x00080000) == 0x00080000)) { - output.writeMessage(20, redeliverUnacknowledgedMessages_); - } - if (((bitField0_ & 0x00100000) == 0x00100000)) { - output.writeMessage(21, partitionMetadata_); - } - if (((bitField0_ & 0x00200000) == 0x00200000)) { - output.writeMessage(22, partitionMetadataResponse_); - } - if (((bitField0_ & 0x00400000) == 0x00400000)) { - output.writeMessage(23, lookupTopic_); - } - if (((bitField0_ & 0x00800000) == 0x00800000)) { - output.writeMessage(24, lookupTopicResponse_); - } - if (((bitField0_ & 0x01000000) == 0x01000000)) { - output.writeMessage(25, consumerStats_); - } - if (((bitField0_ & 0x02000000) == 0x02000000)) { - output.writeMessage(26, consumerStatsResponse_); - } - if (((bitField0_ & 0x04000000) == 0x04000000)) { - output.writeMessage(27, reachedEndOfTopic_); - } - if (((bitField0_ & 0x08000000) == 0x08000000)) { - output.writeMessage(28, seek_); - } - if (((bitField0_ & 0x10000000) == 0x10000000)) { - output.writeMessage(29, getLastMessageId_); - } - if (((bitField0_ & 0x20000000) == 0x20000000)) { - output.writeMessage(30, getLastMessageIdResponse_); - } - if (((bitField0_ & 0x40000000) == 0x40000000)) { - output.writeMessage(31, activeConsumerChange_); - } - if (((bitField0_ & 0x80000000) == 0x80000000)) { - output.writeMessage(32, getTopicsOfNamespace_); - } - if (((bitField1_ & 0x00000001) == 0x00000001)) { - output.writeMessage(33, getTopicsOfNamespaceResponse_); - } - if (((bitField1_ & 0x00000002) == 0x00000002)) { - output.writeMessage(34, getSchema_); - } - if (((bitField1_ & 0x00000004) == 0x00000004)) { - output.writeMessage(35, getSchemaResponse_); - } - if (((bitField1_ & 0x00000008) == 0x00000008)) { - output.writeMessage(36, authChallenge_); - } - if (((bitField1_ & 0x00000010) == 0x00000010)) { - output.writeMessage(37, authResponse_); - } - if (((bitField1_ & 0x00000020) == 0x00000020)) { - output.writeMessage(38, ackResponse_); - } - if (((bitField1_ & 0x00000040) == 0x00000040)) { - output.writeMessage(39, getOrCreateSchema_); - } - if (((bitField1_ & 0x00000080) == 0x00000080)) { - output.writeMessage(40, getOrCreateSchemaResponse_); - } - if (((bitField1_ & 0x00000100) == 0x00000100)) { - output.writeMessage(50, newTxn_); - } - if (((bitField1_ & 0x00000200) == 0x00000200)) { - output.writeMessage(51, newTxnResponse_); - } - if (((bitField1_ & 0x00000400) == 0x00000400)) { - output.writeMessage(52, addPartitionToTxn_); - } - if (((bitField1_ & 0x00000800) == 0x00000800)) { - output.writeMessage(53, addPartitionToTxnResponse_); - } - if (((bitField1_ & 0x00001000) == 0x00001000)) { - output.writeMessage(54, addSubscriptionToTxn_); - } - if (((bitField1_ & 0x00002000) == 0x00002000)) { - output.writeMessage(55, addSubscriptionToTxnResponse_); - } - if (((bitField1_ & 0x00004000) == 0x00004000)) { - output.writeMessage(56, endTxn_); - } - if (((bitField1_ & 0x00008000) == 0x00008000)) { - output.writeMessage(57, endTxnResponse_); - } - if (((bitField1_ & 0x00010000) == 0x00010000)) { - output.writeMessage(58, endTxnOnPartition_); - } - if (((bitField1_ & 0x00020000) == 0x00020000)) { - output.writeMessage(59, endTxnOnPartitionResponse_); - } - if (((bitField1_ & 0x00040000) == 0x00040000)) { - output.writeMessage(60, endTxnOnSubscription_); - } - if (((bitField1_ & 0x00080000) == 0x00080000)) { - output.writeMessage(61, endTxnOnSubscriptionResponse_); - } - } - - 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 - .computeEnumSize(1, type_.getNumber()); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(2, connect_); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(3, connected_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(4, subscribe_); - } - if (((bitField0_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(5, producer_); - } - if (((bitField0_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(6, send_); - } - if (((bitField0_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(7, sendReceipt_); - } - if (((bitField0_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(8, sendError_); - } - if (((bitField0_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(9, message_); - } - if (((bitField0_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(10, ack_); - } - if (((bitField0_ & 0x00000400) == 0x00000400)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(11, flow_); - } - if (((bitField0_ & 0x00000800) == 0x00000800)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(12, unsubscribe_); - } - if (((bitField0_ & 0x00001000) == 0x00001000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(13, success_); - } - if (((bitField0_ & 0x00002000) == 0x00002000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(14, error_); - } - if (((bitField0_ & 0x00004000) == 0x00004000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(15, closeProducer_); - } - if (((bitField0_ & 0x00008000) == 0x00008000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(16, closeConsumer_); - } - if (((bitField0_ & 0x00010000) == 0x00010000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(17, producerSuccess_); - } - if (((bitField0_ & 0x00020000) == 0x00020000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(18, ping_); - } - if (((bitField0_ & 0x00040000) == 0x00040000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(19, pong_); - } - if (((bitField0_ & 0x00080000) == 0x00080000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(20, redeliverUnacknowledgedMessages_); - } - if (((bitField0_ & 0x00100000) == 0x00100000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(21, partitionMetadata_); - } - if (((bitField0_ & 0x00200000) == 0x00200000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(22, partitionMetadataResponse_); - } - if (((bitField0_ & 0x00400000) == 0x00400000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(23, lookupTopic_); - } - if (((bitField0_ & 0x00800000) == 0x00800000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(24, lookupTopicResponse_); - } - if (((bitField0_ & 0x01000000) == 0x01000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(25, consumerStats_); - } - if (((bitField0_ & 0x02000000) == 0x02000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(26, consumerStatsResponse_); - } - if (((bitField0_ & 0x04000000) == 0x04000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(27, reachedEndOfTopic_); - } - if (((bitField0_ & 0x08000000) == 0x08000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(28, seek_); - } - if (((bitField0_ & 0x10000000) == 0x10000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(29, getLastMessageId_); - } - if (((bitField0_ & 0x20000000) == 0x20000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(30, getLastMessageIdResponse_); - } - if (((bitField0_ & 0x40000000) == 0x40000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(31, activeConsumerChange_); - } - if (((bitField0_ & 0x80000000) == 0x80000000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(32, getTopicsOfNamespace_); - } - if (((bitField1_ & 0x00000001) == 0x00000001)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(33, getTopicsOfNamespaceResponse_); - } - if (((bitField1_ & 0x00000002) == 0x00000002)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(34, getSchema_); - } - if (((bitField1_ & 0x00000004) == 0x00000004)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(35, getSchemaResponse_); - } - if (((bitField1_ & 0x00000008) == 0x00000008)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(36, authChallenge_); - } - if (((bitField1_ & 0x00000010) == 0x00000010)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(37, authResponse_); - } - if (((bitField1_ & 0x00000020) == 0x00000020)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(38, ackResponse_); - } - if (((bitField1_ & 0x00000040) == 0x00000040)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(39, getOrCreateSchema_); - } - if (((bitField1_ & 0x00000080) == 0x00000080)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(40, getOrCreateSchemaResponse_); - } - if (((bitField1_ & 0x00000100) == 0x00000100)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(50, newTxn_); - } - if (((bitField1_ & 0x00000200) == 0x00000200)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(51, newTxnResponse_); - } - if (((bitField1_ & 0x00000400) == 0x00000400)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(52, addPartitionToTxn_); - } - if (((bitField1_ & 0x00000800) == 0x00000800)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(53, addPartitionToTxnResponse_); - } - if (((bitField1_ & 0x00001000) == 0x00001000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(54, addSubscriptionToTxn_); - } - if (((bitField1_ & 0x00002000) == 0x00002000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(55, addSubscriptionToTxnResponse_); - } - if (((bitField1_ & 0x00004000) == 0x00004000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(56, endTxn_); - } - if (((bitField1_ & 0x00008000) == 0x00008000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(57, endTxnResponse_); - } - if (((bitField1_ & 0x00010000) == 0x00010000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(58, endTxnOnPartition_); - } - if (((bitField1_ & 0x00020000) == 0x00020000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(59, endTxnOnPartitionResponse_); - } - if (((bitField1_ & 0x00040000) == 0x00040000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(60, endTxnOnSubscription_); - } - if (((bitField1_ & 0x00080000) == 0x00080000)) { - size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream - .computeMessageSize(61, endTxnOnSubscriptionResponse_); - } - 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand, Builder> - implements org.apache.pulsar.common.api.proto.PulsarApi.BaseCommandOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { - // Construct using org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.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(); - type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; - bitField0_ = (bitField0_ & ~0x00000001); - connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000002); - connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000004); - subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000008); - producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000010); - send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000020); - sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000040); - sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000080); - message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000100); - ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000200); - flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000400); - unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00000800); - success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00001000); - error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00002000); - closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00004000); - closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00008000); - producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00010000); - ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00020000); - pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00040000); - redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00080000); - partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00100000); - partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00200000); - lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00400000); - lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x00800000); - consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x01000000); - consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x02000000); - reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x04000000); - seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x08000000); - getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x10000000); - getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x20000000); - activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x40000000); - getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); - bitField0_ = (bitField0_ & ~0x80000000); - getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000001); - getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000002); - getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000004); - authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000008); - authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000010); - ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000020); - getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000040); - getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000080); - newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000100); - newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000200); - addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000400); - addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00000800); - addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00001000); - addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00002000); - endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00004000); - endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00008000); - endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00010000); - endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00020000); - endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00040000); - endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); - bitField1_ = (bitField1_ & ~0x00080000); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand getDefaultInstanceForType() { - return org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.getDefaultInstance(); - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand build() { - org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand buildParsed() - throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { - org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand buildPartial() { - org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.RECYCLER.get(); - int from_bitField0_ = bitField0_; - int from_bitField1_ = bitField1_; - int to_bitField0_ = 0; - int to_bitField1_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.type_ = type_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.connect_ = connect_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.connected_ = connected_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.subscribe_ = subscribe_; - if (((from_bitField0_ & 0x00000010) == 0x00000010)) { - to_bitField0_ |= 0x00000010; - } - result.producer_ = producer_; - if (((from_bitField0_ & 0x00000020) == 0x00000020)) { - to_bitField0_ |= 0x00000020; - } - result.send_ = send_; - if (((from_bitField0_ & 0x00000040) == 0x00000040)) { - to_bitField0_ |= 0x00000040; - } - result.sendReceipt_ = sendReceipt_; - if (((from_bitField0_ & 0x00000080) == 0x00000080)) { - to_bitField0_ |= 0x00000080; - } - result.sendError_ = sendError_; - if (((from_bitField0_ & 0x00000100) == 0x00000100)) { - to_bitField0_ |= 0x00000100; - } - result.message_ = message_; - if (((from_bitField0_ & 0x00000200) == 0x00000200)) { - to_bitField0_ |= 0x00000200; - } - result.ack_ = ack_; - if (((from_bitField0_ & 0x00000400) == 0x00000400)) { - to_bitField0_ |= 0x00000400; - } - result.flow_ = flow_; - if (((from_bitField0_ & 0x00000800) == 0x00000800)) { - to_bitField0_ |= 0x00000800; - } - result.unsubscribe_ = unsubscribe_; - if (((from_bitField0_ & 0x00001000) == 0x00001000)) { - to_bitField0_ |= 0x00001000; - } - result.success_ = success_; - if (((from_bitField0_ & 0x00002000) == 0x00002000)) { - to_bitField0_ |= 0x00002000; - } - result.error_ = error_; - if (((from_bitField0_ & 0x00004000) == 0x00004000)) { - to_bitField0_ |= 0x00004000; - } - result.closeProducer_ = closeProducer_; - if (((from_bitField0_ & 0x00008000) == 0x00008000)) { - to_bitField0_ |= 0x00008000; - } - result.closeConsumer_ = closeConsumer_; - if (((from_bitField0_ & 0x00010000) == 0x00010000)) { - to_bitField0_ |= 0x00010000; - } - result.producerSuccess_ = producerSuccess_; - if (((from_bitField0_ & 0x00020000) == 0x00020000)) { - to_bitField0_ |= 0x00020000; - } - result.ping_ = ping_; - if (((from_bitField0_ & 0x00040000) == 0x00040000)) { - to_bitField0_ |= 0x00040000; - } - result.pong_ = pong_; - if (((from_bitField0_ & 0x00080000) == 0x00080000)) { - to_bitField0_ |= 0x00080000; - } - result.redeliverUnacknowledgedMessages_ = redeliverUnacknowledgedMessages_; - if (((from_bitField0_ & 0x00100000) == 0x00100000)) { - to_bitField0_ |= 0x00100000; - } - result.partitionMetadata_ = partitionMetadata_; - if (((from_bitField0_ & 0x00200000) == 0x00200000)) { - to_bitField0_ |= 0x00200000; - } - result.partitionMetadataResponse_ = partitionMetadataResponse_; - if (((from_bitField0_ & 0x00400000) == 0x00400000)) { - to_bitField0_ |= 0x00400000; - } - result.lookupTopic_ = lookupTopic_; - if (((from_bitField0_ & 0x00800000) == 0x00800000)) { - to_bitField0_ |= 0x00800000; - } - result.lookupTopicResponse_ = lookupTopicResponse_; - if (((from_bitField0_ & 0x01000000) == 0x01000000)) { - to_bitField0_ |= 0x01000000; - } - result.consumerStats_ = consumerStats_; - if (((from_bitField0_ & 0x02000000) == 0x02000000)) { - to_bitField0_ |= 0x02000000; - } - result.consumerStatsResponse_ = consumerStatsResponse_; - if (((from_bitField0_ & 0x04000000) == 0x04000000)) { - to_bitField0_ |= 0x04000000; - } - result.reachedEndOfTopic_ = reachedEndOfTopic_; - if (((from_bitField0_ & 0x08000000) == 0x08000000)) { - to_bitField0_ |= 0x08000000; - } - result.seek_ = seek_; - if (((from_bitField0_ & 0x10000000) == 0x10000000)) { - to_bitField0_ |= 0x10000000; - } - result.getLastMessageId_ = getLastMessageId_; - if (((from_bitField0_ & 0x20000000) == 0x20000000)) { - to_bitField0_ |= 0x20000000; - } - result.getLastMessageIdResponse_ = getLastMessageIdResponse_; - if (((from_bitField0_ & 0x40000000) == 0x40000000)) { - to_bitField0_ |= 0x40000000; - } - result.activeConsumerChange_ = activeConsumerChange_; - if (((from_bitField0_ & 0x80000000) == 0x80000000)) { - to_bitField0_ |= 0x80000000; - } - result.getTopicsOfNamespace_ = getTopicsOfNamespace_; - if (((from_bitField1_ & 0x00000001) == 0x00000001)) { - to_bitField1_ |= 0x00000001; - } - result.getTopicsOfNamespaceResponse_ = getTopicsOfNamespaceResponse_; - if (((from_bitField1_ & 0x00000002) == 0x00000002)) { - to_bitField1_ |= 0x00000002; - } - result.getSchema_ = getSchema_; - if (((from_bitField1_ & 0x00000004) == 0x00000004)) { - to_bitField1_ |= 0x00000004; - } - result.getSchemaResponse_ = getSchemaResponse_; - if (((from_bitField1_ & 0x00000008) == 0x00000008)) { - to_bitField1_ |= 0x00000008; - } - result.authChallenge_ = authChallenge_; - if (((from_bitField1_ & 0x00000010) == 0x00000010)) { - to_bitField1_ |= 0x00000010; - } - result.authResponse_ = authResponse_; - if (((from_bitField1_ & 0x00000020) == 0x00000020)) { - to_bitField1_ |= 0x00000020; - } - result.ackResponse_ = ackResponse_; - if (((from_bitField1_ & 0x00000040) == 0x00000040)) { - to_bitField1_ |= 0x00000040; - } - result.getOrCreateSchema_ = getOrCreateSchema_; - if (((from_bitField1_ & 0x00000080) == 0x00000080)) { - to_bitField1_ |= 0x00000080; - } - result.getOrCreateSchemaResponse_ = getOrCreateSchemaResponse_; - if (((from_bitField1_ & 0x00000100) == 0x00000100)) { - to_bitField1_ |= 0x00000100; - } - result.newTxn_ = newTxn_; - if (((from_bitField1_ & 0x00000200) == 0x00000200)) { - to_bitField1_ |= 0x00000200; - } - result.newTxnResponse_ = newTxnResponse_; - if (((from_bitField1_ & 0x00000400) == 0x00000400)) { - to_bitField1_ |= 0x00000400; - } - result.addPartitionToTxn_ = addPartitionToTxn_; - if (((from_bitField1_ & 0x00000800) == 0x00000800)) { - to_bitField1_ |= 0x00000800; - } - result.addPartitionToTxnResponse_ = addPartitionToTxnResponse_; - if (((from_bitField1_ & 0x00001000) == 0x00001000)) { - to_bitField1_ |= 0x00001000; - } - result.addSubscriptionToTxn_ = addSubscriptionToTxn_; - if (((from_bitField1_ & 0x00002000) == 0x00002000)) { - to_bitField1_ |= 0x00002000; - } - result.addSubscriptionToTxnResponse_ = addSubscriptionToTxnResponse_; - if (((from_bitField1_ & 0x00004000) == 0x00004000)) { - to_bitField1_ |= 0x00004000; - } - result.endTxn_ = endTxn_; - if (((from_bitField1_ & 0x00008000) == 0x00008000)) { - to_bitField1_ |= 0x00008000; - } - result.endTxnResponse_ = endTxnResponse_; - if (((from_bitField1_ & 0x00010000) == 0x00010000)) { - to_bitField1_ |= 0x00010000; - } - result.endTxnOnPartition_ = endTxnOnPartition_; - if (((from_bitField1_ & 0x00020000) == 0x00020000)) { - to_bitField1_ |= 0x00020000; - } - result.endTxnOnPartitionResponse_ = endTxnOnPartitionResponse_; - if (((from_bitField1_ & 0x00040000) == 0x00040000)) { - to_bitField1_ |= 0x00040000; - } - result.endTxnOnSubscription_ = endTxnOnSubscription_; - if (((from_bitField1_ & 0x00080000) == 0x00080000)) { - to_bitField1_ |= 0x00080000; - } - result.endTxnOnSubscriptionResponse_ = endTxnOnSubscriptionResponse_; - result.bitField0_ = to_bitField0_; - result.bitField1_ = to_bitField1_; - return result; - } - - public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand other) { - if (other == org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.getDefaultInstance()) return this; - if (other.hasType()) { - setType(other.getType()); - } - if (other.hasConnect()) { - mergeConnect(other.getConnect()); - } - if (other.hasConnected()) { - mergeConnected(other.getConnected()); - } - if (other.hasSubscribe()) { - mergeSubscribe(other.getSubscribe()); - } - if (other.hasProducer()) { - mergeProducer(other.getProducer()); - } - if (other.hasSend()) { - mergeSend(other.getSend()); - } - if (other.hasSendReceipt()) { - mergeSendReceipt(other.getSendReceipt()); - } - if (other.hasSendError()) { - mergeSendError(other.getSendError()); - } - if (other.hasMessage()) { - mergeMessage(other.getMessage()); - } - if (other.hasAck()) { - mergeAck(other.getAck()); - } - if (other.hasFlow()) { - mergeFlow(other.getFlow()); - } - if (other.hasUnsubscribe()) { - mergeUnsubscribe(other.getUnsubscribe()); - } - if (other.hasSuccess()) { - mergeSuccess(other.getSuccess()); - } - if (other.hasError()) { - mergeError(other.getError()); - } - if (other.hasCloseProducer()) { - mergeCloseProducer(other.getCloseProducer()); - } - if (other.hasCloseConsumer()) { - mergeCloseConsumer(other.getCloseConsumer()); - } - if (other.hasProducerSuccess()) { - mergeProducerSuccess(other.getProducerSuccess()); - } - if (other.hasPing()) { - mergePing(other.getPing()); - } - if (other.hasPong()) { - mergePong(other.getPong()); - } - if (other.hasRedeliverUnacknowledgedMessages()) { - mergeRedeliverUnacknowledgedMessages(other.getRedeliverUnacknowledgedMessages()); - } - if (other.hasPartitionMetadata()) { - mergePartitionMetadata(other.getPartitionMetadata()); - } - if (other.hasPartitionMetadataResponse()) { - mergePartitionMetadataResponse(other.getPartitionMetadataResponse()); - } - if (other.hasLookupTopic()) { - mergeLookupTopic(other.getLookupTopic()); - } - if (other.hasLookupTopicResponse()) { - mergeLookupTopicResponse(other.getLookupTopicResponse()); - } - if (other.hasConsumerStats()) { - mergeConsumerStats(other.getConsumerStats()); - } - if (other.hasConsumerStatsResponse()) { - mergeConsumerStatsResponse(other.getConsumerStatsResponse()); - } - if (other.hasReachedEndOfTopic()) { - mergeReachedEndOfTopic(other.getReachedEndOfTopic()); - } - if (other.hasSeek()) { - mergeSeek(other.getSeek()); - } - if (other.hasGetLastMessageId()) { - mergeGetLastMessageId(other.getGetLastMessageId()); - } - if (other.hasGetLastMessageIdResponse()) { - mergeGetLastMessageIdResponse(other.getGetLastMessageIdResponse()); - } - if (other.hasActiveConsumerChange()) { - mergeActiveConsumerChange(other.getActiveConsumerChange()); - } - if (other.hasGetTopicsOfNamespace()) { - mergeGetTopicsOfNamespace(other.getGetTopicsOfNamespace()); - } - if (other.hasGetTopicsOfNamespaceResponse()) { - mergeGetTopicsOfNamespaceResponse(other.getGetTopicsOfNamespaceResponse()); - } - if (other.hasGetSchema()) { - mergeGetSchema(other.getGetSchema()); - } - if (other.hasGetSchemaResponse()) { - mergeGetSchemaResponse(other.getGetSchemaResponse()); - } - if (other.hasAuthChallenge()) { - mergeAuthChallenge(other.getAuthChallenge()); - } - if (other.hasAuthResponse()) { - mergeAuthResponse(other.getAuthResponse()); - } - if (other.hasAckResponse()) { - mergeAckResponse(other.getAckResponse()); - } - if (other.hasGetOrCreateSchema()) { - mergeGetOrCreateSchema(other.getGetOrCreateSchema()); - } - if (other.hasGetOrCreateSchemaResponse()) { - mergeGetOrCreateSchemaResponse(other.getGetOrCreateSchemaResponse()); - } - if (other.hasNewTxn()) { - mergeNewTxn(other.getNewTxn()); - } - if (other.hasNewTxnResponse()) { - mergeNewTxnResponse(other.getNewTxnResponse()); - } - if (other.hasAddPartitionToTxn()) { - mergeAddPartitionToTxn(other.getAddPartitionToTxn()); - } - if (other.hasAddPartitionToTxnResponse()) { - mergeAddPartitionToTxnResponse(other.getAddPartitionToTxnResponse()); - } - if (other.hasAddSubscriptionToTxn()) { - mergeAddSubscriptionToTxn(other.getAddSubscriptionToTxn()); - } - if (other.hasAddSubscriptionToTxnResponse()) { - mergeAddSubscriptionToTxnResponse(other.getAddSubscriptionToTxnResponse()); - } - if (other.hasEndTxn()) { - mergeEndTxn(other.getEndTxn()); - } - if (other.hasEndTxnResponse()) { - mergeEndTxnResponse(other.getEndTxnResponse()); - } - if (other.hasEndTxnOnPartition()) { - mergeEndTxnOnPartition(other.getEndTxnOnPartition()); - } - if (other.hasEndTxnOnPartitionResponse()) { - mergeEndTxnOnPartitionResponse(other.getEndTxnOnPartitionResponse()); - } - if (other.hasEndTxnOnSubscription()) { - mergeEndTxnOnSubscription(other.getEndTxnOnSubscription()); - } - if (other.hasEndTxnOnSubscriptionResponse()) { - mergeEndTxnOnSubscriptionResponse(other.getEndTxnOnSubscriptionResponse()); - } - return this; - } - - public final boolean isInitialized() { - if (!hasType()) { - - return false; - } - if (hasConnect()) { - if (!getConnect().isInitialized()) { - - return false; - } - } - if (hasConnected()) { - if (!getConnected().isInitialized()) { - - return false; - } - } - if (hasSubscribe()) { - if (!getSubscribe().isInitialized()) { - - return false; - } - } - if (hasProducer()) { - if (!getProducer().isInitialized()) { - - return false; - } - } - if (hasSend()) { - if (!getSend().isInitialized()) { - - return false; - } - } - if (hasSendReceipt()) { - if (!getSendReceipt().isInitialized()) { - - return false; - } - } - if (hasSendError()) { - if (!getSendError().isInitialized()) { - - return false; - } - } - if (hasMessage()) { - if (!getMessage().isInitialized()) { - - return false; - } - } - if (hasAck()) { - if (!getAck().isInitialized()) { - - return false; - } - } - if (hasFlow()) { - if (!getFlow().isInitialized()) { - - return false; - } - } - if (hasUnsubscribe()) { - if (!getUnsubscribe().isInitialized()) { - - return false; - } - } - if (hasSuccess()) { - if (!getSuccess().isInitialized()) { - - return false; - } - } - if (hasError()) { - if (!getError().isInitialized()) { - - return false; - } - } - if (hasCloseProducer()) { - if (!getCloseProducer().isInitialized()) { - - return false; - } - } - if (hasCloseConsumer()) { - if (!getCloseConsumer().isInitialized()) { - - return false; - } - } - if (hasProducerSuccess()) { - if (!getProducerSuccess().isInitialized()) { - - return false; - } - } - if (hasRedeliverUnacknowledgedMessages()) { - if (!getRedeliverUnacknowledgedMessages().isInitialized()) { - - return false; - } - } - if (hasPartitionMetadata()) { - if (!getPartitionMetadata().isInitialized()) { - - return false; - } - } - if (hasPartitionMetadataResponse()) { - if (!getPartitionMetadataResponse().isInitialized()) { - - return false; - } - } - if (hasLookupTopic()) { - if (!getLookupTopic().isInitialized()) { - - return false; - } - } - if (hasLookupTopicResponse()) { - if (!getLookupTopicResponse().isInitialized()) { - - return false; - } - } - if (hasConsumerStats()) { - if (!getConsumerStats().isInitialized()) { - - return false; - } - } - if (hasConsumerStatsResponse()) { - if (!getConsumerStatsResponse().isInitialized()) { - - return false; - } - } - if (hasReachedEndOfTopic()) { - if (!getReachedEndOfTopic().isInitialized()) { - - return false; - } - } - if (hasSeek()) { - if (!getSeek().isInitialized()) { - - return false; - } - } - if (hasGetLastMessageId()) { - if (!getGetLastMessageId().isInitialized()) { - - return false; - } - } - if (hasGetLastMessageIdResponse()) { - if (!getGetLastMessageIdResponse().isInitialized()) { - - return false; - } - } - if (hasActiveConsumerChange()) { - if (!getActiveConsumerChange().isInitialized()) { - - return false; - } - } - if (hasGetTopicsOfNamespace()) { - if (!getGetTopicsOfNamespace().isInitialized()) { - - return false; - } - } - if (hasGetTopicsOfNamespaceResponse()) { - if (!getGetTopicsOfNamespaceResponse().isInitialized()) { - - return false; - } - } - if (hasGetSchema()) { - if (!getGetSchema().isInitialized()) { - - return false; - } - } - if (hasGetSchemaResponse()) { - if (!getGetSchemaResponse().isInitialized()) { - - return false; - } - } - if (hasAckResponse()) { - if (!getAckResponse().isInitialized()) { - - return false; - } - } - if (hasGetOrCreateSchema()) { - if (!getGetOrCreateSchema().isInitialized()) { - - return false; - } - } - if (hasGetOrCreateSchemaResponse()) { - if (!getGetOrCreateSchemaResponse().isInitialized()) { - - return false; - } - } - if (hasNewTxn()) { - if (!getNewTxn().isInitialized()) { - - return false; - } - } - if (hasNewTxnResponse()) { - if (!getNewTxnResponse().isInitialized()) { - - return false; - } - } - if (hasAddPartitionToTxn()) { - if (!getAddPartitionToTxn().isInitialized()) { - - return false; - } - } - if (hasAddPartitionToTxnResponse()) { - if (!getAddPartitionToTxnResponse().isInitialized()) { - - return false; - } - } - if (hasAddSubscriptionToTxn()) { - if (!getAddSubscriptionToTxn().isInitialized()) { - - return false; - } - } - if (hasAddSubscriptionToTxnResponse()) { - if (!getAddSubscriptionToTxnResponse().isInitialized()) { - - return false; - } - } - if (hasEndTxn()) { - if (!getEndTxn().isInitialized()) { - - return false; - } - } - if (hasEndTxnResponse()) { - if (!getEndTxnResponse().isInitialized()) { - - return false; - } - } - if (hasEndTxnOnPartition()) { - if (!getEndTxnOnPartition().isInitialized()) { - - return false; - } - } - if (hasEndTxnOnPartitionResponse()) { - if (!getEndTxnOnPartitionResponse().isInitialized()) { - - return false; - } - } - if (hasEndTxnOnSubscription()) { - if (!getEndTxnOnSubscription().isInitialized()) { - - return false; - } - } - if (hasEndTxnOnSubscriptionResponse()) { - if (!getEndTxnOnSubscriptionResponse().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 8: { - int rawValue = input.readEnum(); - org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type value = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.valueOf(rawValue); - if (value != null) { - bitField0_ |= 0x00000001; - type_ = value; - } - break; - } - case 18: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.newBuilder(); - if (hasConnect()) { - subBuilder.mergeFrom(getConnect()); - } - input.readMessage(subBuilder, extensionRegistry); - setConnect(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 26: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.newBuilder(); - if (hasConnected()) { - subBuilder.mergeFrom(getConnected()); - } - input.readMessage(subBuilder, extensionRegistry); - setConnected(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 34: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.newBuilder(); - if (hasSubscribe()) { - subBuilder.mergeFrom(getSubscribe()); - } - input.readMessage(subBuilder, extensionRegistry); - setSubscribe(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 42: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.newBuilder(); - if (hasProducer()) { - subBuilder.mergeFrom(getProducer()); - } - input.readMessage(subBuilder, extensionRegistry); - setProducer(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 50: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.newBuilder(); - if (hasSend()) { - subBuilder.mergeFrom(getSend()); - } - input.readMessage(subBuilder, extensionRegistry); - setSend(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 58: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.newBuilder(); - if (hasSendReceipt()) { - subBuilder.mergeFrom(getSendReceipt()); - } - input.readMessage(subBuilder, extensionRegistry); - setSendReceipt(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 66: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.newBuilder(); - if (hasSendError()) { - subBuilder.mergeFrom(getSendError()); - } - input.readMessage(subBuilder, extensionRegistry); - setSendError(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 74: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.newBuilder(); - if (hasMessage()) { - subBuilder.mergeFrom(getMessage()); - } - input.readMessage(subBuilder, extensionRegistry); - setMessage(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 82: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.newBuilder(); - if (hasAck()) { - subBuilder.mergeFrom(getAck()); - } - input.readMessage(subBuilder, extensionRegistry); - setAck(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 90: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.newBuilder(); - if (hasFlow()) { - subBuilder.mergeFrom(getFlow()); - } - input.readMessage(subBuilder, extensionRegistry); - setFlow(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 98: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.newBuilder(); - if (hasUnsubscribe()) { - subBuilder.mergeFrom(getUnsubscribe()); - } - input.readMessage(subBuilder, extensionRegistry); - setUnsubscribe(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 106: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.newBuilder(); - if (hasSuccess()) { - subBuilder.mergeFrom(getSuccess()); - } - input.readMessage(subBuilder, extensionRegistry); - setSuccess(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 114: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandError.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.newBuilder(); - if (hasError()) { - subBuilder.mergeFrom(getError()); - } - input.readMessage(subBuilder, extensionRegistry); - setError(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 122: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.newBuilder(); - if (hasCloseProducer()) { - subBuilder.mergeFrom(getCloseProducer()); - } - input.readMessage(subBuilder, extensionRegistry); - setCloseProducer(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 130: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.newBuilder(); - if (hasCloseConsumer()) { - subBuilder.mergeFrom(getCloseConsumer()); - } - input.readMessage(subBuilder, extensionRegistry); - setCloseConsumer(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 138: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.newBuilder(); - if (hasProducerSuccess()) { - subBuilder.mergeFrom(getProducerSuccess()); - } - input.readMessage(subBuilder, extensionRegistry); - setProducerSuccess(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 146: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.newBuilder(); - if (hasPing()) { - subBuilder.mergeFrom(getPing()); - } - input.readMessage(subBuilder, extensionRegistry); - setPing(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 154: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.newBuilder(); - if (hasPong()) { - subBuilder.mergeFrom(getPong()); - } - input.readMessage(subBuilder, extensionRegistry); - setPong(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 162: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.newBuilder(); - if (hasRedeliverUnacknowledgedMessages()) { - subBuilder.mergeFrom(getRedeliverUnacknowledgedMessages()); - } - input.readMessage(subBuilder, extensionRegistry); - setRedeliverUnacknowledgedMessages(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 170: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.newBuilder(); - if (hasPartitionMetadata()) { - subBuilder.mergeFrom(getPartitionMetadata()); - } - input.readMessage(subBuilder, extensionRegistry); - setPartitionMetadata(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 178: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.newBuilder(); - if (hasPartitionMetadataResponse()) { - subBuilder.mergeFrom(getPartitionMetadataResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setPartitionMetadataResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 186: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.newBuilder(); - if (hasLookupTopic()) { - subBuilder.mergeFrom(getLookupTopic()); - } - input.readMessage(subBuilder, extensionRegistry); - setLookupTopic(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 194: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.newBuilder(); - if (hasLookupTopicResponse()) { - subBuilder.mergeFrom(getLookupTopicResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setLookupTopicResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 202: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder(); - if (hasConsumerStats()) { - subBuilder.mergeFrom(getConsumerStats()); - } - input.readMessage(subBuilder, extensionRegistry); - setConsumerStats(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 210: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder(); - if (hasConsumerStatsResponse()) { - subBuilder.mergeFrom(getConsumerStatsResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setConsumerStatsResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 218: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.newBuilder(); - if (hasReachedEndOfTopic()) { - subBuilder.mergeFrom(getReachedEndOfTopic()); - } - input.readMessage(subBuilder, extensionRegistry); - setReachedEndOfTopic(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 226: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.newBuilder(); - if (hasSeek()) { - subBuilder.mergeFrom(getSeek()); - } - input.readMessage(subBuilder, extensionRegistry); - setSeek(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 234: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.newBuilder(); - if (hasGetLastMessageId()) { - subBuilder.mergeFrom(getGetLastMessageId()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetLastMessageId(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 242: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.newBuilder(); - if (hasGetLastMessageIdResponse()) { - subBuilder.mergeFrom(getGetLastMessageIdResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetLastMessageIdResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 250: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.newBuilder(); - if (hasActiveConsumerChange()) { - subBuilder.mergeFrom(getActiveConsumerChange()); - } - input.readMessage(subBuilder, extensionRegistry); - setActiveConsumerChange(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 258: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.newBuilder(); - if (hasGetTopicsOfNamespace()) { - subBuilder.mergeFrom(getGetTopicsOfNamespace()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetTopicsOfNamespace(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 266: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.newBuilder(); - if (hasGetTopicsOfNamespaceResponse()) { - subBuilder.mergeFrom(getGetTopicsOfNamespaceResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetTopicsOfNamespaceResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 274: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.newBuilder(); - if (hasGetSchema()) { - subBuilder.mergeFrom(getGetSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 282: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.newBuilder(); - if (hasGetSchemaResponse()) { - subBuilder.mergeFrom(getGetSchemaResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetSchemaResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 290: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.newBuilder(); - if (hasAuthChallenge()) { - subBuilder.mergeFrom(getAuthChallenge()); - } - input.readMessage(subBuilder, extensionRegistry); - setAuthChallenge(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 298: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.newBuilder(); - if (hasAuthResponse()) { - subBuilder.mergeFrom(getAuthResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setAuthResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 306: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.newBuilder(); - if (hasAckResponse()) { - subBuilder.mergeFrom(getAckResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setAckResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 314: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.newBuilder(); - if (hasGetOrCreateSchema()) { - subBuilder.mergeFrom(getGetOrCreateSchema()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetOrCreateSchema(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 322: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.newBuilder(); - if (hasGetOrCreateSchemaResponse()) { - subBuilder.mergeFrom(getGetOrCreateSchemaResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setGetOrCreateSchemaResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 402: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.newBuilder(); - if (hasNewTxn()) { - subBuilder.mergeFrom(getNewTxn()); - } - input.readMessage(subBuilder, extensionRegistry); - setNewTxn(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 410: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.newBuilder(); - if (hasNewTxnResponse()) { - subBuilder.mergeFrom(getNewTxnResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setNewTxnResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 418: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.newBuilder(); - if (hasAddPartitionToTxn()) { - subBuilder.mergeFrom(getAddPartitionToTxn()); - } - input.readMessage(subBuilder, extensionRegistry); - setAddPartitionToTxn(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 426: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.newBuilder(); - if (hasAddPartitionToTxnResponse()) { - subBuilder.mergeFrom(getAddPartitionToTxnResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setAddPartitionToTxnResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 434: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.newBuilder(); - if (hasAddSubscriptionToTxn()) { - subBuilder.mergeFrom(getAddSubscriptionToTxn()); - } - input.readMessage(subBuilder, extensionRegistry); - setAddSubscriptionToTxn(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 442: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.newBuilder(); - if (hasAddSubscriptionToTxnResponse()) { - subBuilder.mergeFrom(getAddSubscriptionToTxnResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setAddSubscriptionToTxnResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 450: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.newBuilder(); - if (hasEndTxn()) { - subBuilder.mergeFrom(getEndTxn()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxn(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 458: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.newBuilder(); - if (hasEndTxnResponse()) { - subBuilder.mergeFrom(getEndTxnResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxnResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 466: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.newBuilder(); - if (hasEndTxnOnPartition()) { - subBuilder.mergeFrom(getEndTxnOnPartition()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxnOnPartition(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 474: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.newBuilder(); - if (hasEndTxnOnPartitionResponse()) { - subBuilder.mergeFrom(getEndTxnOnPartitionResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxnOnPartitionResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 482: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.newBuilder(); - if (hasEndTxnOnSubscription()) { - subBuilder.mergeFrom(getEndTxnOnSubscription()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxnOnSubscription(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - case 490: { - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.newBuilder(); - if (hasEndTxnOnSubscriptionResponse()) { - subBuilder.mergeFrom(getEndTxnOnSubscriptionResponse()); - } - input.readMessage(subBuilder, extensionRegistry); - setEndTxnOnSubscriptionResponse(subBuilder.buildPartial()); - subBuilder.recycle(); - break; - } - } - } - } - - private int bitField0_; - private int bitField1_; - - // required .pulsar.proto.BaseCommand.Type type = 1; - private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; - public boolean hasType() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType() { - return type_; - } - public Builder setType(org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000001; - type_ = value; - - return this; - } - public Builder clearType() { - bitField0_ = (bitField0_ & ~0x00000001); - type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; - - return this; - } - - // optional .pulsar.proto.CommandConnect connect = 2; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); - public boolean hasConnect() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect() { - return connect_; - } - public Builder setConnect(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect value) { - if (value == null) { - throw new NullPointerException(); - } - connect_ = value; - - bitField0_ |= 0x00000002; - return this; - } - public Builder setConnect( - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.Builder builderForValue) { - connect_ = builderForValue.build(); - - bitField0_ |= 0x00000002; - return this; - } - public Builder mergeConnect(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect value) { - if (((bitField0_ & 0x00000002) == 0x00000002) && - connect_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance()) { - connect_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.newBuilder(connect_).mergeFrom(value).buildPartial(); - } else { - connect_ = value; - } - - bitField0_ |= 0x00000002; - return this; - } - public Builder clearConnect() { - connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - // optional .pulsar.proto.CommandConnected connected = 3; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); - public boolean hasConnected() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected() { - return connected_; - } - public Builder setConnected(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected value) { - if (value == null) { - throw new NullPointerException(); - } - connected_ = value; - - bitField0_ |= 0x00000004; - return this; - } - public Builder setConnected( - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.Builder builderForValue) { - connected_ = builderForValue.build(); - - bitField0_ |= 0x00000004; - return this; - } - public Builder mergeConnected(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected value) { - if (((bitField0_ & 0x00000004) == 0x00000004) && - connected_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance()) { - connected_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.newBuilder(connected_).mergeFrom(value).buildPartial(); - } else { - connected_ = value; - } - - bitField0_ |= 0x00000004; - return this; - } - public Builder clearConnected() { - connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000004); - return this; - } - - // optional .pulsar.proto.CommandSubscribe subscribe = 4; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); - public boolean hasSubscribe() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe() { - return subscribe_; - } - public Builder setSubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe value) { - if (value == null) { - throw new NullPointerException(); - } - subscribe_ = value; - - bitField0_ |= 0x00000008; - return this; - } - public Builder setSubscribe( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.Builder builderForValue) { - subscribe_ = builderForValue.build(); - - bitField0_ |= 0x00000008; - return this; - } - public Builder mergeSubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe value) { - if (((bitField0_ & 0x00000008) == 0x00000008) && - subscribe_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance()) { - subscribe_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.newBuilder(subscribe_).mergeFrom(value).buildPartial(); - } else { - subscribe_ = value; - } - - bitField0_ |= 0x00000008; - return this; - } - public Builder clearSubscribe() { - subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - // optional .pulsar.proto.CommandProducer producer = 5; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); - public boolean hasProducer() { - return ((bitField0_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer() { - return producer_; - } - public Builder setProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer value) { - if (value == null) { - throw new NullPointerException(); - } - producer_ = value; - - bitField0_ |= 0x00000010; - return this; - } - public Builder setProducer( - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.Builder builderForValue) { - producer_ = builderForValue.build(); - - bitField0_ |= 0x00000010; - return this; - } - public Builder mergeProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer value) { - if (((bitField0_ & 0x00000010) == 0x00000010) && - producer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance()) { - producer_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.newBuilder(producer_).mergeFrom(value).buildPartial(); - } else { - producer_ = value; - } - - bitField0_ |= 0x00000010; - return this; - } - public Builder clearProducer() { - producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000010); - return this; - } - - // optional .pulsar.proto.CommandSend send = 6; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); - public boolean hasSend() { - return ((bitField0_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend() { - return send_; - } - public Builder setSend(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend value) { - if (value == null) { - throw new NullPointerException(); - } - send_ = value; - - bitField0_ |= 0x00000020; - return this; - } - public Builder setSend( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.Builder builderForValue) { - send_ = builderForValue.build(); - - bitField0_ |= 0x00000020; - return this; - } - public Builder mergeSend(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend value) { - if (((bitField0_ & 0x00000020) == 0x00000020) && - send_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance()) { - send_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.newBuilder(send_).mergeFrom(value).buildPartial(); - } else { - send_ = value; - } - - bitField0_ |= 0x00000020; - return this; - } - public Builder clearSend() { - send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000020); - return this; - } - - // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); - public boolean hasSendReceipt() { - return ((bitField0_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt() { - return sendReceipt_; - } - public Builder setSendReceipt(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt value) { - if (value == null) { - throw new NullPointerException(); - } - sendReceipt_ = value; - - bitField0_ |= 0x00000040; - return this; - } - public Builder setSendReceipt( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.Builder builderForValue) { - sendReceipt_ = builderForValue.build(); - - bitField0_ |= 0x00000040; - return this; - } - public Builder mergeSendReceipt(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt value) { - if (((bitField0_ & 0x00000040) == 0x00000040) && - sendReceipt_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance()) { - sendReceipt_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.newBuilder(sendReceipt_).mergeFrom(value).buildPartial(); - } else { - sendReceipt_ = value; - } - - bitField0_ |= 0x00000040; - return this; - } - public Builder clearSendReceipt() { - sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000040); - return this; - } - - // optional .pulsar.proto.CommandSendError send_error = 8; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); - public boolean hasSendError() { - return ((bitField0_ & 0x00000080) == 0x00000080); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError() { - return sendError_; - } - public Builder setSendError(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError value) { - if (value == null) { - throw new NullPointerException(); - } - sendError_ = value; - - bitField0_ |= 0x00000080; - return this; - } - public Builder setSendError( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.Builder builderForValue) { - sendError_ = builderForValue.build(); - - bitField0_ |= 0x00000080; - return this; - } - public Builder mergeSendError(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError value) { - if (((bitField0_ & 0x00000080) == 0x00000080) && - sendError_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance()) { - sendError_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.newBuilder(sendError_).mergeFrom(value).buildPartial(); - } else { - sendError_ = value; - } - - bitField0_ |= 0x00000080; - return this; - } - public Builder clearSendError() { - sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000080); - return this; - } - - // optional .pulsar.proto.CommandMessage message = 9; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); - public boolean hasMessage() { - return ((bitField0_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage() { - return message_; - } - public Builder setMessage(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage value) { - if (value == null) { - throw new NullPointerException(); - } - message_ = value; - - bitField0_ |= 0x00000100; - return this; - } - public Builder setMessage( - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.Builder builderForValue) { - message_ = builderForValue.build(); - - bitField0_ |= 0x00000100; - return this; - } - public Builder mergeMessage(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage value) { - if (((bitField0_ & 0x00000100) == 0x00000100) && - message_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance()) { - message_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.newBuilder(message_).mergeFrom(value).buildPartial(); - } else { - message_ = value; - } - - bitField0_ |= 0x00000100; - return this; - } - public Builder clearMessage() { - message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000100); - return this; - } - - // optional .pulsar.proto.CommandAck ack = 10; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); - public boolean hasAck() { - return ((bitField0_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck() { - return ack_; - } - public Builder setAck(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck value) { - if (value == null) { - throw new NullPointerException(); - } - ack_ = value; - - bitField0_ |= 0x00000200; - return this; - } - public Builder setAck( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.Builder builderForValue) { - ack_ = builderForValue.build(); - - bitField0_ |= 0x00000200; - return this; - } - public Builder mergeAck(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck value) { - if (((bitField0_ & 0x00000200) == 0x00000200) && - ack_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance()) { - ack_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.newBuilder(ack_).mergeFrom(value).buildPartial(); - } else { - ack_ = value; - } - - bitField0_ |= 0x00000200; - return this; - } - public Builder clearAck() { - ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000200); - return this; - } - - // optional .pulsar.proto.CommandFlow flow = 11; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); - public boolean hasFlow() { - return ((bitField0_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow() { - return flow_; - } - public Builder setFlow(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow value) { - if (value == null) { - throw new NullPointerException(); - } - flow_ = value; - - bitField0_ |= 0x00000400; - return this; - } - public Builder setFlow( - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.Builder builderForValue) { - flow_ = builderForValue.build(); - - bitField0_ |= 0x00000400; - return this; - } - public Builder mergeFlow(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow value) { - if (((bitField0_ & 0x00000400) == 0x00000400) && - flow_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance()) { - flow_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.newBuilder(flow_).mergeFrom(value).buildPartial(); - } else { - flow_ = value; - } - - bitField0_ |= 0x00000400; - return this; - } - public Builder clearFlow() { - flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000400); - return this; - } - - // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); - public boolean hasUnsubscribe() { - return ((bitField0_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe() { - return unsubscribe_; - } - public Builder setUnsubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe value) { - if (value == null) { - throw new NullPointerException(); - } - unsubscribe_ = value; - - bitField0_ |= 0x00000800; - return this; - } - public Builder setUnsubscribe( - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.Builder builderForValue) { - unsubscribe_ = builderForValue.build(); - - bitField0_ |= 0x00000800; - return this; - } - public Builder mergeUnsubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe value) { - if (((bitField0_ & 0x00000800) == 0x00000800) && - unsubscribe_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance()) { - unsubscribe_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.newBuilder(unsubscribe_).mergeFrom(value).buildPartial(); - } else { - unsubscribe_ = value; - } - - bitField0_ |= 0x00000800; - return this; - } - public Builder clearUnsubscribe() { - unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00000800); - return this; - } - - // optional .pulsar.proto.CommandSuccess success = 13; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); - public boolean hasSuccess() { - return ((bitField0_ & 0x00001000) == 0x00001000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess() { - return success_; - } - public Builder setSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess value) { - if (value == null) { - throw new NullPointerException(); - } - success_ = value; - - bitField0_ |= 0x00001000; - return this; - } - public Builder setSuccess( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.Builder builderForValue) { - success_ = builderForValue.build(); - - bitField0_ |= 0x00001000; - return this; - } - public Builder mergeSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess value) { - if (((bitField0_ & 0x00001000) == 0x00001000) && - success_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance()) { - success_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.newBuilder(success_).mergeFrom(value).buildPartial(); - } else { - success_ = value; - } - - bitField0_ |= 0x00001000; - return this; - } - public Builder clearSuccess() { - success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00001000); - return this; - } - - // optional .pulsar.proto.CommandError error = 14; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandError error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); - public boolean hasError() { - return ((bitField0_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError() { - return error_; - } - public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.CommandError value) { - if (value == null) { - throw new NullPointerException(); - } - error_ = value; - - bitField0_ |= 0x00002000; - return this; - } - public Builder setError( - org.apache.pulsar.common.api.proto.PulsarApi.CommandError.Builder builderForValue) { - error_ = builderForValue.build(); - - bitField0_ |= 0x00002000; - return this; - } - public Builder mergeError(org.apache.pulsar.common.api.proto.PulsarApi.CommandError value) { - if (((bitField0_ & 0x00002000) == 0x00002000) && - error_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance()) { - error_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandError.newBuilder(error_).mergeFrom(value).buildPartial(); - } else { - error_ = value; - } - - bitField0_ |= 0x00002000; - return this; - } - public Builder clearError() { - error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00002000); - return this; - } - - // optional .pulsar.proto.CommandCloseProducer close_producer = 15; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); - public boolean hasCloseProducer() { - return ((bitField0_ & 0x00004000) == 0x00004000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer() { - return closeProducer_; - } - public Builder setCloseProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer value) { - if (value == null) { - throw new NullPointerException(); - } - closeProducer_ = value; - - bitField0_ |= 0x00004000; - return this; - } - public Builder setCloseProducer( - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.Builder builderForValue) { - closeProducer_ = builderForValue.build(); - - bitField0_ |= 0x00004000; - return this; - } - public Builder mergeCloseProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer value) { - if (((bitField0_ & 0x00004000) == 0x00004000) && - closeProducer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance()) { - closeProducer_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.newBuilder(closeProducer_).mergeFrom(value).buildPartial(); - } else { - closeProducer_ = value; - } - - bitField0_ |= 0x00004000; - return this; - } - public Builder clearCloseProducer() { - closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00004000); - return this; - } - - // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); - public boolean hasCloseConsumer() { - return ((bitField0_ & 0x00008000) == 0x00008000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer() { - return closeConsumer_; - } - public Builder setCloseConsumer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer value) { - if (value == null) { - throw new NullPointerException(); - } - closeConsumer_ = value; - - bitField0_ |= 0x00008000; - return this; - } - public Builder setCloseConsumer( - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.Builder builderForValue) { - closeConsumer_ = builderForValue.build(); - - bitField0_ |= 0x00008000; - return this; - } - public Builder mergeCloseConsumer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer value) { - if (((bitField0_ & 0x00008000) == 0x00008000) && - closeConsumer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance()) { - closeConsumer_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.newBuilder(closeConsumer_).mergeFrom(value).buildPartial(); - } else { - closeConsumer_ = value; - } - - bitField0_ |= 0x00008000; - return this; - } - public Builder clearCloseConsumer() { - closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00008000); - return this; - } - - // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); - public boolean hasProducerSuccess() { - return ((bitField0_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess() { - return producerSuccess_; - } - public Builder setProducerSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess value) { - if (value == null) { - throw new NullPointerException(); - } - producerSuccess_ = value; - - bitField0_ |= 0x00010000; - return this; - } - public Builder setProducerSuccess( - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.Builder builderForValue) { - producerSuccess_ = builderForValue.build(); - - bitField0_ |= 0x00010000; - return this; - } - public Builder mergeProducerSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess value) { - if (((bitField0_ & 0x00010000) == 0x00010000) && - producerSuccess_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance()) { - producerSuccess_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.newBuilder(producerSuccess_).mergeFrom(value).buildPartial(); - } else { - producerSuccess_ = value; - } - - bitField0_ |= 0x00010000; - return this; - } - public Builder clearProducerSuccess() { - producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00010000); - return this; - } - - // optional .pulsar.proto.CommandPing ping = 18; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); - public boolean hasPing() { - return ((bitField0_ & 0x00020000) == 0x00020000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing() { - return ping_; - } - public Builder setPing(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing value) { - if (value == null) { - throw new NullPointerException(); - } - ping_ = value; - - bitField0_ |= 0x00020000; - return this; - } - public Builder setPing( - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.Builder builderForValue) { - ping_ = builderForValue.build(); - - bitField0_ |= 0x00020000; - return this; - } - public Builder mergePing(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing value) { - if (((bitField0_ & 0x00020000) == 0x00020000) && - ping_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance()) { - ping_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.newBuilder(ping_).mergeFrom(value).buildPartial(); - } else { - ping_ = value; - } - - bitField0_ |= 0x00020000; - return this; - } - public Builder clearPing() { - ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00020000); - return this; - } - - // optional .pulsar.proto.CommandPong pong = 19; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); - public boolean hasPong() { - return ((bitField0_ & 0x00040000) == 0x00040000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong() { - return pong_; - } - public Builder setPong(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong value) { - if (value == null) { - throw new NullPointerException(); - } - pong_ = value; - - bitField0_ |= 0x00040000; - return this; - } - public Builder setPong( - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.Builder builderForValue) { - pong_ = builderForValue.build(); - - bitField0_ |= 0x00040000; - return this; - } - public Builder mergePong(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong value) { - if (((bitField0_ & 0x00040000) == 0x00040000) && - pong_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance()) { - pong_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.newBuilder(pong_).mergeFrom(value).buildPartial(); - } else { - pong_ = value; - } - - bitField0_ |= 0x00040000; - return this; - } - public Builder clearPong() { - pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00040000); - return this; - } - - // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); - public boolean hasRedeliverUnacknowledgedMessages() { - return ((bitField0_ & 0x00080000) == 0x00080000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages() { - return redeliverUnacknowledgedMessages_; - } - public Builder setRedeliverUnacknowledgedMessages(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages value) { - if (value == null) { - throw new NullPointerException(); - } - redeliverUnacknowledgedMessages_ = value; - - bitField0_ |= 0x00080000; - return this; - } - public Builder setRedeliverUnacknowledgedMessages( - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.Builder builderForValue) { - redeliverUnacknowledgedMessages_ = builderForValue.build(); - - bitField0_ |= 0x00080000; - return this; - } - public Builder mergeRedeliverUnacknowledgedMessages(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages value) { - if (((bitField0_ & 0x00080000) == 0x00080000) && - redeliverUnacknowledgedMessages_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance()) { - redeliverUnacknowledgedMessages_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.newBuilder(redeliverUnacknowledgedMessages_).mergeFrom(value).buildPartial(); - } else { - redeliverUnacknowledgedMessages_ = value; - } - - bitField0_ |= 0x00080000; - return this; - } - public Builder clearRedeliverUnacknowledgedMessages() { - redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00080000); - return this; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); - public boolean hasPartitionMetadata() { - return ((bitField0_ & 0x00100000) == 0x00100000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata() { - return partitionMetadata_; - } - public Builder setPartitionMetadata(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata value) { - if (value == null) { - throw new NullPointerException(); - } - partitionMetadata_ = value; - - bitField0_ |= 0x00100000; - return this; - } - public Builder setPartitionMetadata( - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.Builder builderForValue) { - partitionMetadata_ = builderForValue.build(); - - bitField0_ |= 0x00100000; - return this; - } - public Builder mergePartitionMetadata(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata value) { - if (((bitField0_ & 0x00100000) == 0x00100000) && - partitionMetadata_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance()) { - partitionMetadata_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.newBuilder(partitionMetadata_).mergeFrom(value).buildPartial(); - } else { - partitionMetadata_ = value; - } - - bitField0_ |= 0x00100000; - return this; - } - public Builder clearPartitionMetadata() { - partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00100000); - return this; - } - - // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); - public boolean hasPartitionMetadataResponse() { - return ((bitField0_ & 0x00200000) == 0x00200000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse() { - return partitionMetadataResponse_; - } - public Builder setPartitionMetadataResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse value) { - if (value == null) { - throw new NullPointerException(); - } - partitionMetadataResponse_ = value; - - bitField0_ |= 0x00200000; - return this; - } - public Builder setPartitionMetadataResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.Builder builderForValue) { - partitionMetadataResponse_ = builderForValue.build(); - - bitField0_ |= 0x00200000; - return this; - } - public Builder mergePartitionMetadataResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse value) { - if (((bitField0_ & 0x00200000) == 0x00200000) && - partitionMetadataResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance()) { - partitionMetadataResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.newBuilder(partitionMetadataResponse_).mergeFrom(value).buildPartial(); - } else { - partitionMetadataResponse_ = value; - } - - bitField0_ |= 0x00200000; - return this; - } - public Builder clearPartitionMetadataResponse() { - partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00200000); - return this; - } - - // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); - public boolean hasLookupTopic() { - return ((bitField0_ & 0x00400000) == 0x00400000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic() { - return lookupTopic_; - } - public Builder setLookupTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic value) { - if (value == null) { - throw new NullPointerException(); - } - lookupTopic_ = value; - - bitField0_ |= 0x00400000; - return this; - } - public Builder setLookupTopic( - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.Builder builderForValue) { - lookupTopic_ = builderForValue.build(); - - bitField0_ |= 0x00400000; - return this; - } - public Builder mergeLookupTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic value) { - if (((bitField0_ & 0x00400000) == 0x00400000) && - lookupTopic_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance()) { - lookupTopic_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.newBuilder(lookupTopic_).mergeFrom(value).buildPartial(); - } else { - lookupTopic_ = value; - } - - bitField0_ |= 0x00400000; - return this; - } - public Builder clearLookupTopic() { - lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00400000); - return this; - } - - // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); - public boolean hasLookupTopicResponse() { - return ((bitField0_ & 0x00800000) == 0x00800000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse() { - return lookupTopicResponse_; - } - public Builder setLookupTopicResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse value) { - if (value == null) { - throw new NullPointerException(); - } - lookupTopicResponse_ = value; - - bitField0_ |= 0x00800000; - return this; - } - public Builder setLookupTopicResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.Builder builderForValue) { - lookupTopicResponse_ = builderForValue.build(); - - bitField0_ |= 0x00800000; - return this; - } - public Builder mergeLookupTopicResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse value) { - if (((bitField0_ & 0x00800000) == 0x00800000) && - lookupTopicResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance()) { - lookupTopicResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.newBuilder(lookupTopicResponse_).mergeFrom(value).buildPartial(); - } else { - lookupTopicResponse_ = value; - } - - bitField0_ |= 0x00800000; - return this; - } - public Builder clearLookupTopicResponse() { - lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x00800000); - return this; - } - - // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); - public boolean hasConsumerStats() { - return ((bitField0_ & 0x01000000) == 0x01000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() { - return consumerStats_; - } - public Builder setConsumerStats(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) { - if (value == null) { - throw new NullPointerException(); - } - consumerStats_ = value; - - bitField0_ |= 0x01000000; - return this; - } - public Builder setConsumerStats( - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder builderForValue) { - consumerStats_ = builderForValue.build(); - - bitField0_ |= 0x01000000; - return this; - } - public Builder mergeConsumerStats(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) { - if (((bitField0_ & 0x01000000) == 0x01000000) && - consumerStats_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) { - consumerStats_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder(consumerStats_).mergeFrom(value).buildPartial(); - } else { - consumerStats_ = value; - } - - bitField0_ |= 0x01000000; - return this; - } - public Builder clearConsumerStats() { - consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x01000000); - return this; - } - - // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); - public boolean hasConsumerStatsResponse() { - return ((bitField0_ & 0x02000000) == 0x02000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() { - return consumerStatsResponse_; - } - public Builder setConsumerStatsResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) { - if (value == null) { - throw new NullPointerException(); - } - consumerStatsResponse_ = value; - - bitField0_ |= 0x02000000; - return this; - } - public Builder setConsumerStatsResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder builderForValue) { - consumerStatsResponse_ = builderForValue.build(); - - bitField0_ |= 0x02000000; - return this; - } - public Builder mergeConsumerStatsResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) { - if (((bitField0_ & 0x02000000) == 0x02000000) && - consumerStatsResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) { - consumerStatsResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder(consumerStatsResponse_).mergeFrom(value).buildPartial(); - } else { - consumerStatsResponse_ = value; - } - - bitField0_ |= 0x02000000; - return this; - } - public Builder clearConsumerStatsResponse() { - consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x02000000); - return this; - } - - // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); - public boolean hasReachedEndOfTopic() { - return ((bitField0_ & 0x04000000) == 0x04000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic() { - return reachedEndOfTopic_; - } - public Builder setReachedEndOfTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic value) { - if (value == null) { - throw new NullPointerException(); - } - reachedEndOfTopic_ = value; - - bitField0_ |= 0x04000000; - return this; - } - public Builder setReachedEndOfTopic( - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.Builder builderForValue) { - reachedEndOfTopic_ = builderForValue.build(); - - bitField0_ |= 0x04000000; - return this; - } - public Builder mergeReachedEndOfTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic value) { - if (((bitField0_ & 0x04000000) == 0x04000000) && - reachedEndOfTopic_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance()) { - reachedEndOfTopic_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.newBuilder(reachedEndOfTopic_).mergeFrom(value).buildPartial(); - } else { - reachedEndOfTopic_ = value; - } - - bitField0_ |= 0x04000000; - return this; - } - public Builder clearReachedEndOfTopic() { - reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x04000000); - return this; - } - - // optional .pulsar.proto.CommandSeek seek = 28; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); - public boolean hasSeek() { - return ((bitField0_ & 0x08000000) == 0x08000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek() { - return seek_; - } - public Builder setSeek(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek value) { - if (value == null) { - throw new NullPointerException(); - } - seek_ = value; - - bitField0_ |= 0x08000000; - return this; - } - public Builder setSeek( - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.Builder builderForValue) { - seek_ = builderForValue.build(); - - bitField0_ |= 0x08000000; - return this; - } - public Builder mergeSeek(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek value) { - if (((bitField0_ & 0x08000000) == 0x08000000) && - seek_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance()) { - seek_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.newBuilder(seek_).mergeFrom(value).buildPartial(); - } else { - seek_ = value; - } - - bitField0_ |= 0x08000000; - return this; - } - public Builder clearSeek() { - seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x08000000); - return this; - } - - // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); - public boolean hasGetLastMessageId() { - return ((bitField0_ & 0x10000000) == 0x10000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId() { - return getLastMessageId_; - } - public Builder setGetLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId value) { - if (value == null) { - throw new NullPointerException(); - } - getLastMessageId_ = value; - - bitField0_ |= 0x10000000; - return this; - } - public Builder setGetLastMessageId( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.Builder builderForValue) { - getLastMessageId_ = builderForValue.build(); - - bitField0_ |= 0x10000000; - return this; - } - public Builder mergeGetLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId value) { - if (((bitField0_ & 0x10000000) == 0x10000000) && - getLastMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance()) { - getLastMessageId_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.newBuilder(getLastMessageId_).mergeFrom(value).buildPartial(); - } else { - getLastMessageId_ = value; - } - - bitField0_ |= 0x10000000; - return this; - } - public Builder clearGetLastMessageId() { - getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x10000000); - return this; - } - - // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); - public boolean hasGetLastMessageIdResponse() { - return ((bitField0_ & 0x20000000) == 0x20000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse() { - return getLastMessageIdResponse_; - } - public Builder setGetLastMessageIdResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse value) { - if (value == null) { - throw new NullPointerException(); - } - getLastMessageIdResponse_ = value; - - bitField0_ |= 0x20000000; - return this; - } - public Builder setGetLastMessageIdResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.Builder builderForValue) { - getLastMessageIdResponse_ = builderForValue.build(); - - bitField0_ |= 0x20000000; - return this; - } - public Builder mergeGetLastMessageIdResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse value) { - if (((bitField0_ & 0x20000000) == 0x20000000) && - getLastMessageIdResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance()) { - getLastMessageIdResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.newBuilder(getLastMessageIdResponse_).mergeFrom(value).buildPartial(); - } else { - getLastMessageIdResponse_ = value; - } - - bitField0_ |= 0x20000000; - return this; - } - public Builder clearGetLastMessageIdResponse() { - getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x20000000); - return this; - } - - // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); - public boolean hasActiveConsumerChange() { - return ((bitField0_ & 0x40000000) == 0x40000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange() { - return activeConsumerChange_; - } - public Builder setActiveConsumerChange(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange value) { - if (value == null) { - throw new NullPointerException(); - } - activeConsumerChange_ = value; - - bitField0_ |= 0x40000000; - return this; - } - public Builder setActiveConsumerChange( - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.Builder builderForValue) { - activeConsumerChange_ = builderForValue.build(); - - bitField0_ |= 0x40000000; - return this; - } - public Builder mergeActiveConsumerChange(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange value) { - if (((bitField0_ & 0x40000000) == 0x40000000) && - activeConsumerChange_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance()) { - activeConsumerChange_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.newBuilder(activeConsumerChange_).mergeFrom(value).buildPartial(); - } else { - activeConsumerChange_ = value; - } - - bitField0_ |= 0x40000000; - return this; - } - public Builder clearActiveConsumerChange() { - activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x40000000); - return this; - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); - public boolean hasGetTopicsOfNamespace() { - return ((bitField0_ & 0x80000000) == 0x80000000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace() { - return getTopicsOfNamespace_; - } - public Builder setGetTopicsOfNamespace(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace value) { - if (value == null) { - throw new NullPointerException(); - } - getTopicsOfNamespace_ = value; - - bitField0_ |= 0x80000000; - return this; - } - public Builder setGetTopicsOfNamespace( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Builder builderForValue) { - getTopicsOfNamespace_ = builderForValue.build(); - - bitField0_ |= 0x80000000; - return this; - } - public Builder mergeGetTopicsOfNamespace(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace value) { - if (((bitField0_ & 0x80000000) == 0x80000000) && - getTopicsOfNamespace_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance()) { - getTopicsOfNamespace_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.newBuilder(getTopicsOfNamespace_).mergeFrom(value).buildPartial(); - } else { - getTopicsOfNamespace_ = value; - } - - bitField0_ |= 0x80000000; - return this; - } - public Builder clearGetTopicsOfNamespace() { - getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); - - bitField0_ = (bitField0_ & ~0x80000000); - return this; - } - - // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); - public boolean hasGetTopicsOfNamespaceResponse() { - return ((bitField1_ & 0x00000001) == 0x00000001); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse() { - return getTopicsOfNamespaceResponse_; - } - public Builder setGetTopicsOfNamespaceResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse value) { - if (value == null) { - throw new NullPointerException(); - } - getTopicsOfNamespaceResponse_ = value; - - bitField1_ |= 0x00000001; - return this; - } - public Builder setGetTopicsOfNamespaceResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.Builder builderForValue) { - getTopicsOfNamespaceResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000001; - return this; - } - public Builder mergeGetTopicsOfNamespaceResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse value) { - if (((bitField1_ & 0x00000001) == 0x00000001) && - getTopicsOfNamespaceResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance()) { - getTopicsOfNamespaceResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.newBuilder(getTopicsOfNamespaceResponse_).mergeFrom(value).buildPartial(); - } else { - getTopicsOfNamespaceResponse_ = value; - } - - bitField1_ |= 0x00000001; - return this; - } - public Builder clearGetTopicsOfNamespaceResponse() { - getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000001); - return this; - } - - // optional .pulsar.proto.CommandGetSchema getSchema = 34; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); - public boolean hasGetSchema() { - return ((bitField1_ & 0x00000002) == 0x00000002); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema() { - return getSchema_; - } - public Builder setGetSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema value) { - if (value == null) { - throw new NullPointerException(); - } - getSchema_ = value; - - bitField1_ |= 0x00000002; - return this; - } - public Builder setGetSchema( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.Builder builderForValue) { - getSchema_ = builderForValue.build(); - - bitField1_ |= 0x00000002; - return this; - } - public Builder mergeGetSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema value) { - if (((bitField1_ & 0x00000002) == 0x00000002) && - getSchema_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance()) { - getSchema_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.newBuilder(getSchema_).mergeFrom(value).buildPartial(); - } else { - getSchema_ = value; - } - - bitField1_ |= 0x00000002; - return this; - } - public Builder clearGetSchema() { - getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000002); - return this; - } - - // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); - public boolean hasGetSchemaResponse() { - return ((bitField1_ & 0x00000004) == 0x00000004); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse() { - return getSchemaResponse_; - } - public Builder setGetSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse value) { - if (value == null) { - throw new NullPointerException(); - } - getSchemaResponse_ = value; - - bitField1_ |= 0x00000004; - return this; - } - public Builder setGetSchemaResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.Builder builderForValue) { - getSchemaResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000004; - return this; - } - public Builder mergeGetSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse value) { - if (((bitField1_ & 0x00000004) == 0x00000004) && - getSchemaResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance()) { - getSchemaResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.newBuilder(getSchemaResponse_).mergeFrom(value).buildPartial(); - } else { - getSchemaResponse_ = value; - } - - bitField1_ |= 0x00000004; - return this; - } - public Builder clearGetSchemaResponse() { - getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000004); - return this; - } - - // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); - public boolean hasAuthChallenge() { - return ((bitField1_ & 0x00000008) == 0x00000008); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge() { - return authChallenge_; - } - public Builder setAuthChallenge(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge value) { - if (value == null) { - throw new NullPointerException(); - } - authChallenge_ = value; - - bitField1_ |= 0x00000008; - return this; - } - public Builder setAuthChallenge( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.Builder builderForValue) { - authChallenge_ = builderForValue.build(); - - bitField1_ |= 0x00000008; - return this; - } - public Builder mergeAuthChallenge(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge value) { - if (((bitField1_ & 0x00000008) == 0x00000008) && - authChallenge_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance()) { - authChallenge_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.newBuilder(authChallenge_).mergeFrom(value).buildPartial(); - } else { - authChallenge_ = value; - } - - bitField1_ |= 0x00000008; - return this; - } - public Builder clearAuthChallenge() { - authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000008); - return this; - } - - // optional .pulsar.proto.CommandAuthResponse authResponse = 37; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); - public boolean hasAuthResponse() { - return ((bitField1_ & 0x00000010) == 0x00000010); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse() { - return authResponse_; - } - public Builder setAuthResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse value) { - if (value == null) { - throw new NullPointerException(); - } - authResponse_ = value; - - bitField1_ |= 0x00000010; - return this; - } - public Builder setAuthResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.Builder builderForValue) { - authResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000010; - return this; - } - public Builder mergeAuthResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse value) { - if (((bitField1_ & 0x00000010) == 0x00000010) && - authResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance()) { - authResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.newBuilder(authResponse_).mergeFrom(value).buildPartial(); - } else { - authResponse_ = value; - } - - bitField1_ |= 0x00000010; - return this; - } - public Builder clearAuthResponse() { - authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000010); - return this; - } - - // optional .pulsar.proto.CommandAckResponse ackResponse = 38; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); - public boolean hasAckResponse() { - return ((bitField1_ & 0x00000020) == 0x00000020); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse() { - return ackResponse_; - } - public Builder setAckResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse value) { - if (value == null) { - throw new NullPointerException(); - } - ackResponse_ = value; - - bitField1_ |= 0x00000020; - return this; - } - public Builder setAckResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.Builder builderForValue) { - ackResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000020; - return this; - } - public Builder mergeAckResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse value) { - if (((bitField1_ & 0x00000020) == 0x00000020) && - ackResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance()) { - ackResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.newBuilder(ackResponse_).mergeFrom(value).buildPartial(); - } else { - ackResponse_ = value; - } - - bitField1_ |= 0x00000020; - return this; - } - public Builder clearAckResponse() { - ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000020); - return this; - } - - // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); - public boolean hasGetOrCreateSchema() { - return ((bitField1_ & 0x00000040) == 0x00000040); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema() { - return getOrCreateSchema_; - } - public Builder setGetOrCreateSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema value) { - if (value == null) { - throw new NullPointerException(); - } - getOrCreateSchema_ = value; - - bitField1_ |= 0x00000040; - return this; - } - public Builder setGetOrCreateSchema( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.Builder builderForValue) { - getOrCreateSchema_ = builderForValue.build(); - - bitField1_ |= 0x00000040; - return this; - } - public Builder mergeGetOrCreateSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema value) { - if (((bitField1_ & 0x00000040) == 0x00000040) && - getOrCreateSchema_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance()) { - getOrCreateSchema_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.newBuilder(getOrCreateSchema_).mergeFrom(value).buildPartial(); - } else { - getOrCreateSchema_ = value; - } - - bitField1_ |= 0x00000040; - return this; - } - public Builder clearGetOrCreateSchema() { - getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000040); - return this; - } - - // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); - public boolean hasGetOrCreateSchemaResponse() { - return ((bitField1_ & 0x00000080) == 0x00000080); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse() { - return getOrCreateSchemaResponse_; - } - public Builder setGetOrCreateSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse value) { - if (value == null) { - throw new NullPointerException(); - } - getOrCreateSchemaResponse_ = value; - - bitField1_ |= 0x00000080; - return this; - } - public Builder setGetOrCreateSchemaResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.Builder builderForValue) { - getOrCreateSchemaResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000080; - return this; - } - public Builder mergeGetOrCreateSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse value) { - if (((bitField1_ & 0x00000080) == 0x00000080) && - getOrCreateSchemaResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance()) { - getOrCreateSchemaResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.newBuilder(getOrCreateSchemaResponse_).mergeFrom(value).buildPartial(); - } else { - getOrCreateSchemaResponse_ = value; - } - - bitField1_ |= 0x00000080; - return this; - } - public Builder clearGetOrCreateSchemaResponse() { - getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000080); - return this; - } - - // optional .pulsar.proto.CommandNewTxn newTxn = 50; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); - public boolean hasNewTxn() { - return ((bitField1_ & 0x00000100) == 0x00000100); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn() { - return newTxn_; - } - public Builder setNewTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn value) { - if (value == null) { - throw new NullPointerException(); - } - newTxn_ = value; - - bitField1_ |= 0x00000100; - return this; - } - public Builder setNewTxn( - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.Builder builderForValue) { - newTxn_ = builderForValue.build(); - - bitField1_ |= 0x00000100; - return this; - } - public Builder mergeNewTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn value) { - if (((bitField1_ & 0x00000100) == 0x00000100) && - newTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance()) { - newTxn_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.newBuilder(newTxn_).mergeFrom(value).buildPartial(); - } else { - newTxn_ = value; - } - - bitField1_ |= 0x00000100; - return this; - } - public Builder clearNewTxn() { - newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000100); - return this; - } - - // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); - public boolean hasNewTxnResponse() { - return ((bitField1_ & 0x00000200) == 0x00000200); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse() { - return newTxnResponse_; - } - public Builder setNewTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse value) { - if (value == null) { - throw new NullPointerException(); - } - newTxnResponse_ = value; - - bitField1_ |= 0x00000200; - return this; - } - public Builder setNewTxnResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.Builder builderForValue) { - newTxnResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000200; - return this; - } - public Builder mergeNewTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse value) { - if (((bitField1_ & 0x00000200) == 0x00000200) && - newTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance()) { - newTxnResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.newBuilder(newTxnResponse_).mergeFrom(value).buildPartial(); - } else { - newTxnResponse_ = value; - } - - bitField1_ |= 0x00000200; - return this; - } - public Builder clearNewTxnResponse() { - newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000200); - return this; - } - - // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); - public boolean hasAddPartitionToTxn() { - return ((bitField1_ & 0x00000400) == 0x00000400); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn() { - return addPartitionToTxn_; - } - public Builder setAddPartitionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn value) { - if (value == null) { - throw new NullPointerException(); - } - addPartitionToTxn_ = value; - - bitField1_ |= 0x00000400; - return this; - } - public Builder setAddPartitionToTxn( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.Builder builderForValue) { - addPartitionToTxn_ = builderForValue.build(); - - bitField1_ |= 0x00000400; - return this; - } - public Builder mergeAddPartitionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn value) { - if (((bitField1_ & 0x00000400) == 0x00000400) && - addPartitionToTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance()) { - addPartitionToTxn_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.newBuilder(addPartitionToTxn_).mergeFrom(value).buildPartial(); - } else { - addPartitionToTxn_ = value; - } - - bitField1_ |= 0x00000400; - return this; - } - public Builder clearAddPartitionToTxn() { - addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000400); - return this; - } - - // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); - public boolean hasAddPartitionToTxnResponse() { - return ((bitField1_ & 0x00000800) == 0x00000800); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse() { - return addPartitionToTxnResponse_; - } - public Builder setAddPartitionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse value) { - if (value == null) { - throw new NullPointerException(); - } - addPartitionToTxnResponse_ = value; - - bitField1_ |= 0x00000800; - return this; - } - public Builder setAddPartitionToTxnResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.Builder builderForValue) { - addPartitionToTxnResponse_ = builderForValue.build(); - - bitField1_ |= 0x00000800; - return this; - } - public Builder mergeAddPartitionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse value) { - if (((bitField1_ & 0x00000800) == 0x00000800) && - addPartitionToTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance()) { - addPartitionToTxnResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.newBuilder(addPartitionToTxnResponse_).mergeFrom(value).buildPartial(); - } else { - addPartitionToTxnResponse_ = value; - } - - bitField1_ |= 0x00000800; - return this; - } - public Builder clearAddPartitionToTxnResponse() { - addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00000800); - return this; - } - - // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); - public boolean hasAddSubscriptionToTxn() { - return ((bitField1_ & 0x00001000) == 0x00001000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn() { - return addSubscriptionToTxn_; - } - public Builder setAddSubscriptionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn value) { - if (value == null) { - throw new NullPointerException(); - } - addSubscriptionToTxn_ = value; - - bitField1_ |= 0x00001000; - return this; - } - public Builder setAddSubscriptionToTxn( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.Builder builderForValue) { - addSubscriptionToTxn_ = builderForValue.build(); - - bitField1_ |= 0x00001000; - return this; - } - public Builder mergeAddSubscriptionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn value) { - if (((bitField1_ & 0x00001000) == 0x00001000) && - addSubscriptionToTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance()) { - addSubscriptionToTxn_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.newBuilder(addSubscriptionToTxn_).mergeFrom(value).buildPartial(); - } else { - addSubscriptionToTxn_ = value; - } - - bitField1_ |= 0x00001000; - return this; - } - public Builder clearAddSubscriptionToTxn() { - addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00001000); - return this; - } - - // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); - public boolean hasAddSubscriptionToTxnResponse() { - return ((bitField1_ & 0x00002000) == 0x00002000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse() { - return addSubscriptionToTxnResponse_; - } - public Builder setAddSubscriptionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse value) { - if (value == null) { - throw new NullPointerException(); - } - addSubscriptionToTxnResponse_ = value; - - bitField1_ |= 0x00002000; - return this; - } - public Builder setAddSubscriptionToTxnResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.Builder builderForValue) { - addSubscriptionToTxnResponse_ = builderForValue.build(); - - bitField1_ |= 0x00002000; - return this; - } - public Builder mergeAddSubscriptionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse value) { - if (((bitField1_ & 0x00002000) == 0x00002000) && - addSubscriptionToTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance()) { - addSubscriptionToTxnResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.newBuilder(addSubscriptionToTxnResponse_).mergeFrom(value).buildPartial(); - } else { - addSubscriptionToTxnResponse_ = value; - } - - bitField1_ |= 0x00002000; - return this; - } - public Builder clearAddSubscriptionToTxnResponse() { - addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00002000); - return this; - } - - // optional .pulsar.proto.CommandEndTxn endTxn = 56; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); - public boolean hasEndTxn() { - return ((bitField1_ & 0x00004000) == 0x00004000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn() { - return endTxn_; - } - public Builder setEndTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn value) { - if (value == null) { - throw new NullPointerException(); - } - endTxn_ = value; - - bitField1_ |= 0x00004000; - return this; - } - public Builder setEndTxn( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.Builder builderForValue) { - endTxn_ = builderForValue.build(); - - bitField1_ |= 0x00004000; - return this; - } - public Builder mergeEndTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn value) { - if (((bitField1_ & 0x00004000) == 0x00004000) && - endTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance()) { - endTxn_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.newBuilder(endTxn_).mergeFrom(value).buildPartial(); - } else { - endTxn_ = value; - } - - bitField1_ |= 0x00004000; - return this; - } - public Builder clearEndTxn() { - endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00004000); - return this; - } - - // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); - public boolean hasEndTxnResponse() { - return ((bitField1_ & 0x00008000) == 0x00008000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse() { - return endTxnResponse_; - } - public Builder setEndTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse value) { - if (value == null) { - throw new NullPointerException(); - } - endTxnResponse_ = value; - - bitField1_ |= 0x00008000; - return this; - } - public Builder setEndTxnResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.Builder builderForValue) { - endTxnResponse_ = builderForValue.build(); - - bitField1_ |= 0x00008000; - return this; - } - public Builder mergeEndTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse value) { - if (((bitField1_ & 0x00008000) == 0x00008000) && - endTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance()) { - endTxnResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.newBuilder(endTxnResponse_).mergeFrom(value).buildPartial(); - } else { - endTxnResponse_ = value; - } - - bitField1_ |= 0x00008000; - return this; - } - public Builder clearEndTxnResponse() { - endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00008000); - return this; - } - - // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); - public boolean hasEndTxnOnPartition() { - return ((bitField1_ & 0x00010000) == 0x00010000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition() { - return endTxnOnPartition_; - } - public Builder setEndTxnOnPartition(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition value) { - if (value == null) { - throw new NullPointerException(); - } - endTxnOnPartition_ = value; - - bitField1_ |= 0x00010000; - return this; - } - public Builder setEndTxnOnPartition( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.Builder builderForValue) { - endTxnOnPartition_ = builderForValue.build(); - - bitField1_ |= 0x00010000; - return this; - } - public Builder mergeEndTxnOnPartition(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition value) { - if (((bitField1_ & 0x00010000) == 0x00010000) && - endTxnOnPartition_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance()) { - endTxnOnPartition_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.newBuilder(endTxnOnPartition_).mergeFrom(value).buildPartial(); - } else { - endTxnOnPartition_ = value; - } - - bitField1_ |= 0x00010000; - return this; - } - public Builder clearEndTxnOnPartition() { - endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00010000); - return this; - } - - // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); - public boolean hasEndTxnOnPartitionResponse() { - return ((bitField1_ & 0x00020000) == 0x00020000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse() { - return endTxnOnPartitionResponse_; - } - public Builder setEndTxnOnPartitionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse value) { - if (value == null) { - throw new NullPointerException(); - } - endTxnOnPartitionResponse_ = value; - - bitField1_ |= 0x00020000; - return this; - } - public Builder setEndTxnOnPartitionResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.Builder builderForValue) { - endTxnOnPartitionResponse_ = builderForValue.build(); - - bitField1_ |= 0x00020000; - return this; - } - public Builder mergeEndTxnOnPartitionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse value) { - if (((bitField1_ & 0x00020000) == 0x00020000) && - endTxnOnPartitionResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance()) { - endTxnOnPartitionResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.newBuilder(endTxnOnPartitionResponse_).mergeFrom(value).buildPartial(); - } else { - endTxnOnPartitionResponse_ = value; - } - - bitField1_ |= 0x00020000; - return this; - } - public Builder clearEndTxnOnPartitionResponse() { - endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00020000); - return this; - } - - // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); - public boolean hasEndTxnOnSubscription() { - return ((bitField1_ & 0x00040000) == 0x00040000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription() { - return endTxnOnSubscription_; - } - public Builder setEndTxnOnSubscription(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription value) { - if (value == null) { - throw new NullPointerException(); - } - endTxnOnSubscription_ = value; - - bitField1_ |= 0x00040000; - return this; - } - public Builder setEndTxnOnSubscription( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.Builder builderForValue) { - endTxnOnSubscription_ = builderForValue.build(); - - bitField1_ |= 0x00040000; - return this; - } - public Builder mergeEndTxnOnSubscription(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription value) { - if (((bitField1_ & 0x00040000) == 0x00040000) && - endTxnOnSubscription_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance()) { - endTxnOnSubscription_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.newBuilder(endTxnOnSubscription_).mergeFrom(value).buildPartial(); - } else { - endTxnOnSubscription_ = value; - } - - bitField1_ |= 0x00040000; - return this; - } - public Builder clearEndTxnOnSubscription() { - endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00040000); - return this; - } - - // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; - private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); - public boolean hasEndTxnOnSubscriptionResponse() { - return ((bitField1_ & 0x00080000) == 0x00080000); - } - public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse() { - return endTxnOnSubscriptionResponse_; - } - public Builder setEndTxnOnSubscriptionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse value) { - if (value == null) { - throw new NullPointerException(); - } - endTxnOnSubscriptionResponse_ = value; - - bitField1_ |= 0x00080000; - return this; - } - public Builder setEndTxnOnSubscriptionResponse( - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.Builder builderForValue) { - endTxnOnSubscriptionResponse_ = builderForValue.build(); - - bitField1_ |= 0x00080000; - return this; - } - public Builder mergeEndTxnOnSubscriptionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse value) { - if (((bitField1_ & 0x00080000) == 0x00080000) && - endTxnOnSubscriptionResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance()) { - endTxnOnSubscriptionResponse_ = - org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.newBuilder(endTxnOnSubscriptionResponse_).mergeFrom(value).buildPartial(); - } else { - endTxnOnSubscriptionResponse_ = value; - } - - bitField1_ |= 0x00080000; - return this; - } - public Builder clearEndTxnOnSubscriptionResponse() { - endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); - - bitField1_ = (bitField1_ & ~0x00080000); - return this; - } - - // @@protoc_insertion_point(builder_scope:pulsar.proto.BaseCommand) - } - - static { - defaultInstance = new BaseCommand(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:pulsar.proto.BaseCommand) - } - - - static { - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index b1f5d09d020fb..4b6ce54b6062e 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -30,6 +30,7 @@ import io.netty.buffer.Unpooled; import java.io.IOException; +import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; @@ -40,6 +41,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -48,7 +50,6 @@ import org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod; import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand; import org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type; -import org.apache.pulsar.common.api.proto.PulsarApi.BatchMessageIndexesAckData; 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.CommandAck.ValidationError; @@ -119,6 +120,7 @@ import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import org.apache.pulsar.common.util.SafeCollectionUtils; import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream; import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream; import org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString; @@ -439,15 +441,15 @@ public static void skipMessageMetadata(ByteBuf buffer) { } public static ByteBufPair newMessage(long consumerId, MessageIdData messageId, int redeliveryCount, - ByteBuf metadataAndPayload, List ackedBatchIndexRanges) { + ByteBuf metadataAndPayload, long[] ackSet) { CommandMessage.Builder msgBuilder = CommandMessage.newBuilder(); msgBuilder.setConsumerId(consumerId); msgBuilder.setMessageId(messageId); if (redeliveryCount > 0) { msgBuilder.setRedeliveryCount(redeliveryCount); } - if (ackedBatchIndexRanges != null && ackedBatchIndexRanges.size() > 0) { - msgBuilder.addAllAckedIndexes(ackedBatchIndexRanges); + if (ackSet != null) { + msgBuilder.addAllAckSet(SafeCollectionUtils.longArrayToList(ackSet)); } CommandMessage msg = msgBuilder.build(); BaseCommand.Builder cmdBuilder = BaseCommand.newBuilder(); @@ -864,7 +866,7 @@ public static ByteBuf newLookupErrorResponse(ServerError error, String errorMsg, return res; } - public static ByteBuf newMultiMessageAck(long consumerId, List> entries) { + public static ByteBuf newMultiMessageAck(long consumerId, List> entries) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); ackBuilder.setConsumerId(consumerId); ackBuilder.setAckType(AckType.Individual); @@ -872,11 +874,14 @@ public static ByteBuf newMultiMessageAck(long consumerId, List> int entriesCount = entries.size(); for (int i = 0; i < entriesCount; i++) { long ledgerId = entries.get(i).getLeft(); - long entryId = entries.get(i).getRight(); - + long entryId = entries.get(i).getMiddle(); + BitSet bitSet = entries.get(i).getRight(); MessageIdData.Builder messageIdDataBuilder = MessageIdData.newBuilder(); messageIdDataBuilder.setLedgerId(ledgerId); messageIdDataBuilder.setEntryId(entryId); + if (bitSet != null) { + messageIdDataBuilder.addAllAckSet(SafeCollectionUtils.longArrayToList(bitSet.toLongArray())); + } MessageIdData messageIdData = messageIdDataBuilder.build(); ackBuilder.addMessageId(messageIdData); @@ -895,12 +900,12 @@ public static ByteBuf newMultiMessageAck(long consumerId, List> return res; } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, ValidationError validationError, Map properties) { - return newAck(consumerId, ledgerId, entryId, ackType, validationError, properties, 0, 0); + return newAck(consumerId, ledgerId, entryId, ackSet, ackType, validationError, properties, 0, 0); } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, ValidationError validationError, Map properties, long txnIdLeastBits, long txnIdMostBits) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); @@ -909,6 +914,9 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, AckTy MessageIdData.Builder messageIdDataBuilder = MessageIdData.newBuilder(); messageIdDataBuilder.setLedgerId(ledgerId); messageIdDataBuilder.setEntryId(entryId); + if (ackSet != null) { + messageIdDataBuilder.addAllAckSet(SafeCollectionUtils.longArrayToList(ackSet.toLongArray())); + } MessageIdData messageIdData = messageIdDataBuilder.build(); ackBuilder.addMessageId(messageIdData); if (validationError != null) { @@ -934,56 +942,6 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, AckTy return res; } - public static ByteBuf newBatchIndexAck(long consumerId, - List batchMessageIndexesAckData, - AckType ackType, ValidationError validationError, Map properties) { - CommandAck.Builder ackBuilder = CommandAck.newBuilder(); - ackBuilder.setConsumerId(consumerId); - ackBuilder.setAckType(ackType); - MessageIdData.Builder messageIdDataBuilder = MessageIdData.newBuilder(); - batchMessageIndexesAckData.forEach(singleBatchMessageIndexesAck -> { - BatchMessageIndexesAckData.Builder indexAckBuilder = BatchMessageIndexesAckData.newBuilder(); - messageIdDataBuilder.setLedgerId(singleBatchMessageIndexesAck.getLedgerId()); - messageIdDataBuilder.setEntryId(singleBatchMessageIndexesAck.getEntryId()); - indexAckBuilder.setMessageId(messageIdDataBuilder.build()); - indexAckBuilder.setBatchSize(singleBatchMessageIndexesAck.getBatchSize()); - if (ackType == AckType.Individual) { - indexAckBuilder.addAllAckIndexes(singleBatchMessageIndexesAck.getIndexRangesToAck()); - } else { - if (singleBatchMessageIndexesAck.getIndexRangesToAck() != null - && singleBatchMessageIndexesAck.getIndexRangesToAck().size() == 1) { - indexAckBuilder.addAckIndexes(singleBatchMessageIndexesAck.getIndexRangesToAck().get(0)); - } - } - ackBuilder.addBatchMessageAckIndexes(indexAckBuilder.build()); - indexAckBuilder.recycle(); - }); - - if (validationError != null) { - ackBuilder.setValidationError(validationError); - } - - if (properties != null) { - for (Map.Entry e : properties.entrySet()) { - ackBuilder.addProperties( - KeyLongValue.newBuilder().setKey(e.getKey()).setValue(e.getValue()).build()); - } - } - - CommandAck ack = ackBuilder.build(); - - ByteBuf res = serializeWithSize(BaseCommand.newBuilder().setType(Type.ACK).setAck(ack)); - ack.getBatchMessageAckIndexesList().forEach(ackData -> { - ackData.getMessageId().recycle(); - ackData.getAckIndexesList().forEach(IntRange::recycle); - ackData.recycle(); - }); - ack.recycle(); - messageIdDataBuilder.recycle(); - ackBuilder.recycle(); - return res; - } - public static ByteBuf newAckResponse(long consumerId, long txnIdLeastBits, long txnIdMostBits) { CommandAckResponse.Builder commandAckResponseBuilder = CommandAckResponse.newBuilder(); commandAckResponseBuilder.setConsumerId(consumerId); @@ -1817,54 +1775,4 @@ public static boolean peerSupportJsonSchemaAvroFormat(int peerVersion) { public static boolean peerSupportsGetOrCreateSchema(int peerVersion) { return peerVersion >= ProtocolVersion.v15.getNumber(); } - - public static class SingleBatchMessageIndexesAck { - private long ledgerId; - private long entryId; - private int batchSize; - private List indexRangesToAck; - - public SingleBatchMessageIndexesAck() { - } - - public SingleBatchMessageIndexesAck(long ledgerId, long entryId, int batchSize, - List indexRangesToAck) { - this.ledgerId = ledgerId; - this.entryId = entryId; - this.batchSize = batchSize; - this.indexRangesToAck = indexRangesToAck; - } - - public long getLedgerId() { - return ledgerId; - } - - public void setLedgerId(long ledgerId) { - this.ledgerId = ledgerId; - } - - public long getEntryId() { - return entryId; - } - - public void setEntryId(long entryId) { - this.entryId = entryId; - } - - public int getBatchSize() { - return batchSize; - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - public List getIndexRangesToAck() { - return indexRangesToAck; - } - - public void setIndexRangesToAck(List indexRangesToAck) { - this.indexRangesToAck = indexRangesToAck; - } - } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java new file mode 100644 index 0000000000000..180e59b6e5c6e --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java @@ -0,0 +1,53 @@ +/** + * 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.util; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Safe collection utils. + */ +public class SafeCollectionUtils { + + public static List longArrayToList(long[] array) { + if (array == null || array.length == 0) { + return Collections.emptyList(); + } else { + List result = new ArrayList<>(array.length); + for (long l : array) { + result.add(l); + } + return result; + } + } + + public static long[] longListToArray(List list) { + if (list == null || list.size() == 0) { + return new long[0]; + } else { + long[] array = new long[list.size()]; + for (int i = 0; i < list.size(); i++) { + array[i] = list.get(i); + } + return array; + } + } +} diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index f16083998a29a..c05d31bc94268 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -54,12 +54,7 @@ message MessageIdData { required uint64 entryId = 2; optional int32 partition = 3 [default = -1]; optional int32 batch_index = 4 [default = -1]; -} - -message BatchMessageIndexesAckData { - required MessageIdData message_id = 1; - required int32 batchSize = 2; - repeated IntRange ack_indexes = 3; + repeated int64 ackSet = 5; } message KeyValue { @@ -459,7 +454,7 @@ message CommandMessage { required uint64 consumer_id = 1; required MessageIdData message_id = 2; optional uint32 redelivery_count = 3 [default = 0]; - repeated IntRange acked_indexes = 4; + repeated int64 ack_set = 4; } message CommandAck { @@ -490,8 +485,6 @@ message CommandAck { optional uint64 txnid_least_bits = 6 [default = 0]; optional uint64 txnid_most_bits = 7 [default = 0]; - - repeated BatchMessageIndexesAckData batch_message_ack_indexes = 8; } message CommandAckResponse { From 6c8725c7ad4e9fcc50e6f1c7918f60a536d158d2 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 30 Jan 2020 22:29:06 +0800 Subject: [PATCH 03/21] Fix comments --- .../broker/service/EntryBatchIndexesAcks.java | 15 +- .../pulsar/client/impl/RawReaderImpl.java | 2 +- .../apache/pulsar/client/impl/ClientCnx.java | 2 +- .../client/impl/ZeroQueueConsumerImpl.java | 2 +- .../pulsar/common/api/proto/PulsarApi.java | 43264 ++++++++++++++++ 5 files changed, 43275 insertions(+), 10 deletions(-) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java index f0ca64ed9ba4c..89c0c771c66ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java @@ -21,28 +21,29 @@ import io.netty.util.Recycler; import org.apache.commons.lang3.tuple.Pair; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import java.util.ArrayList; +import java.util.BitSet; import java.util.List; public class EntryBatchIndexesAcks { - List>> indexesAcks = new ArrayList<>(); + List> indexesAcks = new ArrayList<>(); - public void setIndexesAcks(Pair> indexesAcks) { + public void setIndexesAcks(Pair indexesAcks) { this.indexesAcks.add(indexesAcks); } - public Pair> getIndexesAcks(int entryIdx) { - return this.indexesAcks.get(entryIdx); + public long[] getAckSet(int entryIdx) { + Pair pair = indexesAcks.get(entryIdx); + return pair == null ? null : pair.getRight(); } public int getTotalAckedIndexCount() { int count = 0; - for (Pair> pair : indexesAcks) { + for (Pair pair : indexesAcks) { if (pair != null) { - count += pair.getLeft(); + count += pair.getLeft() - BitSet.valueOf(pair.getRight()).cardinality(); } } return count; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java index ec622c828ff7a..f7f35fa5c0907 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java @@ -204,7 +204,7 @@ public CompletableFuture closeAsync() { } @Override - void messageReceived(MessageIdData messageId, int redeliveryCount, List ackedBatchIndexRanges, ByteBuf headersAndPayload, ClientCnx cnx) { + void messageReceived(MessageIdData messageId, int redeliveryCount, List ackSet, ByteBuf headersAndPayload, ClientCnx cnx) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Received raw message: {}/{}/{}", topic, subscription, messageId.getEntryId(), messageId.getLedgerId(), messageId.getPartition()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java index 8218fefba83d1..6dde157ce8482 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java @@ -384,7 +384,7 @@ protected void handleMessage(CommandMessage cmdMessage, ByteBuf headersAndPayloa } ConsumerImpl consumer = consumers.get(cmdMessage.getConsumerId()); if (consumer != null) { - consumer.messageReceived(cmdMessage.getMessageId(), cmdMessage.getRedeliveryCount(), cmdMessage.getAckedIndexesList(), headersAndPayload, this); + consumer.messageReceived(cmdMessage.getMessageId(), cmdMessage.getRedeliveryCount(), cmdMessage.getAckSetList(), headersAndPayload, this); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java index 7ddb3639a1ef2..79e368e7e2703 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ZeroQueueConsumerImpl.java @@ -179,7 +179,7 @@ protected void triggerListener(int numMessages) { @Override void receiveIndividualMessagesFromBatch(MessageMetadata msgMetadata, int redeliveryCount, - List ackedBatchIndexRanges, + List ackSet, ByteBuf uncompressedPayload, MessageIdData messageId, ClientCnx cnx) { log.warn( "Closing consumer [{}]-[{}] due to unsupported received batch-message with zero receiver queue size", 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 new file mode 100644 index 0000000000000..a00005f43def6 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/api/proto/PulsarApi.java @@ -0,0 +1,43264 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: src/main/proto/PulsarApi.proto + +package org.apache.pulsar.common.api.proto; + +public final class PulsarApi { + private PulsarApi() {} + public static void registerAllExtensions( + org.apache.pulsar.shaded.com.google.protobuf.v241.ExtensionRegistryLite registry) { + } + public enum CompressionType + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + NONE(0, 0), + LZ4(1, 1), + ZLIB(2, 2), + ZSTD(3, 3), + SNAPPY(4, 4), + ; + + public static final int NONE_VALUE = 0; + public static final int LZ4_VALUE = 1; + public static final int ZLIB_VALUE = 2; + public static final int ZSTD_VALUE = 3; + public static final int SNAPPY_VALUE = 4; + + + public final int getNumber() { return value; } + + public static CompressionType valueOf(int value) { + switch (value) { + case 0: return NONE; + case 1: return LZ4; + case 2: return ZLIB; + case 3: return ZSTD; + case 4: return SNAPPY; + 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 CompressionType findValueByNumber(int number) { + return CompressionType.valueOf(number); + } + }; + + private final int value; + + private CompressionType(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CompressionType) + } + + public enum ServerError + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + UnknownError(0, 0), + MetadataError(1, 1), + PersistenceError(2, 2), + AuthenticationError(3, 3), + AuthorizationError(4, 4), + ConsumerBusy(5, 5), + ServiceNotReady(6, 6), + ProducerBlockedQuotaExceededError(7, 7), + ProducerBlockedQuotaExceededException(8, 8), + ChecksumError(9, 9), + UnsupportedVersionError(10, 10), + TopicNotFound(11, 11), + SubscriptionNotFound(12, 12), + ConsumerNotFound(13, 13), + TooManyRequests(14, 14), + TopicTerminatedError(15, 15), + ProducerBusy(16, 16), + InvalidTopicName(17, 17), + IncompatibleSchema(18, 18), + ConsumerAssignError(19, 19), + TransactionCoordinatorNotFound(20, 20), + InvalidTxnStatus(21, 21), + ; + + public static final int UnknownError_VALUE = 0; + public static final int MetadataError_VALUE = 1; + public static final int PersistenceError_VALUE = 2; + public static final int AuthenticationError_VALUE = 3; + public static final int AuthorizationError_VALUE = 4; + public static final int ConsumerBusy_VALUE = 5; + public static final int ServiceNotReady_VALUE = 6; + public static final int ProducerBlockedQuotaExceededError_VALUE = 7; + public static final int ProducerBlockedQuotaExceededException_VALUE = 8; + public static final int ChecksumError_VALUE = 9; + public static final int UnsupportedVersionError_VALUE = 10; + public static final int TopicNotFound_VALUE = 11; + public static final int SubscriptionNotFound_VALUE = 12; + public static final int ConsumerNotFound_VALUE = 13; + public static final int TooManyRequests_VALUE = 14; + public static final int TopicTerminatedError_VALUE = 15; + public static final int ProducerBusy_VALUE = 16; + public static final int InvalidTopicName_VALUE = 17; + public static final int IncompatibleSchema_VALUE = 18; + public static final int ConsumerAssignError_VALUE = 19; + public static final int TransactionCoordinatorNotFound_VALUE = 20; + public static final int InvalidTxnStatus_VALUE = 21; + + + public final int getNumber() { return value; } + + public static ServerError valueOf(int value) { + switch (value) { + case 0: return UnknownError; + case 1: return MetadataError; + case 2: return PersistenceError; + case 3: return AuthenticationError; + case 4: return AuthorizationError; + case 5: return ConsumerBusy; + case 6: return ServiceNotReady; + case 7: return ProducerBlockedQuotaExceededError; + case 8: return ProducerBlockedQuotaExceededException; + case 9: return ChecksumError; + case 10: return UnsupportedVersionError; + case 11: return TopicNotFound; + case 12: return SubscriptionNotFound; + case 13: return ConsumerNotFound; + case 14: return TooManyRequests; + case 15: return TopicTerminatedError; + case 16: return ProducerBusy; + case 17: return InvalidTopicName; + case 18: return IncompatibleSchema; + case 19: return ConsumerAssignError; + case 20: return TransactionCoordinatorNotFound; + case 21: return InvalidTxnStatus; + 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 ServerError findValueByNumber(int number) { + return ServerError.valueOf(number); + } + }; + + private final int value; + + private ServerError(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.ServerError) + } + + public enum AuthMethod + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + AuthMethodNone(0, 0), + AuthMethodYcaV1(1, 1), + AuthMethodAthens(2, 2), + ; + + public static final int AuthMethodNone_VALUE = 0; + public static final int AuthMethodYcaV1_VALUE = 1; + public static final int AuthMethodAthens_VALUE = 2; + + + public final int getNumber() { return value; } + + public static AuthMethod valueOf(int value) { + switch (value) { + case 0: return AuthMethodNone; + case 1: return AuthMethodYcaV1; + case 2: return AuthMethodAthens; + 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 AuthMethod findValueByNumber(int number) { + return AuthMethod.valueOf(number); + } + }; + + private final int value; + + private AuthMethod(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.AuthMethod) + } + + public enum ProtocolVersion + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + v0(0, 0), + v1(1, 1), + v2(2, 2), + v3(3, 3), + v4(4, 4), + v5(5, 5), + v6(6, 6), + v7(7, 7), + v8(8, 8), + v9(9, 9), + v10(10, 10), + v11(11, 11), + v12(12, 12), + v13(13, 13), + v14(14, 14), + v15(15, 15), + ; + + public static final int v0_VALUE = 0; + public static final int v1_VALUE = 1; + public static final int v2_VALUE = 2; + public static final int v3_VALUE = 3; + public static final int v4_VALUE = 4; + public static final int v5_VALUE = 5; + public static final int v6_VALUE = 6; + public static final int v7_VALUE = 7; + public static final int v8_VALUE = 8; + public static final int v9_VALUE = 9; + public static final int v10_VALUE = 10; + public static final int v11_VALUE = 11; + public static final int v12_VALUE = 12; + public static final int v13_VALUE = 13; + public static final int v14_VALUE = 14; + public static final int v15_VALUE = 15; + + + public final int getNumber() { return value; } + + public static ProtocolVersion valueOf(int value) { + switch (value) { + case 0: return v0; + case 1: return v1; + case 2: return v2; + case 3: return v3; + case 4: return v4; + case 5: return v5; + case 6: return v6; + case 7: return v7; + case 8: return v8; + case 9: return v9; + case 10: return v10; + case 11: return v11; + case 12: return v12; + case 13: return v13; + case 14: return v14; + case 15: return v15; + 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 ProtocolVersion findValueByNumber(int number) { + return ProtocolVersion.valueOf(number); + } + }; + + private final int value; + + private ProtocolVersion(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.ProtocolVersion) + } + + public enum KeySharedMode + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + AUTO_SPLIT(0, 0), + STICKY(1, 1), + ; + + public static final int AUTO_SPLIT_VALUE = 0; + public static final int STICKY_VALUE = 1; + + + public final int getNumber() { return value; } + + public static KeySharedMode valueOf(int value) { + switch (value) { + case 0: return AUTO_SPLIT; + case 1: return STICKY; + 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 KeySharedMode findValueByNumber(int number) { + return KeySharedMode.valueOf(number); + } + }; + + private final int value; + + private KeySharedMode(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.KeySharedMode) + } + + public enum TxnAction + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + COMMIT(0, 0), + ABORT(1, 1), + ; + + public static final int COMMIT_VALUE = 0; + public static final int ABORT_VALUE = 1; + + + public final int getNumber() { return value; } + + public static TxnAction valueOf(int value) { + switch (value) { + case 0: return COMMIT; + case 1: return ABORT; + 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 TxnAction findValueByNumber(int number) { + return TxnAction.valueOf(number); + } + }; + + private final int value; + + private TxnAction(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.TxnAction) + } + + public interface SchemaOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string name = 1; + boolean hasName(); + String getName(); + + // required bytes schema_data = 3; + boolean hasSchemaData(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData(); + + // required .pulsar.proto.Schema.Type type = 4; + boolean hasType(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType(); + + // repeated .pulsar.proto.KeyValue properties = 5; + java.util.List + getPropertiesList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); + int getPropertiesCount(); + } + public static final class Schema extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements SchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use Schema.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private Schema(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected Schema newObject(Handle handle) { + return new Schema(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private Schema(boolean noInit) {} + + private static final Schema defaultInstance; + public static Schema getDefaultInstance() { + return defaultInstance; + } + + public Schema getDefaultInstanceForType() { + return defaultInstance; + } + + public enum Type + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + None(0, 0), + String(1, 1), + Json(2, 2), + Protobuf(3, 3), + Avro(4, 4), + Bool(5, 5), + Int8(6, 6), + Int16(7, 7), + Int32(8, 8), + Int64(9, 9), + Float(10, 10), + Double(11, 11), + Date(12, 12), + Time(13, 13), + Timestamp(14, 14), + KeyValue(15, 15), + ; + + public static final int None_VALUE = 0; + public static final int String_VALUE = 1; + public static final int Json_VALUE = 2; + public static final int Protobuf_VALUE = 3; + public static final int Avro_VALUE = 4; + public static final int Bool_VALUE = 5; + public static final int Int8_VALUE = 6; + public static final int Int16_VALUE = 7; + public static final int Int32_VALUE = 8; + public static final int Int64_VALUE = 9; + public static final int Float_VALUE = 10; + public static final int Double_VALUE = 11; + public static final int Date_VALUE = 12; + public static final int Time_VALUE = 13; + public static final int Timestamp_VALUE = 14; + public static final int KeyValue_VALUE = 15; + + + public final int getNumber() { return value; } + + public static Type valueOf(int value) { + switch (value) { + case 0: return None; + case 1: return String; + case 2: return Json; + case 3: return Protobuf; + case 4: return Avro; + case 5: return Bool; + case 6: return Int8; + case 7: return Int16; + case 8: return Int32; + case 9: return Int64; + case 10: return Float; + case 11: return Double; + case 12: return Date; + case 13: return Time; + case 14: return Timestamp; + case 15: return KeyValue; + 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 Type findValueByNumber(int number) { + return Type.valueOf(number); + } + }; + + private final int value; + + private Type(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.Schema.Type) + } + + private int bitField0_; + // required string name = 1; + public static final int NAME_FIELD_NUMBER = 1; + private java.lang.Object name_; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + 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)) { + name_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getNameBytes() { + java.lang.Object ref = name_; + 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); + name_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required bytes schema_data = 3; + public static final int SCHEMA_DATA_FIELD_NUMBER = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaData_; + public boolean hasSchemaData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData() { + return schemaData_; + } + + // required .pulsar.proto.Schema.Type type = 4; + public static final int TYPE_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type type_; + public boolean hasType() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType() { + return type_; + } + + // repeated .pulsar.proto.KeyValue properties = 5; + public static final int PROPERTIES_FIELD_NUMBER = 5; + private java.util.List properties_; + public java.util.List getPropertiesList() { + return properties_; + } + public java.util.List + getPropertiesOrBuilderList() { + return properties_; + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( + int index) { + return properties_.get(index); + } + + private void initFields() { + name_ = ""; + schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; + properties_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSchemaData()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasType()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(3, schemaData_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(4, type_.getNumber()); + } + for (int i = 0; i < properties_.size(); i++) { + output.writeMessage(5, properties_.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, getNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, schemaData_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, type_.getNumber()); + } + for (int i = 0; i < properties_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(5, properties_.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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema 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.PulsarApi.Schema, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.SchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.Schema.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(); + name_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; + bitField0_ = (bitField0_ & ~0x00000004); + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Schema build() { + org.apache.pulsar.common.api.proto.PulsarApi.Schema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.Schema buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.Schema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Schema buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.Schema result = org.apache.pulsar.common.api.proto.PulsarApi.Schema.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.name_ = name_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.schemaData_ = schemaData_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.type_ = type_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + properties_ = java.util.Collections.unmodifiableList(properties_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.properties_ = properties_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.Schema other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) return this; + if (other.hasName()) { + setName(other.getName()); + } + if (other.hasSchemaData()) { + setSchemaData(other.getSchemaData()); + } + if (other.hasType()) { + setType(other.getType()); + } + if (!other.properties_.isEmpty()) { + if (properties_.isEmpty()) { + properties_ = other.properties_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensurePropertiesIsMutable(); + properties_.addAll(other.properties_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasName()) { + + return false; + } + if (!hasSchemaData()) { + + return false; + } + if (!hasType()) { + + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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; + name_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000002; + schemaData_ = input.readBytes(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type value = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + type_ = value; + } + break; + } + case 42: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addProperties(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required string name = 1; + private java.lang.Object name_ = ""; + public boolean hasName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getName() { + java.lang.Object ref = name_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + name_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + name_ = value; + + return this; + } + public Builder clearName() { + bitField0_ = (bitField0_ & ~0x00000001); + name_ = getDefaultInstance().getName(); + + return this; + } + void setName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + name_ = value; + + } + + // required bytes schema_data = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaData() { + return schemaData_; + } + public Builder setSchemaData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + schemaData_ = value; + + return this; + } + public Builder clearSchemaData() { + bitField0_ = (bitField0_ & ~0x00000002); + schemaData_ = getDefaultInstance().getSchemaData(); + + return this; + } + + // required .pulsar.proto.Schema.Type type = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; + public boolean hasType() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type getType() { + return type_; + } + public Builder setType(org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + type_ = value; + + return this; + } + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00000004); + type_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.Type.None; + + return this; + } + + // repeated .pulsar.proto.KeyValue properties = 5; + private java.util.List properties_ = + java.util.Collections.emptyList(); + private void ensurePropertiesIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + properties_ = new java.util.ArrayList(properties_); + bitField0_ |= 0x00000008; + } + } + + public java.util.List getPropertiesList() { + return java.util.Collections.unmodifiableList(properties_); + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.set(index, value); + + return this; + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.set(index, builderForValue.build()); + + return this; + } + public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(value); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(index, value); + + return this; + } + public Builder addProperties( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(builderForValue.build()); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllProperties( + java.lang.Iterable values) { + ensurePropertiesIsMutable(); + super.addAll(values, properties_); + + return this; + } + public Builder clearProperties() { + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + public Builder removeProperties(int index) { + ensurePropertiesIsMutable(); + properties_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.Schema) + } + + static { + defaultInstance = new Schema(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.Schema) + } + + public interface MessageIdDataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 ledgerId = 1; + boolean hasLedgerId(); + long getLedgerId(); + + // required uint64 entryId = 2; + boolean hasEntryId(); + long getEntryId(); + + // optional int32 partition = 3 [default = -1]; + boolean hasPartition(); + int getPartition(); + + // optional int32 batch_index = 4 [default = -1]; + boolean hasBatchIndex(); + int getBatchIndex(); + + // repeated int64 ackSet = 5; + java.util.List getAckSetList(); + int getAckSetCount(); + long getAckSet(int index); + } + 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 ledgerId = 1; + public static final int LEDGERID_FIELD_NUMBER = 1; + private long ledgerId_; + public boolean hasLedgerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getLedgerId() { + return ledgerId_; + } + + // required uint64 entryId = 2; + public static final int ENTRYID_FIELD_NUMBER = 2; + private long entryId_; + public boolean hasEntryId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getEntryId() { + return entryId_; + } + + // optional int32 partition = 3 [default = -1]; + public static final int PARTITION_FIELD_NUMBER = 3; + private int partition_; + public boolean hasPartition() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getPartition() { + return partition_; + } + + // optional int32 batch_index = 4 [default = -1]; + public static final int BATCH_INDEX_FIELD_NUMBER = 4; + private int batchIndex_; + public boolean hasBatchIndex() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getBatchIndex() { + return batchIndex_; + } + + // repeated int64 ackSet = 5; + public static final int ACKSET_FIELD_NUMBER = 5; + private java.util.List ackSet_; + public java.util.List + getAckSetList() { + return ackSet_; + } + public int getAckSetCount() { + return ackSet_.size(); + } + public long getAckSet(int index) { + return ackSet_.get(index); + } + + private void initFields() { + ledgerId_ = 0L; + entryId_ = 0L; + partition_ = -1; + batchIndex_ = -1; + ackSet_ = java.util.Collections.emptyList();; + } + 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_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, partition_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeInt32(4, batchIndex_); + } + for (int i = 0; i < ackSet_.size(); i++) { + output.writeInt64(5, ackSet_.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 + .computeUInt64Size(1, ledgerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, entryId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, partition_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(4, batchIndex_); + } + { + int dataSize = 0; + for (int i = 0; i < ackSet_.size(); i++) { + dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt64SizeNoTag(ackSet_.get(i)); + } + size += dataSize; + size += 1 * getAckSetList().size(); + } + 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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.MessageIdData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.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.PulsarApi.MessageIdData, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.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); + partition_ = -1; + bitField0_ = (bitField0_ & ~0x00000004); + batchIndex_ = -1; + bitField0_ = (bitField0_ & ~0x00000008); + ackSet_ = java.util.Collections.emptyList();; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData build() { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData result = org.apache.pulsar.common.api.proto.PulsarApi.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_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.partition_ = partition_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.batchIndex_ = batchIndex_; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + ackSet_ = java.util.Collections.unmodifiableList(ackSet_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.ackSet_ = ackSet_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) return this; + if (other.hasLedgerId()) { + setLedgerId(other.getLedgerId()); + } + if (other.hasEntryId()) { + setEntryId(other.getEntryId()); + } + if (other.hasPartition()) { + setPartition(other.getPartition()); + } + if (other.hasBatchIndex()) { + setBatchIndex(other.getBatchIndex()); + } + if (!other.ackSet_.isEmpty()) { + if (ackSet_.isEmpty()) { + ackSet_ = other.ackSet_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensureAckSetIsMutable(); + ackSet_.addAll(other.ackSet_); + } + + } + 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; + } + case 24: { + bitField0_ |= 0x00000004; + partition_ = input.readInt32(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + batchIndex_ = input.readInt32(); + break; + } + case 40: { + ensureAckSetIsMutable(); + ackSet_.add(input.readInt64()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 ledgerId = 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 entryId = 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; + } + + // optional int32 partition = 3 [default = -1]; + private int partition_ = -1; + public boolean hasPartition() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getPartition() { + return partition_; + } + public Builder setPartition(int value) { + bitField0_ |= 0x00000004; + partition_ = value; + + return this; + } + public Builder clearPartition() { + bitField0_ = (bitField0_ & ~0x00000004); + partition_ = -1; + + return this; + } + + // optional int32 batch_index = 4 [default = -1]; + private int batchIndex_ = -1; + public boolean hasBatchIndex() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getBatchIndex() { + return batchIndex_; + } + public Builder setBatchIndex(int value) { + bitField0_ |= 0x00000008; + batchIndex_ = value; + + return this; + } + public Builder clearBatchIndex() { + bitField0_ = (bitField0_ & ~0x00000008); + batchIndex_ = -1; + + return this; + } + + // repeated int64 ackSet = 5; + private java.util.List ackSet_ = java.util.Collections.emptyList();; + private void ensureAckSetIsMutable() { + if (!((bitField0_ & 0x00000010) == 0x00000010)) { + ackSet_ = new java.util.ArrayList(ackSet_); + bitField0_ |= 0x00000010; + } + } + public java.util.List + getAckSetList() { + return java.util.Collections.unmodifiableList(ackSet_); + } + public int getAckSetCount() { + return ackSet_.size(); + } + public long getAckSet(int index) { + return ackSet_.get(index); + } + public Builder setAckSet( + int index, long value) { + ensureAckSetIsMutable(); + ackSet_.set(index, value); + + return this; + } + public Builder addAckSet(long value) { + ensureAckSetIsMutable(); + ackSet_.add(value); + + return this; + } + public Builder addAllAckSet( + java.lang.Iterable values) { + ensureAckSetIsMutable(); + super.addAll(values, ackSet_); + + return this; + } + public Builder clearAckSet() { + ackSet_ = java.util.Collections.emptyList();; + bitField0_ = (bitField0_ & ~0x00000010); + + 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) + } + + public interface KeyValueOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string key = 1; + boolean hasKey(); + String getKey(); + + // required string value = 2; + boolean hasValue(); + String getValue(); + } + public static final class KeyValue extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements KeyValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use KeyValue.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private KeyValue(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected KeyValue newObject(Handle handle) { + return new KeyValue(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private KeyValue(boolean noInit) {} + + private static final KeyValue defaultInstance; + public static KeyValue getDefaultInstance() { + return defaultInstance; + } + + public KeyValue getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string key = 1; + public static final int KEY_FIELD_NUMBER = 1; + private java.lang.Object key_; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + 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)) { + key_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { + java.lang.Object ref = key_; + 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); + key_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required string value = 2; + public static final int VALUE_FIELD_NUMBER = 2; + private java.lang.Object value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getValue() { + java.lang.Object ref = value_; + 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)) { + value_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValueBytes() { + java.lang.Object ref = value_; + 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); + value_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + key_ = ""; + value_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasValue()) { + 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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getValueBytes()); + } + } + + 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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getValueBytes()); + } + 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue 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.PulsarApi.KeyValue, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.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(); + key_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + value_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue build() { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.KeyValue buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue result = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.key_ = key_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.value_ = value_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.getDefaultInstance()) return this; + if (other.hasKey()) { + setKey(other.getKey()); + } + if (other.hasValue()) { + setValue(other.getValue()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasKey()) { + + return false; + } + if (!hasValue()) { + + 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; + key_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + value_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string key = 1; + private java.lang.Object key_ = ""; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + key_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + key_ = value; + + return this; + } + public Builder clearKey() { + bitField0_ = (bitField0_ & ~0x00000001); + key_ = getDefaultInstance().getKey(); + + return this; + } + void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + key_ = value; + + } + + // required string value = 2; + private java.lang.Object value_ = ""; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getValue() { + java.lang.Object ref = value_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + value_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setValue(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + value_ = value; + + return this; + } + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000002); + value_ = getDefaultInstance().getValue(); + + return this; + } + void setValue(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + value_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.KeyValue) + } + + static { + defaultInstance = new KeyValue(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.KeyValue) + } + + public interface KeyLongValueOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string key = 1; + boolean hasKey(); + String getKey(); + + // required uint64 value = 2; + boolean hasValue(); + long getValue(); + } + public static final class KeyLongValue extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements KeyLongValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use KeyLongValue.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private KeyLongValue(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected KeyLongValue newObject(Handle handle) { + return new KeyLongValue(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private KeyLongValue(boolean noInit) {} + + private static final KeyLongValue defaultInstance; + public static KeyLongValue getDefaultInstance() { + return defaultInstance; + } + + public KeyLongValue getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string key = 1; + public static final int KEY_FIELD_NUMBER = 1; + private java.lang.Object key_; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + 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)) { + key_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { + java.lang.Object ref = key_; + 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); + key_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required uint64 value = 2; + public static final int VALUE_FIELD_NUMBER = 2; + private long value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getValue() { + return value_; + } + + private void initFields() { + key_ = ""; + value_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasValue()) { + 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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, value_); + } + } + + 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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, value_); + } + 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue 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.PulsarApi.KeyLongValue, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValueOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.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(); + key_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + value_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue build() { + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue result = org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.key_ = key_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.value_ = value_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.getDefaultInstance()) return this; + if (other.hasKey()) { + setKey(other.getKey()); + } + if (other.hasValue()) { + setValue(other.getValue()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasKey()) { + + return false; + } + if (!hasValue()) { + + 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; + key_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + value_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required string key = 1; + private java.lang.Object key_ = ""; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + key_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + key_ = value; + + return this; + } + public Builder clearKey() { + bitField0_ = (bitField0_ & ~0x00000001); + key_ = getDefaultInstance().getKey(); + + return this; + } + void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + key_ = value; + + } + + // required uint64 value = 2; + private long value_ ; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getValue() { + return value_; + } + public Builder setValue(long value) { + bitField0_ |= 0x00000002; + value_ = value; + + return this; + } + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000002); + value_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.KeyLongValue) + } + + static { + defaultInstance = new KeyLongValue(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.KeyLongValue) + } + + public interface IntRangeOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required int32 start = 1; + boolean hasStart(); + int getStart(); + + // required int32 end = 2; + boolean hasEnd(); + int getEnd(); + } + public static final class IntRange extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements IntRangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use IntRange.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private IntRange(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected IntRange newObject(Handle handle) { + return new IntRange(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private IntRange(boolean noInit) {} + + private static final IntRange defaultInstance; + public static IntRange getDefaultInstance() { + return defaultInstance; + } + + public IntRange getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required int32 start = 1; + public static final int START_FIELD_NUMBER = 1; + private int start_; + public boolean hasStart() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getStart() { + return start_; + } + + // required int32 end = 2; + public static final int END_FIELD_NUMBER = 2; + private int end_; + public boolean hasEnd() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getEnd() { + return end_; + } + + private void initFields() { + start_ = 0; + end_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasStart()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasEnd()) { + 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.writeInt32(1, start_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, end_); + } + } + + 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 + .computeInt32Size(1, start_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(2, end_); + } + 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange 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.PulsarApi.IntRange, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.IntRange.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(); + start_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + end_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.IntRange.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange build() { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.IntRange buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange result = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.start_ = start_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.end_ = end_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.IntRange other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.IntRange.getDefaultInstance()) return this; + if (other.hasStart()) { + setStart(other.getStart()); + } + if (other.hasEnd()) { + setEnd(other.getEnd()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasStart()) { + + return false; + } + if (!hasEnd()) { + + 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; + start_ = input.readInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + end_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required int32 start = 1; + private int start_ ; + public boolean hasStart() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getStart() { + return start_; + } + public Builder setStart(int value) { + bitField0_ |= 0x00000001; + start_ = value; + + return this; + } + public Builder clearStart() { + bitField0_ = (bitField0_ & ~0x00000001); + start_ = 0; + + return this; + } + + // required int32 end = 2; + private int end_ ; + public boolean hasEnd() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getEnd() { + return end_; + } + public Builder setEnd(int value) { + bitField0_ |= 0x00000002; + end_ = value; + + return this; + } + public Builder clearEnd() { + bitField0_ = (bitField0_ & ~0x00000002); + end_ = 0; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.IntRange) + } + + static { + defaultInstance = new IntRange(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.IntRange) + } + + public interface EncryptionKeysOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string key = 1; + boolean hasKey(); + String getKey(); + + // required bytes value = 2; + boolean hasValue(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue(); + + // repeated .pulsar.proto.KeyValue metadata = 3; + java.util.List + getMetadataList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); + int getMetadataCount(); + } + public static final class EncryptionKeys extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements EncryptionKeysOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use EncryptionKeys.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private EncryptionKeys(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected EncryptionKeys newObject(Handle handle) { + return new EncryptionKeys(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private EncryptionKeys(boolean noInit) {} + + private static final EncryptionKeys defaultInstance; + public static EncryptionKeys getDefaultInstance() { + return defaultInstance; + } + + public EncryptionKeys getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string key = 1; + public static final int KEY_FIELD_NUMBER = 1; + private java.lang.Object key_; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + 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)) { + key_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getKeyBytes() { + java.lang.Object ref = key_; + 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); + key_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required bytes value = 2; + public static final int VALUE_FIELD_NUMBER = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue() { + return value_; + } + + // repeated .pulsar.proto.KeyValue metadata = 3; + public static final int METADATA_FIELD_NUMBER = 3; + private java.util.List metadata_; + public java.util.List getMetadataList() { + return metadata_; + } + public java.util.List + getMetadataOrBuilderList() { + return metadata_; + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( + int index) { + return metadata_.get(index); + } + + private void initFields() { + key_ = ""; + value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + metadata_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasValue()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, value_); + } + for (int i = 0; i < metadata_.size(); i++) { + output.writeMessage(3, metadata_.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, getKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, value_); + } + for (int i = 0; i < metadata_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, metadata_.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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys 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.PulsarApi.EncryptionKeys, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeysOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.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(); + key_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys build() { + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys result = org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.key_ = key_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.value_ = value_; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + metadata_ = java.util.Collections.unmodifiableList(metadata_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.metadata_ = metadata_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.getDefaultInstance()) return this; + if (other.hasKey()) { + setKey(other.getKey()); + } + if (other.hasValue()) { + setValue(other.getValue()); + } + if (!other.metadata_.isEmpty()) { + if (metadata_.isEmpty()) { + metadata_ = other.metadata_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureMetadataIsMutable(); + metadata_.addAll(other.metadata_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasKey()) { + + return false; + } + if (!hasValue()) { + + return false; + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(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; + key_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + value_ = input.readBytes(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMetadata(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required string key = 1; + private java.lang.Object key_ = ""; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + key_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + key_ = value; + + return this; + } + public Builder clearKey() { + bitField0_ = (bitField0_ & ~0x00000001); + key_ = getDefaultInstance().getKey(); + + return this; + } + void setKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + key_ = value; + + } + + // required bytes value = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getValue() { + return value_; + } + public Builder setValue(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + value_ = value; + + return this; + } + public Builder clearValue() { + bitField0_ = (bitField0_ & ~0x00000002); + value_ = getDefaultInstance().getValue(); + + return this; + } + + // repeated .pulsar.proto.KeyValue metadata = 3; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + private void ensureMetadataIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + metadata_ = new java.util.ArrayList(metadata_); + bitField0_ |= 0x00000004; + } + } + + public java.util.List getMetadataList() { + return java.util.Collections.unmodifiableList(metadata_); + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.set(index, value); + + return this; + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.set(index, builderForValue.build()); + + return this; + } + public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(value); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(index, value); + + return this; + } + public Builder addMetadata( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(builderForValue.build()); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllMetadata( + java.lang.Iterable values) { + ensureMetadataIsMutable(); + super.addAll(values, metadata_); + + return this; + } + public Builder clearMetadata() { + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + + return this; + } + public Builder removeMetadata(int index) { + ensureMetadataIsMutable(); + metadata_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.EncryptionKeys) + } + + static { + defaultInstance = new EncryptionKeys(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.EncryptionKeys) + } + + public interface MessageMetadataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string producer_name = 1; + boolean hasProducerName(); + String getProducerName(); + + // required uint64 sequence_id = 2; + boolean hasSequenceId(); + long getSequenceId(); + + // required uint64 publish_time = 3; + boolean hasPublishTime(); + long getPublishTime(); + + // repeated .pulsar.proto.KeyValue properties = 4; + java.util.List + getPropertiesList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); + int getPropertiesCount(); + + // optional string replicated_from = 5; + boolean hasReplicatedFrom(); + String getReplicatedFrom(); + + // optional string partition_key = 6; + boolean hasPartitionKey(); + String getPartitionKey(); + + // repeated string replicate_to = 7; + java.util.List getReplicateToList(); + int getReplicateToCount(); + String getReplicateTo(int index); + + // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; + boolean hasCompression(); + org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression(); + + // optional uint32 uncompressed_size = 9 [default = 0]; + boolean hasUncompressedSize(); + int getUncompressedSize(); + + // optional int32 num_messages_in_batch = 11 [default = 1]; + boolean hasNumMessagesInBatch(); + int getNumMessagesInBatch(); + + // optional uint64 event_time = 12 [default = 0]; + boolean hasEventTime(); + long getEventTime(); + + // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; + java.util.List + getEncryptionKeysList(); + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index); + int getEncryptionKeysCount(); + + // optional string encryption_algo = 14; + boolean hasEncryptionAlgo(); + String getEncryptionAlgo(); + + // optional bytes encryption_param = 15; + boolean hasEncryptionParam(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam(); + + // optional bytes schema_version = 16; + boolean hasSchemaVersion(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); + + // optional bool partition_key_b64_encoded = 17 [default = false]; + boolean hasPartitionKeyB64Encoded(); + boolean getPartitionKeyB64Encoded(); + + // optional bytes ordering_key = 18; + boolean hasOrderingKey(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey(); + + // optional int64 deliver_at_time = 19; + boolean hasDeliverAtTime(); + long getDeliverAtTime(); + + // optional int32 marker_type = 20; + boolean hasMarkerType(); + int getMarkerType(); + + // optional uint64 txnid_least_bits = 22 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 23 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional uint64 highest_sequence_id = 24 [default = 0]; + boolean hasHighestSequenceId(); + long getHighestSequenceId(); + } + public static final class MessageMetadata extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements MessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use MessageMetadata.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private MessageMetadata(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected MessageMetadata newObject(Handle handle) { + return new MessageMetadata(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private MessageMetadata(boolean noInit) {} + + private static final MessageMetadata defaultInstance; + public static MessageMetadata getDefaultInstance() { + return defaultInstance; + } + + public MessageMetadata getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string producer_name = 1; + public static final int PRODUCER_NAME_FIELD_NUMBER = 1; + private java.lang.Object producerName_; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + 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)) { + producerName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { + java.lang.Object ref = producerName_; + 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); + producerName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required uint64 sequence_id = 2; + public static final int SEQUENCE_ID_FIELD_NUMBER = 2; + private long sequenceId_; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + + // required uint64 publish_time = 3; + public static final int PUBLISH_TIME_FIELD_NUMBER = 3; + private long publishTime_; + public boolean hasPublishTime() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getPublishTime() { + return publishTime_; + } + + // repeated .pulsar.proto.KeyValue properties = 4; + public static final int PROPERTIES_FIELD_NUMBER = 4; + private java.util.List properties_; + public java.util.List getPropertiesList() { + return properties_; + } + public java.util.List + getPropertiesOrBuilderList() { + return properties_; + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( + int index) { + return properties_.get(index); + } + + // optional string replicated_from = 5; + public static final int REPLICATED_FROM_FIELD_NUMBER = 5; + private java.lang.Object replicatedFrom_; + public boolean hasReplicatedFrom() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getReplicatedFrom() { + java.lang.Object ref = replicatedFrom_; + 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)) { + replicatedFrom_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getReplicatedFromBytes() { + java.lang.Object ref = replicatedFrom_; + 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); + replicatedFrom_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string partition_key = 6; + public static final int PARTITION_KEY_FIELD_NUMBER = 6; + private java.lang.Object partitionKey_; + public boolean hasPartitionKey() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getPartitionKey() { + java.lang.Object ref = partitionKey_; + 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)) { + partitionKey_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getPartitionKeyBytes() { + java.lang.Object ref = partitionKey_; + 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); + partitionKey_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // repeated string replicate_to = 7; + public static final int REPLICATE_TO_FIELD_NUMBER = 7; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList replicateTo_; + public java.util.List + getReplicateToList() { + return replicateTo_; + } + public int getReplicateToCount() { + return replicateTo_.size(); + } + public String getReplicateTo(int index) { + return replicateTo_.get(index); + } + + // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; + public static final int COMPRESSION_FIELD_NUMBER = 8; + private org.apache.pulsar.common.api.proto.PulsarApi.CompressionType compression_; + public boolean hasCompression() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression() { + return compression_; + } + + // optional uint32 uncompressed_size = 9 [default = 0]; + public static final int UNCOMPRESSED_SIZE_FIELD_NUMBER = 9; + private int uncompressedSize_; + public boolean hasUncompressedSize() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public int getUncompressedSize() { + return uncompressedSize_; + } + + // optional int32 num_messages_in_batch = 11 [default = 1]; + public static final int NUM_MESSAGES_IN_BATCH_FIELD_NUMBER = 11; + private int numMessagesInBatch_; + public boolean hasNumMessagesInBatch() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public int getNumMessagesInBatch() { + return numMessagesInBatch_; + } + + // optional uint64 event_time = 12 [default = 0]; + public static final int EVENT_TIME_FIELD_NUMBER = 12; + private long eventTime_; + public boolean hasEventTime() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public long getEventTime() { + return eventTime_; + } + + // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; + public static final int ENCRYPTION_KEYS_FIELD_NUMBER = 13; + private java.util.List encryptionKeys_; + public java.util.List getEncryptionKeysList() { + return encryptionKeys_; + } + public java.util.List + getEncryptionKeysOrBuilderList() { + return encryptionKeys_; + } + public int getEncryptionKeysCount() { + return encryptionKeys_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index) { + return encryptionKeys_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeysOrBuilder getEncryptionKeysOrBuilder( + int index) { + return encryptionKeys_.get(index); + } + + // optional string encryption_algo = 14; + public static final int ENCRYPTION_ALGO_FIELD_NUMBER = 14; + private java.lang.Object encryptionAlgo_; + public boolean hasEncryptionAlgo() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public String getEncryptionAlgo() { + java.lang.Object ref = encryptionAlgo_; + 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)) { + encryptionAlgo_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionAlgoBytes() { + java.lang.Object ref = encryptionAlgo_; + 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); + encryptionAlgo_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bytes encryption_param = 15; + public static final int ENCRYPTION_PARAM_FIELD_NUMBER = 15; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString encryptionParam_; + public boolean hasEncryptionParam() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam() { + return encryptionParam_; + } + + // optional bytes schema_version = 16; + public static final int SCHEMA_VERSION_FIELD_NUMBER = 16; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + + // optional bool partition_key_b64_encoded = 17 [default = false]; + public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 17; + private boolean partitionKeyB64Encoded_; + public boolean hasPartitionKeyB64Encoded() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public boolean getPartitionKeyB64Encoded() { + return partitionKeyB64Encoded_; + } + + // optional bytes ordering_key = 18; + public static final int ORDERING_KEY_FIELD_NUMBER = 18; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_; + public boolean hasOrderingKey() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { + return orderingKey_; + } + + // optional int64 deliver_at_time = 19; + public static final int DELIVER_AT_TIME_FIELD_NUMBER = 19; + private long deliverAtTime_; + public boolean hasDeliverAtTime() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public long getDeliverAtTime() { + return deliverAtTime_; + } + + // optional int32 marker_type = 20; + public static final int MARKER_TYPE_FIELD_NUMBER = 20; + private int markerType_; + public boolean hasMarkerType() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public int getMarkerType() { + return markerType_; + } + + // optional uint64 txnid_least_bits = 22 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 22; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 23 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 23; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional uint64 highest_sequence_id = 24 [default = 0]; + public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 24; + private long highestSequenceId_; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + + private void initFields() { + producerName_ = ""; + sequenceId_ = 0L; + publishTime_ = 0L; + properties_ = java.util.Collections.emptyList(); + replicatedFrom_ = ""; + partitionKey_ = ""; + replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; + uncompressedSize_ = 0; + numMessagesInBatch_ = 1; + eventTime_ = 0L; + encryptionKeys_ = java.util.Collections.emptyList(); + encryptionAlgo_ = ""; + encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + partitionKeyB64Encoded_ = false; + orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + deliverAtTime_ = 0L; + markerType_ = 0; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + highestSequenceId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProducerName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSequenceId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasPublishTime()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getEncryptionKeysCount(); i++) { + if (!getEncryptionKeys(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, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, publishTime_); + } + for (int i = 0; i < properties_.size(); i++) { + output.writeMessage(4, properties_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(5, getReplicatedFromBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(6, getPartitionKeyBytes()); + } + for (int i = 0; i < replicateTo_.size(); i++) { + output.writeBytes(7, replicateTo_.getByteString(i)); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeEnum(8, compression_.getNumber()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeUInt32(9, uncompressedSize_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeInt32(11, numMessagesInBatch_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt64(12, eventTime_); + } + for (int i = 0; i < encryptionKeys_.size(); i++) { + output.writeMessage(13, encryptionKeys_.get(i)); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBytes(14, getEncryptionAlgoBytes()); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeBytes(15, encryptionParam_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeBytes(16, schemaVersion_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeBool(17, partitionKeyB64Encoded_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeBytes(18, orderingKey_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + output.writeInt64(19, deliverAtTime_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + output.writeInt32(20, markerType_); + } + if (((bitField0_ & 0x00010000) == 0x00010000)) { + output.writeUInt64(22, txnidLeastBits_); + } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + output.writeUInt64(23, txnidMostBits_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + output.writeUInt64(24, highestSequenceId_); + } + } + + 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, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, publishTime_); + } + for (int i = 0; i < properties_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, properties_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getReplicatedFromBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(6, getPartitionKeyBytes()); + } + { + int dataSize = 0; + for (int i = 0; i < replicateTo_.size(); i++) { + dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSizeNoTag(replicateTo_.getByteString(i)); + } + size += dataSize; + size += 1 * getReplicateToList().size(); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(8, compression_.getNumber()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt32Size(9, uncompressedSize_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(11, numMessagesInBatch_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(12, eventTime_); + } + for (int i = 0; i < encryptionKeys_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(13, encryptionKeys_.get(i)); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(14, getEncryptionAlgoBytes()); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(15, encryptionParam_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(16, schemaVersion_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(17, partitionKeyB64Encoded_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + 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 + .computeInt64Size(19, deliverAtTime_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(20, markerType_); + } + if (((bitField0_ & 0x00010000) == 0x00010000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(22, txnidLeastBits_); + } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(23, txnidMostBits_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(24, highestSequenceId_); + } + 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata 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.PulsarApi.MessageMetadata, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.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(); + producerName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + publishTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + replicatedFrom_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + partitionKey_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000040); + compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; + bitField0_ = (bitField0_ & ~0x00000080); + uncompressedSize_ = 0; + bitField0_ = (bitField0_ & ~0x00000100); + numMessagesInBatch_ = 1; + bitField0_ = (bitField0_ & ~0x00000200); + eventTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00000400); + encryptionKeys_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + encryptionAlgo_ = ""; + bitField0_ = (bitField0_ & ~0x00001000); + encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00002000); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00004000); + partitionKeyB64Encoded_ = false; + bitField0_ = (bitField0_ & ~0x00008000); + orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00010000); + deliverAtTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00020000); + markerType_ = 0; + bitField0_ = (bitField0_ & ~0x00040000); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00080000); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00100000); + highestSequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00200000); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata build() { + org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.producerName_ = producerName_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.sequenceId_ = sequenceId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.publishTime_ = publishTime_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + properties_ = java.util.Collections.unmodifiableList(properties_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.properties_ = properties_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000008; + } + result.replicatedFrom_ = replicatedFrom_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000010; + } + result.partitionKey_ = partitionKey_; + if (((bitField0_ & 0x00000040) == 0x00000040)) { + replicateTo_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( + replicateTo_); + bitField0_ = (bitField0_ & ~0x00000040); + } + result.replicateTo_ = replicateTo_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000020; + } + result.compression_ = compression_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000040; + } + result.uncompressedSize_ = uncompressedSize_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000080; + } + result.numMessagesInBatch_ = numMessagesInBatch_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000100; + } + result.eventTime_ = eventTime_; + if (((bitField0_ & 0x00000800) == 0x00000800)) { + encryptionKeys_ = java.util.Collections.unmodifiableList(encryptionKeys_); + bitField0_ = (bitField0_ & ~0x00000800); + } + result.encryptionKeys_ = encryptionKeys_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000200; + } + result.encryptionAlgo_ = encryptionAlgo_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00000400; + } + result.encryptionParam_ = encryptionParam_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00000800; + } + result.schemaVersion_ = schemaVersion_; + if (((from_bitField0_ & 0x00008000) == 0x00008000)) { + to_bitField0_ |= 0x00001000; + } + result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_; + if (((from_bitField0_ & 0x00010000) == 0x00010000)) { + to_bitField0_ |= 0x00002000; + } + result.orderingKey_ = orderingKey_; + if (((from_bitField0_ & 0x00020000) == 0x00020000)) { + to_bitField0_ |= 0x00004000; + } + result.deliverAtTime_ = deliverAtTime_; + if (((from_bitField0_ & 0x00040000) == 0x00040000)) { + to_bitField0_ |= 0x00008000; + } + result.markerType_ = markerType_; + if (((from_bitField0_ & 0x00080000) == 0x00080000)) { + to_bitField0_ |= 0x00010000; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00100000) == 0x00100000)) { + to_bitField0_ |= 0x00020000; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00200000) == 0x00200000)) { + to_bitField0_ |= 0x00040000; + } + result.highestSequenceId_ = highestSequenceId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.MessageMetadata.getDefaultInstance()) return this; + if (other.hasProducerName()) { + setProducerName(other.getProducerName()); + } + if (other.hasSequenceId()) { + setSequenceId(other.getSequenceId()); + } + if (other.hasPublishTime()) { + setPublishTime(other.getPublishTime()); + } + if (!other.properties_.isEmpty()) { + if (properties_.isEmpty()) { + properties_ = other.properties_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensurePropertiesIsMutable(); + properties_.addAll(other.properties_); + } + + } + if (other.hasReplicatedFrom()) { + setReplicatedFrom(other.getReplicatedFrom()); + } + if (other.hasPartitionKey()) { + setPartitionKey(other.getPartitionKey()); + } + if (!other.replicateTo_.isEmpty()) { + if (replicateTo_.isEmpty()) { + replicateTo_ = other.replicateTo_; + bitField0_ = (bitField0_ & ~0x00000040); + } else { + ensureReplicateToIsMutable(); + replicateTo_.addAll(other.replicateTo_); + } + + } + if (other.hasCompression()) { + setCompression(other.getCompression()); + } + if (other.hasUncompressedSize()) { + setUncompressedSize(other.getUncompressedSize()); + } + if (other.hasNumMessagesInBatch()) { + setNumMessagesInBatch(other.getNumMessagesInBatch()); + } + if (other.hasEventTime()) { + setEventTime(other.getEventTime()); + } + if (!other.encryptionKeys_.isEmpty()) { + if (encryptionKeys_.isEmpty()) { + encryptionKeys_ = other.encryptionKeys_; + bitField0_ = (bitField0_ & ~0x00000800); + } else { + ensureEncryptionKeysIsMutable(); + encryptionKeys_.addAll(other.encryptionKeys_); + } + + } + if (other.hasEncryptionAlgo()) { + setEncryptionAlgo(other.getEncryptionAlgo()); + } + if (other.hasEncryptionParam()) { + setEncryptionParam(other.getEncryptionParam()); + } + if (other.hasSchemaVersion()) { + setSchemaVersion(other.getSchemaVersion()); + } + if (other.hasPartitionKeyB64Encoded()) { + setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded()); + } + if (other.hasOrderingKey()) { + setOrderingKey(other.getOrderingKey()); + } + if (other.hasDeliverAtTime()) { + setDeliverAtTime(other.getDeliverAtTime()); + } + if (other.hasMarkerType()) { + setMarkerType(other.getMarkerType()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasHighestSequenceId()) { + setHighestSequenceId(other.getHighestSequenceId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasProducerName()) { + + return false; + } + if (!hasSequenceId()) { + + return false; + } + if (!hasPublishTime()) { + + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(i).isInitialized()) { + + return false; + } + } + for (int i = 0; i < getEncryptionKeysCount(); i++) { + if (!getEncryptionKeys(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; + producerName_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + sequenceId_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + publishTime_ = input.readUInt64(); + break; + } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addProperties(subBuilder.buildPartial()); + break; + } + case 42: { + bitField0_ |= 0x00000010; + replicatedFrom_ = input.readBytes(); + break; + } + case 50: { + bitField0_ |= 0x00000020; + partitionKey_ = input.readBytes(); + break; + } + case 58: { + ensureReplicateToIsMutable(); + replicateTo_.add(input.readBytes()); + break; + } + case 64: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CompressionType value = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000080; + compression_ = value; + } + break; + } + case 72: { + bitField0_ |= 0x00000100; + uncompressedSize_ = input.readUInt32(); + break; + } + case 88: { + bitField0_ |= 0x00000200; + numMessagesInBatch_ = input.readInt32(); + break; + } + case 96: { + bitField0_ |= 0x00000400; + eventTime_ = input.readUInt64(); + break; + } + case 106: { + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addEncryptionKeys(subBuilder.buildPartial()); + break; + } + case 114: { + bitField0_ |= 0x00001000; + encryptionAlgo_ = input.readBytes(); + break; + } + case 122: { + bitField0_ |= 0x00002000; + encryptionParam_ = input.readBytes(); + break; + } + case 130: { + bitField0_ |= 0x00004000; + schemaVersion_ = input.readBytes(); + break; + } + case 136: { + bitField0_ |= 0x00008000; + partitionKeyB64Encoded_ = input.readBool(); + break; + } + case 146: { + bitField0_ |= 0x00010000; + orderingKey_ = input.readBytes(); + break; + } + case 152: { + bitField0_ |= 0x00020000; + deliverAtTime_ = input.readInt64(); + break; + } + case 160: { + bitField0_ |= 0x00040000; + markerType_ = input.readInt32(); + break; + } + case 176: { + bitField0_ |= 0x00080000; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 184: { + bitField0_ |= 0x00100000; + txnidMostBits_ = input.readUInt64(); + break; + } + case 192: { + bitField0_ |= 0x00200000; + highestSequenceId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required string producer_name = 1; + private java.lang.Object producerName_ = ""; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + producerName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProducerName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + producerName_ = value; + + return this; + } + public Builder clearProducerName() { + bitField0_ = (bitField0_ & ~0x00000001); + producerName_ = getDefaultInstance().getProducerName(); + + return this; + } + void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + producerName_ = value; + + } + + // required uint64 sequence_id = 2; + private long sequenceId_ ; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + public Builder setSequenceId(long value) { + bitField0_ |= 0x00000002; + sequenceId_ = value; + + return this; + } + public Builder clearSequenceId() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceId_ = 0L; + + return this; + } + + // required uint64 publish_time = 3; + private long publishTime_ ; + public boolean hasPublishTime() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getPublishTime() { + return publishTime_; + } + public Builder setPublishTime(long value) { + bitField0_ |= 0x00000004; + publishTime_ = value; + + return this; + } + public Builder clearPublishTime() { + bitField0_ = (bitField0_ & ~0x00000004); + publishTime_ = 0L; + + return this; + } + + // repeated .pulsar.proto.KeyValue properties = 4; + private java.util.List properties_ = + java.util.Collections.emptyList(); + private void ensurePropertiesIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + properties_ = new java.util.ArrayList(properties_); + bitField0_ |= 0x00000008; + } + } + + public java.util.List getPropertiesList() { + return java.util.Collections.unmodifiableList(properties_); + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.set(index, value); + + return this; + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.set(index, builderForValue.build()); + + return this; + } + public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(value); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(index, value); + + return this; + } + public Builder addProperties( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(builderForValue.build()); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllProperties( + java.lang.Iterable values) { + ensurePropertiesIsMutable(); + super.addAll(values, properties_); + + return this; + } + public Builder clearProperties() { + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + public Builder removeProperties(int index) { + ensurePropertiesIsMutable(); + properties_.remove(index); + + return this; + } + + // optional string replicated_from = 5; + private java.lang.Object replicatedFrom_ = ""; + public boolean hasReplicatedFrom() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getReplicatedFrom() { + java.lang.Object ref = replicatedFrom_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + replicatedFrom_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setReplicatedFrom(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + replicatedFrom_ = value; + + return this; + } + public Builder clearReplicatedFrom() { + bitField0_ = (bitField0_ & ~0x00000010); + replicatedFrom_ = getDefaultInstance().getReplicatedFrom(); + + return this; + } + void setReplicatedFrom(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + replicatedFrom_ = value; + + } + + // optional string partition_key = 6; + private java.lang.Object partitionKey_ = ""; + public boolean hasPartitionKey() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getPartitionKey() { + java.lang.Object ref = partitionKey_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + partitionKey_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setPartitionKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + partitionKey_ = value; + + return this; + } + public Builder clearPartitionKey() { + bitField0_ = (bitField0_ & ~0x00000020); + partitionKey_ = getDefaultInstance().getPartitionKey(); + + return this; + } + void setPartitionKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000020; + partitionKey_ = value; + + } + + // repeated string replicate_to = 7; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + private void ensureReplicateToIsMutable() { + if (!((bitField0_ & 0x00000040) == 0x00000040)) { + replicateTo_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(replicateTo_); + bitField0_ |= 0x00000040; + } + } + public java.util.List + getReplicateToList() { + return java.util.Collections.unmodifiableList(replicateTo_); + } + public int getReplicateToCount() { + return replicateTo_.size(); + } + public String getReplicateTo(int index) { + return replicateTo_.get(index); + } + public Builder setReplicateTo( + int index, String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureReplicateToIsMutable(); + replicateTo_.set(index, value); + + return this; + } + public Builder addReplicateTo(String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureReplicateToIsMutable(); + replicateTo_.add(value); + + return this; + } + public Builder addAllReplicateTo( + java.lang.Iterable values) { + ensureReplicateToIsMutable(); + super.addAll(values, replicateTo_); + + return this; + } + public Builder clearReplicateTo() { + replicateTo_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000040); + + return this; + } + void addReplicateTo(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + ensureReplicateToIsMutable(); + replicateTo_.add(value); + + } + + // optional .pulsar.proto.CompressionType compression = 8 [default = NONE]; + private org.apache.pulsar.common.api.proto.PulsarApi.CompressionType compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; + public boolean hasCompression() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CompressionType getCompression() { + return compression_; + } + public Builder setCompression(org.apache.pulsar.common.api.proto.PulsarApi.CompressionType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000080; + compression_ = value; + + return this; + } + public Builder clearCompression() { + bitField0_ = (bitField0_ & ~0x00000080); + compression_ = org.apache.pulsar.common.api.proto.PulsarApi.CompressionType.NONE; + + return this; + } + + // optional uint32 uncompressed_size = 9 [default = 0]; + private int uncompressedSize_ ; + public boolean hasUncompressedSize() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public int getUncompressedSize() { + return uncompressedSize_; + } + public Builder setUncompressedSize(int value) { + bitField0_ |= 0x00000100; + uncompressedSize_ = value; + + return this; + } + public Builder clearUncompressedSize() { + bitField0_ = (bitField0_ & ~0x00000100); + uncompressedSize_ = 0; + + return this; + } + + // optional int32 num_messages_in_batch = 11 [default = 1]; + private int numMessagesInBatch_ = 1; + public boolean hasNumMessagesInBatch() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public int getNumMessagesInBatch() { + return numMessagesInBatch_; + } + public Builder setNumMessagesInBatch(int value) { + bitField0_ |= 0x00000200; + numMessagesInBatch_ = value; + + return this; + } + public Builder clearNumMessagesInBatch() { + bitField0_ = (bitField0_ & ~0x00000200); + numMessagesInBatch_ = 1; + + return this; + } + + // optional uint64 event_time = 12 [default = 0]; + private long eventTime_ ; + public boolean hasEventTime() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public long getEventTime() { + return eventTime_; + } + public Builder setEventTime(long value) { + bitField0_ |= 0x00000400; + eventTime_ = value; + + return this; + } + public Builder clearEventTime() { + bitField0_ = (bitField0_ & ~0x00000400); + eventTime_ = 0L; + + return this; + } + + // repeated .pulsar.proto.EncryptionKeys encryption_keys = 13; + private java.util.List encryptionKeys_ = + java.util.Collections.emptyList(); + private void ensureEncryptionKeysIsMutable() { + if (!((bitField0_ & 0x00000800) == 0x00000800)) { + encryptionKeys_ = new java.util.ArrayList(encryptionKeys_); + bitField0_ |= 0x00000800; + } + } + + public java.util.List getEncryptionKeysList() { + return java.util.Collections.unmodifiableList(encryptionKeys_); + } + public int getEncryptionKeysCount() { + return encryptionKeys_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys getEncryptionKeys(int index) { + return encryptionKeys_.get(index); + } + public Builder setEncryptionKeys( + int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { + if (value == null) { + throw new NullPointerException(); + } + ensureEncryptionKeysIsMutable(); + encryptionKeys_.set(index, value); + + return this; + } + public Builder setEncryptionKeys( + int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { + ensureEncryptionKeysIsMutable(); + encryptionKeys_.set(index, builderForValue.build()); + + return this; + } + public Builder addEncryptionKeys(org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { + if (value == null) { + throw new NullPointerException(); + } + ensureEncryptionKeysIsMutable(); + encryptionKeys_.add(value); + + return this; + } + public Builder addEncryptionKeys( + int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys value) { + if (value == null) { + throw new NullPointerException(); + } + ensureEncryptionKeysIsMutable(); + encryptionKeys_.add(index, value); + + return this; + } + public Builder addEncryptionKeys( + org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { + ensureEncryptionKeysIsMutable(); + encryptionKeys_.add(builderForValue.build()); + + return this; + } + public Builder addEncryptionKeys( + int index, org.apache.pulsar.common.api.proto.PulsarApi.EncryptionKeys.Builder builderForValue) { + ensureEncryptionKeysIsMutable(); + encryptionKeys_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllEncryptionKeys( + java.lang.Iterable values) { + ensureEncryptionKeysIsMutable(); + super.addAll(values, encryptionKeys_); + + return this; + } + public Builder clearEncryptionKeys() { + encryptionKeys_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + + return this; + } + public Builder removeEncryptionKeys(int index) { + ensureEncryptionKeysIsMutable(); + encryptionKeys_.remove(index); + + return this; + } + + // optional string encryption_algo = 14; + private java.lang.Object encryptionAlgo_ = ""; + public boolean hasEncryptionAlgo() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public String getEncryptionAlgo() { + java.lang.Object ref = encryptionAlgo_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + encryptionAlgo_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setEncryptionAlgo(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00001000; + encryptionAlgo_ = value; + + return this; + } + public Builder clearEncryptionAlgo() { + bitField0_ = (bitField0_ & ~0x00001000); + encryptionAlgo_ = getDefaultInstance().getEncryptionAlgo(); + + return this; + } + void setEncryptionAlgo(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00001000; + encryptionAlgo_ = value; + + } + + // optional bytes encryption_param = 15; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString encryptionParam_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasEncryptionParam() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getEncryptionParam() { + return encryptionParam_; + } + public Builder setEncryptionParam(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00002000; + encryptionParam_ = value; + + return this; + } + public Builder clearEncryptionParam() { + bitField0_ = (bitField0_ & ~0x00002000); + encryptionParam_ = getDefaultInstance().getEncryptionParam(); + + return this; + } + + // optional bytes schema_version = 16; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00004000; + schemaVersion_ = value; + + return this; + } + public Builder clearSchemaVersion() { + bitField0_ = (bitField0_ & ~0x00004000); + schemaVersion_ = getDefaultInstance().getSchemaVersion(); + + return this; + } + + // optional bool partition_key_b64_encoded = 17 [default = false]; + private boolean partitionKeyB64Encoded_ ; + public boolean hasPartitionKeyB64Encoded() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public boolean getPartitionKeyB64Encoded() { + return partitionKeyB64Encoded_; + } + public Builder setPartitionKeyB64Encoded(boolean value) { + bitField0_ |= 0x00008000; + partitionKeyB64Encoded_ = value; + + return this; + } + public Builder clearPartitionKeyB64Encoded() { + bitField0_ = (bitField0_ & ~0x00008000); + partitionKeyB64Encoded_ = false; + + return this; + } + + // optional bytes ordering_key = 18; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasOrderingKey() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { + return orderingKey_; + } + public Builder setOrderingKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00010000; + orderingKey_ = value; + + return this; + } + public Builder clearOrderingKey() { + bitField0_ = (bitField0_ & ~0x00010000); + orderingKey_ = getDefaultInstance().getOrderingKey(); + + return this; + } + + // optional int64 deliver_at_time = 19; + private long deliverAtTime_ ; + public boolean hasDeliverAtTime() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + public long getDeliverAtTime() { + return deliverAtTime_; + } + public Builder setDeliverAtTime(long value) { + bitField0_ |= 0x00020000; + deliverAtTime_ = value; + + return this; + } + public Builder clearDeliverAtTime() { + bitField0_ = (bitField0_ & ~0x00020000); + deliverAtTime_ = 0L; + + return this; + } + + // optional int32 marker_type = 20; + private int markerType_ ; + public boolean hasMarkerType() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + public int getMarkerType() { + return markerType_; + } + public Builder setMarkerType(int value) { + bitField0_ |= 0x00040000; + markerType_ = value; + + return this; + } + public Builder clearMarkerType() { + bitField0_ = (bitField0_ & ~0x00040000); + markerType_ = 0; + + return this; + } + + // optional uint64 txnid_least_bits = 22 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00080000; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00080000); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 23 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00100000; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00100000); + txnidMostBits_ = 0L; + + return this; + } + + // optional uint64 highest_sequence_id = 24 [default = 0]; + private long highestSequenceId_ ; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + public Builder setHighestSequenceId(long value) { + bitField0_ |= 0x00200000; + highestSequenceId_ = value; + + return this; + } + public Builder clearHighestSequenceId() { + bitField0_ = (bitField0_ & ~0x00200000); + highestSequenceId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.MessageMetadata) + } + + static { + defaultInstance = new MessageMetadata(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.MessageMetadata) + } + + public interface SingleMessageMetadataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // repeated .pulsar.proto.KeyValue properties = 1; + java.util.List + getPropertiesList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index); + int getPropertiesCount(); + + // optional string partition_key = 2; + boolean hasPartitionKey(); + String getPartitionKey(); + + // required int32 payload_size = 3; + boolean hasPayloadSize(); + int getPayloadSize(); + + // optional bool compacted_out = 4 [default = false]; + boolean hasCompactedOut(); + boolean getCompactedOut(); + + // optional uint64 event_time = 5 [default = 0]; + boolean hasEventTime(); + long getEventTime(); + + // optional bool partition_key_b64_encoded = 6 [default = false]; + boolean hasPartitionKeyB64Encoded(); + boolean getPartitionKeyB64Encoded(); + + // optional bytes ordering_key = 7; + boolean hasOrderingKey(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey(); + + // optional uint64 sequence_id = 8; + boolean hasSequenceId(); + long getSequenceId(); + } + public static final class SingleMessageMetadata extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements SingleMessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use SingleMessageMetadata.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private SingleMessageMetadata(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected SingleMessageMetadata newObject(Handle handle) { + return new SingleMessageMetadata(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private SingleMessageMetadata(boolean noInit) {} + + private static final SingleMessageMetadata defaultInstance; + public static SingleMessageMetadata getDefaultInstance() { + return defaultInstance; + } + + public SingleMessageMetadata getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // repeated .pulsar.proto.KeyValue properties = 1; + public static final int PROPERTIES_FIELD_NUMBER = 1; + private java.util.List properties_; + public java.util.List getPropertiesList() { + return properties_; + } + public java.util.List + getPropertiesOrBuilderList() { + return properties_; + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getPropertiesOrBuilder( + int index) { + return properties_.get(index); + } + + // optional string partition_key = 2; + public static final int PARTITION_KEY_FIELD_NUMBER = 2; + private java.lang.Object partitionKey_; + public boolean hasPartitionKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getPartitionKey() { + java.lang.Object ref = partitionKey_; + 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)) { + partitionKey_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getPartitionKeyBytes() { + java.lang.Object ref = partitionKey_; + 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); + partitionKey_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required int32 payload_size = 3; + public static final int PAYLOAD_SIZE_FIELD_NUMBER = 3; + private int payloadSize_; + public boolean hasPayloadSize() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getPayloadSize() { + return payloadSize_; + } + + // optional bool compacted_out = 4 [default = false]; + public static final int COMPACTED_OUT_FIELD_NUMBER = 4; + private boolean compactedOut_; + public boolean hasCompactedOut() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public boolean getCompactedOut() { + return compactedOut_; + } + + // optional uint64 event_time = 5 [default = 0]; + public static final int EVENT_TIME_FIELD_NUMBER = 5; + private long eventTime_; + public boolean hasEventTime() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getEventTime() { + return eventTime_; + } + + // optional bool partition_key_b64_encoded = 6 [default = false]; + public static final int PARTITION_KEY_B64_ENCODED_FIELD_NUMBER = 6; + private boolean partitionKeyB64Encoded_; + public boolean hasPartitionKeyB64Encoded() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getPartitionKeyB64Encoded() { + return partitionKeyB64Encoded_; + } + + // optional bytes ordering_key = 7; + public static final int ORDERING_KEY_FIELD_NUMBER = 7; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_; + public boolean hasOrderingKey() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { + return orderingKey_; + } + + // optional uint64 sequence_id = 8; + public static final int SEQUENCE_ID_FIELD_NUMBER = 8; + private long sequenceId_; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public long getSequenceId() { + return sequenceId_; + } + + private void initFields() { + properties_ = java.util.Collections.emptyList(); + partitionKey_ = ""; + payloadSize_ = 0; + compactedOut_ = false; + eventTime_ = 0L; + partitionKeyB64Encoded_ = false; + orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + sequenceId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasPayloadSize()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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(); + for (int i = 0; i < properties_.size(); i++) { + output.writeMessage(1, properties_.get(i)); + } + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(2, getPartitionKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(3, payloadSize_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBool(4, compactedOut_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(5, eventTime_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(6, partitionKeyB64Encoded_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(7, orderingKey_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeUInt64(8, sequenceId_); + } + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < properties_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(1, properties_.get(i)); + } + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getPartitionKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, payloadSize_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(4, compactedOut_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(5, eventTime_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(6, partitionKeyB64Encoded_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(7, orderingKey_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(8, sequenceId_); + } + 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata 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.PulsarApi.SingleMessageMetadata, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.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(); + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + partitionKey_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + payloadSize_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + compactedOut_ = false; + bitField0_ = (bitField0_ & ~0x00000008); + eventTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00000010); + partitionKeyB64Encoded_ = false; + bitField0_ = (bitField0_ & ~0x00000020); + orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000040); + sequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000080); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata build() { + org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + properties_ = java.util.Collections.unmodifiableList(properties_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.properties_ = properties_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000001; + } + result.partitionKey_ = partitionKey_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000002; + } + result.payloadSize_ = payloadSize_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000004; + } + result.compactedOut_ = compactedOut_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000008; + } + result.eventTime_ = eventTime_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000010; + } + result.partitionKeyB64Encoded_ = partitionKeyB64Encoded_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000020; + } + result.orderingKey_ = orderingKey_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000040; + } + result.sequenceId_ = sequenceId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.SingleMessageMetadata.getDefaultInstance()) return this; + if (!other.properties_.isEmpty()) { + if (properties_.isEmpty()) { + properties_ = other.properties_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensurePropertiesIsMutable(); + properties_.addAll(other.properties_); + } + + } + if (other.hasPartitionKey()) { + setPartitionKey(other.getPartitionKey()); + } + if (other.hasPayloadSize()) { + setPayloadSize(other.getPayloadSize()); + } + if (other.hasCompactedOut()) { + setCompactedOut(other.getCompactedOut()); + } + if (other.hasEventTime()) { + setEventTime(other.getEventTime()); + } + if (other.hasPartitionKeyB64Encoded()) { + setPartitionKeyB64Encoded(other.getPartitionKeyB64Encoded()); + } + if (other.hasOrderingKey()) { + setOrderingKey(other.getOrderingKey()); + } + if (other.hasSequenceId()) { + setSequenceId(other.getSequenceId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasPayloadSize()) { + + return false; + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addProperties(subBuilder.buildPartial()); + break; + } + case 18: { + bitField0_ |= 0x00000002; + partitionKey_ = input.readBytes(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + payloadSize_ = input.readInt32(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + compactedOut_ = input.readBool(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + eventTime_ = input.readUInt64(); + break; + } + case 48: { + bitField0_ |= 0x00000020; + partitionKeyB64Encoded_ = input.readBool(); + break; + } + case 58: { + bitField0_ |= 0x00000040; + orderingKey_ = input.readBytes(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + sequenceId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // repeated .pulsar.proto.KeyValue properties = 1; + private java.util.List properties_ = + java.util.Collections.emptyList(); + private void ensurePropertiesIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + properties_ = new java.util.ArrayList(properties_); + bitField0_ |= 0x00000001; + } + } + + public java.util.List getPropertiesList() { + return java.util.Collections.unmodifiableList(properties_); + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getProperties(int index) { + return properties_.get(index); + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.set(index, value); + + return this; + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.set(index, builderForValue.build()); + + return this; + } + public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(value); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(index, value); + + return this; + } + public Builder addProperties( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(builderForValue.build()); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllProperties( + java.lang.Iterable values) { + ensurePropertiesIsMutable(); + super.addAll(values, properties_); + + return this; + } + public Builder clearProperties() { + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + + return this; + } + public Builder removeProperties(int index) { + ensurePropertiesIsMutable(); + properties_.remove(index); + + return this; + } + + // optional string partition_key = 2; + private java.lang.Object partitionKey_ = ""; + public boolean hasPartitionKey() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getPartitionKey() { + java.lang.Object ref = partitionKey_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + partitionKey_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setPartitionKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + partitionKey_ = value; + + return this; + } + public Builder clearPartitionKey() { + bitField0_ = (bitField0_ & ~0x00000002); + partitionKey_ = getDefaultInstance().getPartitionKey(); + + return this; + } + void setPartitionKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + partitionKey_ = value; + + } + + // required int32 payload_size = 3; + private int payloadSize_ ; + public boolean hasPayloadSize() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getPayloadSize() { + return payloadSize_; + } + public Builder setPayloadSize(int value) { + bitField0_ |= 0x00000004; + payloadSize_ = value; + + return this; + } + public Builder clearPayloadSize() { + bitField0_ = (bitField0_ & ~0x00000004); + payloadSize_ = 0; + + return this; + } + + // optional bool compacted_out = 4 [default = false]; + private boolean compactedOut_ ; + public boolean hasCompactedOut() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public boolean getCompactedOut() { + return compactedOut_; + } + public Builder setCompactedOut(boolean value) { + bitField0_ |= 0x00000008; + compactedOut_ = value; + + return this; + } + public Builder clearCompactedOut() { + bitField0_ = (bitField0_ & ~0x00000008); + compactedOut_ = false; + + return this; + } + + // optional uint64 event_time = 5 [default = 0]; + private long eventTime_ ; + public boolean hasEventTime() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getEventTime() { + return eventTime_; + } + public Builder setEventTime(long value) { + bitField0_ |= 0x00000010; + eventTime_ = value; + + return this; + } + public Builder clearEventTime() { + bitField0_ = (bitField0_ & ~0x00000010); + eventTime_ = 0L; + + return this; + } + + // optional bool partition_key_b64_encoded = 6 [default = false]; + private boolean partitionKeyB64Encoded_ ; + public boolean hasPartitionKeyB64Encoded() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public boolean getPartitionKeyB64Encoded() { + return partitionKeyB64Encoded_; + } + public Builder setPartitionKeyB64Encoded(boolean value) { + bitField0_ |= 0x00000020; + partitionKeyB64Encoded_ = value; + + return this; + } + public Builder clearPartitionKeyB64Encoded() { + bitField0_ = (bitField0_ & ~0x00000020); + partitionKeyB64Encoded_ = false; + + return this; + } + + // optional bytes ordering_key = 7; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString orderingKey_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasOrderingKey() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOrderingKey() { + return orderingKey_; + } + public Builder setOrderingKey(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000040; + orderingKey_ = value; + + return this; + } + public Builder clearOrderingKey() { + bitField0_ = (bitField0_ & ~0x00000040); + orderingKey_ = getDefaultInstance().getOrderingKey(); + + return this; + } + + // optional uint64 sequence_id = 8; + private long sequenceId_ ; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public long getSequenceId() { + return sequenceId_; + } + public Builder setSequenceId(long value) { + bitField0_ |= 0x00000080; + sequenceId_ = value; + + return this; + } + public Builder clearSequenceId() { + bitField0_ = (bitField0_ & ~0x00000080); + sequenceId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.SingleMessageMetadata) + } + + static { + defaultInstance = new SingleMessageMetadata(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.SingleMessageMetadata) + } + + public interface CommandConnectOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string client_version = 1; + boolean hasClientVersion(); + String getClientVersion(); + + // optional .pulsar.proto.AuthMethod auth_method = 2; + boolean hasAuthMethod(); + org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod(); + + // optional string auth_method_name = 5; + boolean hasAuthMethodName(); + String getAuthMethodName(); + + // optional bytes auth_data = 3; + boolean hasAuthData(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData(); + + // optional int32 protocol_version = 4 [default = 0]; + boolean hasProtocolVersion(); + int getProtocolVersion(); + + // optional string proxy_to_broker_url = 6; + boolean hasProxyToBrokerUrl(); + String getProxyToBrokerUrl(); + + // optional string original_principal = 7; + boolean hasOriginalPrincipal(); + String getOriginalPrincipal(); + + // optional string original_auth_data = 8; + boolean hasOriginalAuthData(); + String getOriginalAuthData(); + + // optional string original_auth_method = 9; + boolean hasOriginalAuthMethod(); + String getOriginalAuthMethod(); + } + public static final class CommandConnect extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandConnectOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandConnect.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandConnect(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandConnect newObject(Handle handle) { + return new CommandConnect(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandConnect(boolean noInit) {} + + private static final CommandConnect defaultInstance; + public static CommandConnect getDefaultInstance() { + return defaultInstance; + } + + public CommandConnect getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string client_version = 1; + public static final int CLIENT_VERSION_FIELD_NUMBER = 1; + private java.lang.Object clientVersion_; + public boolean hasClientVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClientVersion() { + java.lang.Object ref = clientVersion_; + 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)) { + clientVersion_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getClientVersionBytes() { + java.lang.Object ref = clientVersion_; + 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); + clientVersion_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.AuthMethod auth_method = 2; + public static final int AUTH_METHOD_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod authMethod_; + public boolean hasAuthMethod() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod() { + return authMethod_; + } + + // optional string auth_method_name = 5; + public static final int AUTH_METHOD_NAME_FIELD_NUMBER = 5; + private java.lang.Object authMethodName_; + public boolean hasAuthMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getAuthMethodName() { + java.lang.Object ref = authMethodName_; + 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)) { + authMethodName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthMethodNameBytes() { + java.lang.Object ref = authMethodName_; + 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); + authMethodName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bytes auth_data = 3; + public static final int AUTH_DATA_FIELD_NUMBER = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_; + public boolean hasAuthData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { + return authData_; + } + + // optional int32 protocol_version = 4 [default = 0]; + public static final int PROTOCOL_VERSION_FIELD_NUMBER = 4; + private int protocolVersion_; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public int getProtocolVersion() { + return protocolVersion_; + } + + // optional string proxy_to_broker_url = 6; + public static final int PROXY_TO_BROKER_URL_FIELD_NUMBER = 6; + private java.lang.Object proxyToBrokerUrl_; + public boolean hasProxyToBrokerUrl() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getProxyToBrokerUrl() { + java.lang.Object ref = proxyToBrokerUrl_; + 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)) { + proxyToBrokerUrl_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProxyToBrokerUrlBytes() { + java.lang.Object ref = proxyToBrokerUrl_; + 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); + proxyToBrokerUrl_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_principal = 7; + public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 7; + private java.lang.Object originalPrincipal_; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + 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)) { + originalPrincipal_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { + java.lang.Object ref = originalPrincipal_; + 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); + originalPrincipal_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_data = 8; + public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 8; + private java.lang.Object originalAuthData_; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + 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)) { + originalAuthData_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { + java.lang.Object ref = originalAuthData_; + 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); + originalAuthData_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_method = 9; + public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 9; + private java.lang.Object originalAuthMethod_; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + 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)) { + originalAuthMethod_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { + java.lang.Object ref = originalAuthMethod_; + 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); + originalAuthMethod_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + clientVersion_ = ""; + authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; + authMethodName_ = ""; + authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + protocolVersion_ = 0; + proxyToBrokerUrl_ = ""; + originalPrincipal_ = ""; + originalAuthData_ = ""; + originalAuthMethod_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasClientVersion()) { + 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, getClientVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, authMethod_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(3, authData_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeInt32(4, protocolVersion_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(5, getAuthMethodNameBytes()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, getProxyToBrokerUrlBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBytes(7, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeBytes(8, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeBytes(9, getOriginalAuthMethodBytes()); + } + } + + 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, getClientVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, authMethod_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, authData_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(4, protocolVersion_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getAuthMethodNameBytes()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(6, getProxyToBrokerUrlBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(7, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(8, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(9, getOriginalAuthMethodBytes()); + } + 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect 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.PulsarApi.CommandConnect, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConnectOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.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(); + clientVersion_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; + bitField0_ = (bitField0_ & ~0x00000002); + authMethodName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + protocolVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000010); + proxyToBrokerUrl_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + originalPrincipal_ = ""; + bitField0_ = (bitField0_ & ~0x00000040); + originalAuthData_ = ""; + bitField0_ = (bitField0_ & ~0x00000080); + originalAuthMethod_ = ""; + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.clientVersion_ = clientVersion_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.authMethod_ = authMethod_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.authMethodName_ = authMethodName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.authData_ = authData_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.protocolVersion_ = protocolVersion_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.proxyToBrokerUrl_ = proxyToBrokerUrl_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.originalPrincipal_ = originalPrincipal_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.originalAuthData_ = originalAuthData_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.originalAuthMethod_ = originalAuthMethod_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance()) return this; + if (other.hasClientVersion()) { + setClientVersion(other.getClientVersion()); + } + if (other.hasAuthMethod()) { + setAuthMethod(other.getAuthMethod()); + } + if (other.hasAuthMethodName()) { + setAuthMethodName(other.getAuthMethodName()); + } + if (other.hasAuthData()) { + setAuthData(other.getAuthData()); + } + if (other.hasProtocolVersion()) { + setProtocolVersion(other.getProtocolVersion()); + } + if (other.hasProxyToBrokerUrl()) { + setProxyToBrokerUrl(other.getProxyToBrokerUrl()); + } + if (other.hasOriginalPrincipal()) { + setOriginalPrincipal(other.getOriginalPrincipal()); + } + if (other.hasOriginalAuthData()) { + setOriginalAuthData(other.getOriginalAuthData()); + } + if (other.hasOriginalAuthMethod()) { + setOriginalAuthMethod(other.getOriginalAuthMethod()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasClientVersion()) { + + 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; + clientVersion_ = input.readBytes(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod value = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + authMethod_ = value; + } + break; + } + case 26: { + bitField0_ |= 0x00000008; + authData_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000010; + protocolVersion_ = input.readInt32(); + break; + } + case 42: { + bitField0_ |= 0x00000004; + authMethodName_ = input.readBytes(); + break; + } + case 50: { + bitField0_ |= 0x00000020; + proxyToBrokerUrl_ = input.readBytes(); + break; + } + case 58: { + bitField0_ |= 0x00000040; + originalPrincipal_ = input.readBytes(); + break; + } + case 66: { + bitField0_ |= 0x00000080; + originalAuthData_ = input.readBytes(); + break; + } + case 74: { + bitField0_ |= 0x00000100; + originalAuthMethod_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string client_version = 1; + private java.lang.Object clientVersion_ = ""; + public boolean hasClientVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClientVersion() { + java.lang.Object ref = clientVersion_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + clientVersion_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setClientVersion(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + clientVersion_ = value; + + return this; + } + public Builder clearClientVersion() { + bitField0_ = (bitField0_ & ~0x00000001); + clientVersion_ = getDefaultInstance().getClientVersion(); + + return this; + } + void setClientVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + clientVersion_ = value; + + } + + // optional .pulsar.proto.AuthMethod auth_method = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; + public boolean hasAuthMethod() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod getAuthMethod() { + return authMethod_; + } + public Builder setAuthMethod(org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + authMethod_ = value; + + return this; + } + public Builder clearAuthMethod() { + bitField0_ = (bitField0_ & ~0x00000002); + authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; + + return this; + } + + // optional string auth_method_name = 5; + private java.lang.Object authMethodName_ = ""; + public boolean hasAuthMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getAuthMethodName() { + java.lang.Object ref = authMethodName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + authMethodName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setAuthMethodName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + authMethodName_ = value; + + return this; + } + public Builder clearAuthMethodName() { + bitField0_ = (bitField0_ & ~0x00000004); + authMethodName_ = getDefaultInstance().getAuthMethodName(); + + return this; + } + void setAuthMethodName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + authMethodName_ = value; + + } + + // optional bytes auth_data = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasAuthData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { + return authData_; + } + public Builder setAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + authData_ = value; + + return this; + } + public Builder clearAuthData() { + bitField0_ = (bitField0_ & ~0x00000008); + authData_ = getDefaultInstance().getAuthData(); + + return this; + } + + // optional int32 protocol_version = 4 [default = 0]; + private int protocolVersion_ ; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public int getProtocolVersion() { + return protocolVersion_; + } + public Builder setProtocolVersion(int value) { + bitField0_ |= 0x00000010; + protocolVersion_ = value; + + return this; + } + public Builder clearProtocolVersion() { + bitField0_ = (bitField0_ & ~0x00000010); + protocolVersion_ = 0; + + return this; + } + + // optional string proxy_to_broker_url = 6; + private java.lang.Object proxyToBrokerUrl_ = ""; + public boolean hasProxyToBrokerUrl() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getProxyToBrokerUrl() { + java.lang.Object ref = proxyToBrokerUrl_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + proxyToBrokerUrl_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProxyToBrokerUrl(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + proxyToBrokerUrl_ = value; + + return this; + } + public Builder clearProxyToBrokerUrl() { + bitField0_ = (bitField0_ & ~0x00000020); + proxyToBrokerUrl_ = getDefaultInstance().getProxyToBrokerUrl(); + + return this; + } + void setProxyToBrokerUrl(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000020; + proxyToBrokerUrl_ = value; + + } + + // optional string original_principal = 7; + private java.lang.Object originalPrincipal_ = ""; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalPrincipal_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalPrincipal(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000040; + originalPrincipal_ = value; + + return this; + } + public Builder clearOriginalPrincipal() { + bitField0_ = (bitField0_ & ~0x00000040); + originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); + + return this; + } + void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000040; + originalPrincipal_ = value; + + } + + // optional string original_auth_data = 8; + private java.lang.Object originalAuthData_ = ""; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthData_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthData(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000080; + originalAuthData_ = value; + + return this; + } + public Builder clearOriginalAuthData() { + bitField0_ = (bitField0_ & ~0x00000080); + originalAuthData_ = getDefaultInstance().getOriginalAuthData(); + + return this; + } + void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000080; + originalAuthData_ = value; + + } + + // optional string original_auth_method = 9; + private java.lang.Object originalAuthMethod_ = ""; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthMethod_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthMethod(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000100; + originalAuthMethod_ = value; + + return this; + } + public Builder clearOriginalAuthMethod() { + bitField0_ = (bitField0_ & ~0x00000100); + originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); + + return this; + } + void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000100; + originalAuthMethod_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConnect) + } + + static { + defaultInstance = new CommandConnect(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConnect) + } + + public interface CommandConnectedOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string server_version = 1; + boolean hasServerVersion(); + String getServerVersion(); + + // optional int32 protocol_version = 2 [default = 0]; + boolean hasProtocolVersion(); + int getProtocolVersion(); + + // optional int32 max_message_size = 3; + boolean hasMaxMessageSize(); + int getMaxMessageSize(); + } + public static final class CommandConnected extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandConnectedOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandConnected.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandConnected(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandConnected newObject(Handle handle) { + return new CommandConnected(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandConnected(boolean noInit) {} + + private static final CommandConnected defaultInstance; + public static CommandConnected getDefaultInstance() { + return defaultInstance; + } + + public CommandConnected getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string server_version = 1; + public static final int SERVER_VERSION_FIELD_NUMBER = 1; + private java.lang.Object serverVersion_; + public boolean hasServerVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getServerVersion() { + java.lang.Object ref = serverVersion_; + 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)) { + serverVersion_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getServerVersionBytes() { + java.lang.Object ref = serverVersion_; + 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); + serverVersion_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional int32 protocol_version = 2 [default = 0]; + public static final int PROTOCOL_VERSION_FIELD_NUMBER = 2; + private int protocolVersion_; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getProtocolVersion() { + return protocolVersion_; + } + + // optional int32 max_message_size = 3; + public static final int MAX_MESSAGE_SIZE_FIELD_NUMBER = 3; + private int maxMessageSize_; + public boolean hasMaxMessageSize() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getMaxMessageSize() { + return maxMessageSize_; + } + + private void initFields() { + serverVersion_ = ""; + protocolVersion_ = 0; + maxMessageSize_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasServerVersion()) { + 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, getServerVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeInt32(2, protocolVersion_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, maxMessageSize_); + } + } + + 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, getServerVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(2, protocolVersion_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, maxMessageSize_); + } + 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected 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.PulsarApi.CommandConnected, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConnectedOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.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(); + serverVersion_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + protocolVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + maxMessageSize_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.serverVersion_ = serverVersion_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.protocolVersion_ = protocolVersion_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.maxMessageSize_ = maxMessageSize_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance()) return this; + if (other.hasServerVersion()) { + setServerVersion(other.getServerVersion()); + } + if (other.hasProtocolVersion()) { + setProtocolVersion(other.getProtocolVersion()); + } + if (other.hasMaxMessageSize()) { + setMaxMessageSize(other.getMaxMessageSize()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasServerVersion()) { + + 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; + serverVersion_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + protocolVersion_ = input.readInt32(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + maxMessageSize_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required string server_version = 1; + private java.lang.Object serverVersion_ = ""; + public boolean hasServerVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getServerVersion() { + java.lang.Object ref = serverVersion_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + serverVersion_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setServerVersion(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + serverVersion_ = value; + + return this; + } + public Builder clearServerVersion() { + bitField0_ = (bitField0_ & ~0x00000001); + serverVersion_ = getDefaultInstance().getServerVersion(); + + return this; + } + void setServerVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + serverVersion_ = value; + + } + + // optional int32 protocol_version = 2 [default = 0]; + private int protocolVersion_ ; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getProtocolVersion() { + return protocolVersion_; + } + public Builder setProtocolVersion(int value) { + bitField0_ |= 0x00000002; + protocolVersion_ = value; + + return this; + } + public Builder clearProtocolVersion() { + bitField0_ = (bitField0_ & ~0x00000002); + protocolVersion_ = 0; + + return this; + } + + // optional int32 max_message_size = 3; + private int maxMessageSize_ ; + public boolean hasMaxMessageSize() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getMaxMessageSize() { + return maxMessageSize_; + } + public Builder setMaxMessageSize(int value) { + bitField0_ |= 0x00000004; + maxMessageSize_ = value; + + return this; + } + public Builder clearMaxMessageSize() { + bitField0_ = (bitField0_ & ~0x00000004); + maxMessageSize_ = 0; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConnected) + } + + static { + defaultInstance = new CommandConnected(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConnected) + } + + public interface CommandAuthResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional string client_version = 1; + boolean hasClientVersion(); + String getClientVersion(); + + // optional .pulsar.proto.AuthData response = 2; + boolean hasResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse(); + + // optional int32 protocol_version = 3 [default = 0]; + boolean hasProtocolVersion(); + int getProtocolVersion(); + } + public static final class CommandAuthResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAuthResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAuthResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAuthResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAuthResponse newObject(Handle handle) { + return new CommandAuthResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAuthResponse(boolean noInit) {} + + private static final CommandAuthResponse defaultInstance; + public static CommandAuthResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandAuthResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // optional string client_version = 1; + public static final int CLIENT_VERSION_FIELD_NUMBER = 1; + private java.lang.Object clientVersion_; + public boolean hasClientVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClientVersion() { + java.lang.Object ref = clientVersion_; + 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)) { + clientVersion_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getClientVersionBytes() { + java.lang.Object ref = clientVersion_; + 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); + clientVersion_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.AuthData response = 2; + public static final int RESPONSE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthData response_; + public boolean hasResponse() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse() { + return response_; + } + + // optional int32 protocol_version = 3 [default = 0]; + public static final int PROTOCOL_VERSION_FIELD_NUMBER = 3; + private int protocolVersion_; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getProtocolVersion() { + return protocolVersion_; + } + + private void initFields() { + clientVersion_ = ""; + response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + protocolVersion_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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, getClientVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, response_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, protocolVersion_); + } + } + + 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, getClientVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, response_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, protocolVersion_); + } + 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse 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.PulsarApi.CommandAuthResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.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(); + clientVersion_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000002); + protocolVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.clientVersion_ = clientVersion_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.response_ = response_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.protocolVersion_ = protocolVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance()) return this; + if (other.hasClientVersion()) { + setClientVersion(other.getClientVersion()); + } + if (other.hasResponse()) { + mergeResponse(other.getResponse()); + } + if (other.hasProtocolVersion()) { + setProtocolVersion(other.getProtocolVersion()); + } + return this; + } + + public final boolean isInitialized() { + 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; + clientVersion_ = input.readBytes(); + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(); + if (hasResponse()) { + subBuilder.mergeFrom(getResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + protocolVersion_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // optional string client_version = 1; + private java.lang.Object clientVersion_ = ""; + public boolean hasClientVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClientVersion() { + java.lang.Object ref = clientVersion_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + clientVersion_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setClientVersion(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + clientVersion_ = value; + + return this; + } + public Builder clearClientVersion() { + bitField0_ = (bitField0_ & ~0x00000001); + clientVersion_ = getDefaultInstance().getClientVersion(); + + return this; + } + void setClientVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + clientVersion_ = value; + + } + + // optional .pulsar.proto.AuthData response = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthData response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + public boolean hasResponse() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getResponse() { + return response_; + } + public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { + if (value == null) { + throw new NullPointerException(); + } + response_ = value; + + bitField0_ |= 0x00000002; + return this; + } + public Builder setResponse( + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder builderForValue) { + response_ = builderForValue.build(); + + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeResponse(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + response_ != org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) { + response_ = + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(response_).mergeFrom(value).buildPartial(); + } else { + response_ = value; + } + + bitField0_ |= 0x00000002; + return this; + } + public Builder clearResponse() { + response_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + // optional int32 protocol_version = 3 [default = 0]; + private int protocolVersion_ ; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getProtocolVersion() { + return protocolVersion_; + } + public Builder setProtocolVersion(int value) { + bitField0_ |= 0x00000004; + protocolVersion_ = value; + + return this; + } + public Builder clearProtocolVersion() { + bitField0_ = (bitField0_ & ~0x00000004); + protocolVersion_ = 0; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAuthResponse) + } + + static { + defaultInstance = new CommandAuthResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAuthResponse) + } + + public interface CommandAuthChallengeOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional string server_version = 1; + boolean hasServerVersion(); + String getServerVersion(); + + // optional .pulsar.proto.AuthData challenge = 2; + boolean hasChallenge(); + org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge(); + + // optional int32 protocol_version = 3 [default = 0]; + boolean hasProtocolVersion(); + int getProtocolVersion(); + } + public static final class CommandAuthChallenge extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAuthChallengeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAuthChallenge.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAuthChallenge(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAuthChallenge newObject(Handle handle) { + return new CommandAuthChallenge(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAuthChallenge(boolean noInit) {} + + private static final CommandAuthChallenge defaultInstance; + public static CommandAuthChallenge getDefaultInstance() { + return defaultInstance; + } + + public CommandAuthChallenge getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // optional string server_version = 1; + public static final int SERVER_VERSION_FIELD_NUMBER = 1; + private java.lang.Object serverVersion_; + public boolean hasServerVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getServerVersion() { + java.lang.Object ref = serverVersion_; + 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)) { + serverVersion_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getServerVersionBytes() { + java.lang.Object ref = serverVersion_; + 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); + serverVersion_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.AuthData challenge = 2; + public static final int CHALLENGE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthData challenge_; + public boolean hasChallenge() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge() { + return challenge_; + } + + // optional int32 protocol_version = 3 [default = 0]; + public static final int PROTOCOL_VERSION_FIELD_NUMBER = 3; + private int protocolVersion_; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getProtocolVersion() { + return protocolVersion_; + } + + private void initFields() { + serverVersion_ = ""; + challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + protocolVersion_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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, getServerVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, challenge_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, protocolVersion_); + } + } + + 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, getServerVersionBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, challenge_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, protocolVersion_); + } + 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge 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.PulsarApi.CommandAuthChallenge, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallengeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.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(); + serverVersion_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000002); + protocolVersion_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.serverVersion_ = serverVersion_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.challenge_ = challenge_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.protocolVersion_ = protocolVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance()) return this; + if (other.hasServerVersion()) { + setServerVersion(other.getServerVersion()); + } + if (other.hasChallenge()) { + mergeChallenge(other.getChallenge()); + } + if (other.hasProtocolVersion()) { + setProtocolVersion(other.getProtocolVersion()); + } + return this; + } + + public final boolean isInitialized() { + 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; + serverVersion_ = input.readBytes(); + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(); + if (hasChallenge()) { + subBuilder.mergeFrom(getChallenge()); + } + input.readMessage(subBuilder, extensionRegistry); + setChallenge(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + protocolVersion_ = input.readInt32(); + break; + } + } + } + } + + private int bitField0_; + + // optional string server_version = 1; + private java.lang.Object serverVersion_ = ""; + public boolean hasServerVersion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getServerVersion() { + java.lang.Object ref = serverVersion_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + serverVersion_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setServerVersion(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + serverVersion_ = value; + + return this; + } + public Builder clearServerVersion() { + bitField0_ = (bitField0_ & ~0x00000001); + serverVersion_ = getDefaultInstance().getServerVersion(); + + return this; + } + void setServerVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + serverVersion_ = value; + + } + + // optional .pulsar.proto.AuthData challenge = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.AuthData challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + public boolean hasChallenge() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getChallenge() { + return challenge_; + } + public Builder setChallenge(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { + if (value == null) { + throw new NullPointerException(); + } + challenge_ = value; + + bitField0_ |= 0x00000002; + return this; + } + public Builder setChallenge( + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.Builder builderForValue) { + challenge_ = builderForValue.build(); + + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeChallenge(org.apache.pulsar.common.api.proto.PulsarApi.AuthData value) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + challenge_ != org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) { + challenge_ = + org.apache.pulsar.common.api.proto.PulsarApi.AuthData.newBuilder(challenge_).mergeFrom(value).buildPartial(); + } else { + challenge_ = value; + } + + bitField0_ |= 0x00000002; + return this; + } + public Builder clearChallenge() { + challenge_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + // optional int32 protocol_version = 3 [default = 0]; + private int protocolVersion_ ; + public boolean hasProtocolVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getProtocolVersion() { + return protocolVersion_; + } + public Builder setProtocolVersion(int value) { + bitField0_ |= 0x00000004; + protocolVersion_ = value; + + return this; + } + public Builder clearProtocolVersion() { + bitField0_ = (bitField0_ & ~0x00000004); + protocolVersion_ = 0; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAuthChallenge) + } + + static { + defaultInstance = new CommandAuthChallenge(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAuthChallenge) + } + + public interface AuthDataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional string auth_method_name = 1; + boolean hasAuthMethodName(); + String getAuthMethodName(); + + // optional bytes auth_data = 2; + boolean hasAuthData(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData(); + } + public static final class AuthData extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements AuthDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use AuthData.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private AuthData(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected AuthData newObject(Handle handle) { + return new AuthData(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private AuthData(boolean noInit) {} + + private static final AuthData defaultInstance; + public static AuthData getDefaultInstance() { + return defaultInstance; + } + + public AuthData getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // optional string auth_method_name = 1; + public static final int AUTH_METHOD_NAME_FIELD_NUMBER = 1; + private java.lang.Object authMethodName_; + public boolean hasAuthMethodName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getAuthMethodName() { + java.lang.Object ref = authMethodName_; + 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)) { + authMethodName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthMethodNameBytes() { + java.lang.Object ref = authMethodName_; + 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); + authMethodName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bytes auth_data = 2; + public static final int AUTH_DATA_FIELD_NUMBER = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_; + public boolean hasAuthData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { + return authData_; + } + + private void initFields() { + authMethodName_ = ""; + authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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, getAuthMethodNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, authData_); + } + } + + 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, getAuthMethodNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, authData_); + } + 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData 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.PulsarApi.AuthData, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.AuthDataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.AuthData.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(); + authMethodName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData build() { + org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.AuthData buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.AuthData buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.AuthData result = org.apache.pulsar.common.api.proto.PulsarApi.AuthData.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.authMethodName_ = authMethodName_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.authData_ = authData_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.AuthData other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.AuthData.getDefaultInstance()) return this; + if (other.hasAuthMethodName()) { + setAuthMethodName(other.getAuthMethodName()); + } + if (other.hasAuthData()) { + setAuthData(other.getAuthData()); + } + return this; + } + + public final boolean isInitialized() { + 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; + authMethodName_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + authData_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // optional string auth_method_name = 1; + private java.lang.Object authMethodName_ = ""; + public boolean hasAuthMethodName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getAuthMethodName() { + java.lang.Object ref = authMethodName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + authMethodName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setAuthMethodName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + authMethodName_ = value; + + return this; + } + public Builder clearAuthMethodName() { + bitField0_ = (bitField0_ & ~0x00000001); + authMethodName_ = getDefaultInstance().getAuthMethodName(); + + return this; + } + void setAuthMethodName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + authMethodName_ = value; + + } + + // optional bytes auth_data = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString authData_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasAuthData() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAuthData() { + return authData_; + } + public Builder setAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + authData_ = value; + + return this; + } + public Builder clearAuthData() { + bitField0_ = (bitField0_ & ~0x00000002); + authData_ = getDefaultInstance().getAuthData(); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.AuthData) + } + + static { + defaultInstance = new AuthData(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.AuthData) + } + + public interface KeySharedMetaOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required .pulsar.proto.KeySharedMode keySharedMode = 1; + boolean hasKeySharedMode(); + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode(); + + // repeated .pulsar.proto.IntRange hashRanges = 3; + java.util.List + getHashRangesList(); + org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index); + int getHashRangesCount(); + } + public static final class KeySharedMeta extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements KeySharedMetaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use KeySharedMeta.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private KeySharedMeta(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected KeySharedMeta newObject(Handle handle) { + return new KeySharedMeta(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private KeySharedMeta(boolean noInit) {} + + private static final KeySharedMeta defaultInstance; + public static KeySharedMeta getDefaultInstance() { + return defaultInstance; + } + + public KeySharedMeta getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required .pulsar.proto.KeySharedMode keySharedMode = 1; + public static final int KEYSHAREDMODE_FIELD_NUMBER = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode keySharedMode_; + public boolean hasKeySharedMode() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode() { + return keySharedMode_; + } + + // repeated .pulsar.proto.IntRange hashRanges = 3; + public static final int HASHRANGES_FIELD_NUMBER = 3; + private java.util.List hashRanges_; + public java.util.List getHashRangesList() { + return hashRanges_; + } + public java.util.List + getHashRangesOrBuilderList() { + return hashRanges_; + } + public int getHashRangesCount() { + return hashRanges_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index) { + return hashRanges_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRangeOrBuilder getHashRangesOrBuilder( + int index) { + return hashRanges_.get(index); + } + + private void initFields() { + keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; + hashRanges_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasKeySharedMode()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getHashRangesCount(); i++) { + if (!getHashRanges(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.writeEnum(1, keySharedMode_.getNumber()); + } + for (int i = 0; i < hashRanges_.size(); i++) { + output.writeMessage(3, hashRanges_.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 + .computeEnumSize(1, keySharedMode_.getNumber()); + } + for (int i = 0; i < hashRanges_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, hashRanges_.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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta 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.PulsarApi.KeySharedMeta, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMetaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.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(); + keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; + bitField0_ = (bitField0_ & ~0x00000001); + hashRanges_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta build() { + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta result = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.keySharedMode_ = keySharedMode_; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + hashRanges_ = java.util.Collections.unmodifiableList(hashRanges_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.hashRanges_ = hashRanges_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance()) return this; + if (other.hasKeySharedMode()) { + setKeySharedMode(other.getKeySharedMode()); + } + if (!other.hashRanges_.isEmpty()) { + if (hashRanges_.isEmpty()) { + hashRanges_ = other.hashRanges_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureHashRangesIsMutable(); + hashRanges_.addAll(other.hashRanges_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasKeySharedMode()) { + + return false; + } + for (int i = 0; i < getHashRangesCount(); i++) { + if (!getHashRanges(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 8: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode value = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000001; + keySharedMode_ = value; + } + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.IntRange.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addHashRanges(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .pulsar.proto.KeySharedMode keySharedMode = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; + public boolean hasKeySharedMode() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode getKeySharedMode() { + return keySharedMode_; + } + public Builder setKeySharedMode(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + keySharedMode_ = value; + + return this; + } + public Builder clearKeySharedMode() { + bitField0_ = (bitField0_ & ~0x00000001); + keySharedMode_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMode.AUTO_SPLIT; + + return this; + } + + // repeated .pulsar.proto.IntRange hashRanges = 3; + private java.util.List hashRanges_ = + java.util.Collections.emptyList(); + private void ensureHashRangesIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + hashRanges_ = new java.util.ArrayList(hashRanges_); + bitField0_ |= 0x00000002; + } + } + + public java.util.List getHashRangesList() { + return java.util.Collections.unmodifiableList(hashRanges_); + } + public int getHashRangesCount() { + return hashRanges_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.IntRange getHashRanges(int index) { + return hashRanges_.get(index); + } + public Builder setHashRanges( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureHashRangesIsMutable(); + hashRanges_.set(index, value); + + return this; + } + public Builder setHashRanges( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureHashRangesIsMutable(); + hashRanges_.set(index, builderForValue.build()); + + return this; + } + public Builder addHashRanges(org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureHashRangesIsMutable(); + hashRanges_.add(value); + + return this; + } + public Builder addHashRanges( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange value) { + if (value == null) { + throw new NullPointerException(); + } + ensureHashRangesIsMutable(); + hashRanges_.add(index, value); + + return this; + } + public Builder addHashRanges( + org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureHashRangesIsMutable(); + hashRanges_.add(builderForValue.build()); + + return this; + } + public Builder addHashRanges( + int index, org.apache.pulsar.common.api.proto.PulsarApi.IntRange.Builder builderForValue) { + ensureHashRangesIsMutable(); + hashRanges_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllHashRanges( + java.lang.Iterable values) { + ensureHashRangesIsMutable(); + super.addAll(values, hashRanges_); + + return this; + } + public Builder clearHashRanges() { + hashRanges_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + + return this; + } + public Builder removeHashRanges(int index) { + ensureHashRangesIsMutable(); + hashRanges_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.KeySharedMeta) + } + + static { + defaultInstance = new KeySharedMeta(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.KeySharedMeta) + } + + public interface CommandSubscribeOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string topic = 1; + boolean hasTopic(); + String getTopic(); + + // required string subscription = 2; + boolean hasSubscription(); + String getSubscription(); + + // required .pulsar.proto.CommandSubscribe.SubType subType = 3; + boolean hasSubType(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType(); + + // required uint64 consumer_id = 4; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint64 request_id = 5; + boolean hasRequestId(); + long getRequestId(); + + // optional string consumer_name = 6; + boolean hasConsumerName(); + String getConsumerName(); + + // optional int32 priority_level = 7; + boolean hasPriorityLevel(); + int getPriorityLevel(); + + // optional bool durable = 8 [default = true]; + boolean hasDurable(); + boolean getDurable(); + + // optional .pulsar.proto.MessageIdData start_message_id = 9; + boolean hasStartMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId(); + + // repeated .pulsar.proto.KeyValue metadata = 10; + java.util.List + getMetadataList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); + int getMetadataCount(); + + // optional bool read_compacted = 11; + boolean hasReadCompacted(); + boolean getReadCompacted(); + + // optional .pulsar.proto.Schema schema = 12; + boolean hasSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); + + // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; + boolean hasInitialPosition(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition(); + + // optional bool replicate_subscription_state = 14; + boolean hasReplicateSubscriptionState(); + boolean getReplicateSubscriptionState(); + + // optional bool force_topic_creation = 15 [default = true]; + boolean hasForceTopicCreation(); + boolean getForceTopicCreation(); + + // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; + boolean hasStartMessageRollbackDurationSec(); + long getStartMessageRollbackDurationSec(); + + // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; + boolean hasKeySharedMeta(); + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta(); + } + public static final class CommandSubscribe extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSubscribe.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSubscribe(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSubscribe newObject(Handle handle) { + return new CommandSubscribe(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSubscribe(boolean noInit) {} + + private static final CommandSubscribe defaultInstance; + public static CommandSubscribe getDefaultInstance() { + return defaultInstance; + } + + public CommandSubscribe getDefaultInstanceForType() { + return defaultInstance; + } + + public enum SubType + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + Exclusive(0, 0), + Shared(1, 1), + Failover(2, 2), + Key_Shared(3, 3), + ; + + public static final int Exclusive_VALUE = 0; + public static final int Shared_VALUE = 1; + public static final int Failover_VALUE = 2; + public static final int Key_Shared_VALUE = 3; + + + public final int getNumber() { return value; } + + public static SubType valueOf(int value) { + switch (value) { + case 0: return Exclusive; + case 1: return Shared; + case 2: return Failover; + case 3: return Key_Shared; + 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 SubType findValueByNumber(int number) { + return SubType.valueOf(number); + } + }; + + private final int value; + + private SubType(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandSubscribe.SubType) + } + + public enum InitialPosition + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + Latest(0, 0), + Earliest(1, 1), + ; + + public static final int Latest_VALUE = 0; + public static final int Earliest_VALUE = 1; + + + public final int getNumber() { return value; } + + public static InitialPosition valueOf(int value) { + switch (value) { + case 0: return Latest; + case 1: return Earliest; + 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 InitialPosition findValueByNumber(int number) { + return InitialPosition.valueOf(number); + } + }; + + private final int value; + + private InitialPosition(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandSubscribe.InitialPosition) + } + + private int bitField0_; + // required string topic = 1; + public static final int TOPIC_FIELD_NUMBER = 1; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required string subscription = 2; + public static final int SUBSCRIPTION_FIELD_NUMBER = 2; + private java.lang.Object subscription_; + public boolean hasSubscription() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSubscription() { + java.lang.Object ref = subscription_; + 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)) { + subscription_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSubscriptionBytes() { + java.lang.Object ref = subscription_; + 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); + subscription_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required .pulsar.proto.CommandSubscribe.SubType subType = 3; + public static final int SUBTYPE_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType subType_; + public boolean hasSubType() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType() { + return subType_; + } + + // required uint64 consumer_id = 4; + public static final int CONSUMER_ID_FIELD_NUMBER = 4; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint64 request_id = 5; + public static final int REQUEST_ID_FIELD_NUMBER = 5; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getRequestId() { + return requestId_; + } + + // optional string consumer_name = 6; + public static final int CONSUMER_NAME_FIELD_NUMBER = 6; + private java.lang.Object consumerName_; + public boolean hasConsumerName() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getConsumerName() { + java.lang.Object ref = consumerName_; + 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)) { + consumerName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConsumerNameBytes() { + java.lang.Object ref = consumerName_; + 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); + consumerName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional int32 priority_level = 7; + public static final int PRIORITY_LEVEL_FIELD_NUMBER = 7; + private int priorityLevel_; + public boolean hasPriorityLevel() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public int getPriorityLevel() { + return priorityLevel_; + } + + // optional bool durable = 8 [default = true]; + public static final int DURABLE_FIELD_NUMBER = 8; + private boolean durable_; + public boolean hasDurable() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public boolean getDurable() { + return durable_; + } + + // optional .pulsar.proto.MessageIdData start_message_id = 9; + public static final int START_MESSAGE_ID_FIELD_NUMBER = 9; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData startMessageId_; + public boolean hasStartMessageId() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId() { + return startMessageId_; + } + + // repeated .pulsar.proto.KeyValue metadata = 10; + public static final int METADATA_FIELD_NUMBER = 10; + private java.util.List metadata_; + public java.util.List getMetadataList() { + return metadata_; + } + public java.util.List + getMetadataOrBuilderList() { + return metadata_; + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( + int index) { + return metadata_.get(index); + } + + // optional bool read_compacted = 11; + public static final int READ_COMPACTED_FIELD_NUMBER = 11; + private boolean readCompacted_; + public boolean hasReadCompacted() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public boolean getReadCompacted() { + return readCompacted_; + } + + // optional .pulsar.proto.Schema schema = 12; + public static final int SCHEMA_FIELD_NUMBER = 12; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; + public boolean hasSchema() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + + // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; + public static final int INITIALPOSITION_FIELD_NUMBER = 13; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition initialPosition_; + public boolean hasInitialPosition() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition() { + return initialPosition_; + } + + // optional bool replicate_subscription_state = 14; + public static final int REPLICATE_SUBSCRIPTION_STATE_FIELD_NUMBER = 14; + private boolean replicateSubscriptionState_; + public boolean hasReplicateSubscriptionState() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public boolean getReplicateSubscriptionState() { + return replicateSubscriptionState_; + } + + // optional bool force_topic_creation = 15 [default = true]; + public static final int FORCE_TOPIC_CREATION_FIELD_NUMBER = 15; + private boolean forceTopicCreation_; + public boolean hasForceTopicCreation() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public boolean getForceTopicCreation() { + return forceTopicCreation_; + } + + // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; + public static final int START_MESSAGE_ROLLBACK_DURATION_SEC_FIELD_NUMBER = 16; + private long startMessageRollbackDurationSec_; + public boolean hasStartMessageRollbackDurationSec() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public long getStartMessageRollbackDurationSec() { + return startMessageRollbackDurationSec_; + } + + // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; + public static final int KEYSHAREDMETA_FIELD_NUMBER = 17; + private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta keySharedMeta_; + public boolean hasKeySharedMeta() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta() { + return keySharedMeta_; + } + + private void initFields() { + topic_ = ""; + subscription_ = ""; + subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; + consumerId_ = 0L; + requestId_ = 0L; + consumerName_ = ""; + priorityLevel_ = 0; + durable_ = true; + startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + metadata_ = java.util.Collections.emptyList(); + readCompacted_ = false; + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; + replicateSubscriptionState_ = false; + forceTopicCreation_ = true; + startMessageRollbackDurationSec_ = 0L; + keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSubscription()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSubType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasStartMessageId()) { + if (!getStartMessageId().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSchema()) { + if (!getSchema().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasKeySharedMeta()) { + if (!getKeySharedMeta().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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSubscriptionBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(3, subType_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, consumerId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeUInt64(5, requestId_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, getConsumerNameBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeInt32(7, priorityLevel_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeBool(8, durable_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeMessage(9, startMessageId_); + } + for (int i = 0; i < metadata_.size(); i++) { + output.writeMessage(10, metadata_.get(i)); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBool(11, readCompacted_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeMessage(12, schema_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeEnum(13, initialPosition_.getNumber()); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeBool(14, replicateSubscriptionState_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeBool(15, forceTopicCreation_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + output.writeUInt64(16, startMessageRollbackDurationSec_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + output.writeMessage(17, keySharedMeta_); + } + } + + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getSubscriptionBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(3, subType_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, consumerId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(5, requestId_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(6, getConsumerNameBytes()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(7, priorityLevel_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(8, durable_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(9, startMessageId_); + } + for (int i = 0; i < metadata_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(10, metadata_.get(i)); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(11, readCompacted_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(12, schema_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(13, initialPosition_.getNumber()); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(14, replicateSubscriptionState_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(15, forceTopicCreation_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(16, startMessageRollbackDurationSec_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(17, keySharedMeta_); + } + 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe 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.PulsarApi.CommandSubscribe, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.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(); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + subscription_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; + bitField0_ = (bitField0_ & ~0x00000004); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000010); + consumerName_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + priorityLevel_ = 0; + bitField0_ = (bitField0_ & ~0x00000040); + durable_ = true; + bitField0_ = (bitField0_ & ~0x00000080); + startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000100); + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000200); + readCompacted_ = false; + bitField0_ = (bitField0_ & ~0x00000400); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000800); + initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; + bitField0_ = (bitField0_ & ~0x00001000); + replicateSubscriptionState_ = false; + bitField0_ = (bitField0_ & ~0x00002000); + forceTopicCreation_ = true; + bitField0_ = (bitField0_ & ~0x00004000); + startMessageRollbackDurationSec_ = 0L; + bitField0_ = (bitField0_ & ~0x00008000); + keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00010000); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.subscription_ = subscription_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.subType_ = subType_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.consumerName_ = consumerName_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.priorityLevel_ = priorityLevel_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.durable_ = durable_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.startMessageId_ = startMessageId_; + if (((bitField0_ & 0x00000200) == 0x00000200)) { + metadata_ = java.util.Collections.unmodifiableList(metadata_); + bitField0_ = (bitField0_ & ~0x00000200); + } + result.metadata_ = metadata_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000200; + } + result.readCompacted_ = readCompacted_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000400; + } + result.schema_ = schema_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00000800; + } + result.initialPosition_ = initialPosition_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00001000; + } + result.replicateSubscriptionState_ = replicateSubscriptionState_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00002000; + } + result.forceTopicCreation_ = forceTopicCreation_; + if (((from_bitField0_ & 0x00008000) == 0x00008000)) { + to_bitField0_ |= 0x00004000; + } + result.startMessageRollbackDurationSec_ = startMessageRollbackDurationSec_; + if (((from_bitField0_ & 0x00010000) == 0x00010000)) { + to_bitField0_ |= 0x00008000; + } + result.keySharedMeta_ = keySharedMeta_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance()) return this; + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasSubscription()) { + setSubscription(other.getSubscription()); + } + if (other.hasSubType()) { + setSubType(other.getSubType()); + } + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasConsumerName()) { + setConsumerName(other.getConsumerName()); + } + if (other.hasPriorityLevel()) { + setPriorityLevel(other.getPriorityLevel()); + } + if (other.hasDurable()) { + setDurable(other.getDurable()); + } + if (other.hasStartMessageId()) { + mergeStartMessageId(other.getStartMessageId()); + } + if (!other.metadata_.isEmpty()) { + if (metadata_.isEmpty()) { + metadata_ = other.metadata_; + bitField0_ = (bitField0_ & ~0x00000200); + } else { + ensureMetadataIsMutable(); + metadata_.addAll(other.metadata_); + } + + } + if (other.hasReadCompacted()) { + setReadCompacted(other.getReadCompacted()); + } + if (other.hasSchema()) { + mergeSchema(other.getSchema()); + } + if (other.hasInitialPosition()) { + setInitialPosition(other.getInitialPosition()); + } + if (other.hasReplicateSubscriptionState()) { + setReplicateSubscriptionState(other.getReplicateSubscriptionState()); + } + if (other.hasForceTopicCreation()) { + setForceTopicCreation(other.getForceTopicCreation()); + } + if (other.hasStartMessageRollbackDurationSec()) { + setStartMessageRollbackDurationSec(other.getStartMessageRollbackDurationSec()); + } + if (other.hasKeySharedMeta()) { + mergeKeySharedMeta(other.getKeySharedMeta()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasTopic()) { + + return false; + } + if (!hasSubscription()) { + + return false; + } + if (!hasSubType()) { + + return false; + } + if (!hasConsumerId()) { + + return false; + } + if (!hasRequestId()) { + + return false; + } + if (hasStartMessageId()) { + if (!getStartMessageId().isInitialized()) { + + return false; + } + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + + return false; + } + } + if (hasSchema()) { + if (!getSchema().isInitialized()) { + + return false; + } + } + if (hasKeySharedMeta()) { + if (!getKeySharedMeta().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; + topic_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + subscription_ = input.readBytes(); + break; + } + case 24: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + subType_ = value; + } + break; + } + case 32: { + bitField0_ |= 0x00000008; + consumerId_ = input.readUInt64(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + requestId_ = input.readUInt64(); + break; + } + case 50: { + bitField0_ |= 0x00000020; + consumerName_ = input.readBytes(); + break; + } + case 56: { + bitField0_ |= 0x00000040; + priorityLevel_ = input.readInt32(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + durable_ = input.readBool(); + break; + } + case 74: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasStartMessageId()) { + subBuilder.mergeFrom(getStartMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setStartMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 82: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMetadata(subBuilder.buildPartial()); + break; + } + case 88: { + bitField0_ |= 0x00000400; + readCompacted_ = input.readBool(); + break; + } + case 98: { + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); + if (hasSchema()) { + subBuilder.mergeFrom(getSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 104: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition value = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00001000; + initialPosition_ = value; + } + break; + } + case 112: { + bitField0_ |= 0x00002000; + replicateSubscriptionState_ = input.readBool(); + break; + } + case 120: { + bitField0_ |= 0x00004000; + forceTopicCreation_ = input.readBool(); + break; + } + case 128: { + bitField0_ |= 0x00008000; + startMessageRollbackDurationSec_ = input.readUInt64(); + break; + } + case 138: { + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.newBuilder(); + if (hasKeySharedMeta()) { + subBuilder.mergeFrom(getKeySharedMeta()); + } + input.readMessage(subBuilder, extensionRegistry); + setKeySharedMeta(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + } + } + } + + private int bitField0_; + + // required string topic = 1; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + topic_ = value; + + } + + // required string subscription = 2; + private java.lang.Object subscription_ = ""; + public boolean hasSubscription() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSubscription() { + java.lang.Object ref = subscription_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + subscription_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setSubscription(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + subscription_ = value; + + return this; + } + public Builder clearSubscription() { + bitField0_ = (bitField0_ & ~0x00000002); + subscription_ = getDefaultInstance().getSubscription(); + + return this; + } + void setSubscription(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + subscription_ = value; + + } + + // required .pulsar.proto.CommandSubscribe.SubType subType = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; + public boolean hasSubType() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType getSubType() { + return subType_; + } + public Builder setSubType(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + subType_ = value; + + return this; + } + public Builder clearSubType() { + bitField0_ = (bitField0_ & ~0x00000004); + subType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.SubType.Exclusive; + + return this; + } + + // required uint64 consumer_id = 4; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000008; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000008); + consumerId_ = 0L; + + return this; + } + + // required uint64 request_id = 5; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000010; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000010); + requestId_ = 0L; + + return this; + } + + // optional string consumer_name = 6; + private java.lang.Object consumerName_ = ""; + public boolean hasConsumerName() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getConsumerName() { + java.lang.Object ref = consumerName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + consumerName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConsumerName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + consumerName_ = value; + + return this; + } + public Builder clearConsumerName() { + bitField0_ = (bitField0_ & ~0x00000020); + consumerName_ = getDefaultInstance().getConsumerName(); + + return this; + } + void setConsumerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000020; + consumerName_ = value; + + } + + // optional int32 priority_level = 7; + private int priorityLevel_ ; + public boolean hasPriorityLevel() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public int getPriorityLevel() { + return priorityLevel_; + } + public Builder setPriorityLevel(int value) { + bitField0_ |= 0x00000040; + priorityLevel_ = value; + + return this; + } + public Builder clearPriorityLevel() { + bitField0_ = (bitField0_ & ~0x00000040); + priorityLevel_ = 0; + + return this; + } + + // optional bool durable = 8 [default = true]; + private boolean durable_ = true; + public boolean hasDurable() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public boolean getDurable() { + return durable_; + } + public Builder setDurable(boolean value) { + bitField0_ |= 0x00000080; + durable_ = value; + + return this; + } + public Builder clearDurable() { + bitField0_ = (bitField0_ & ~0x00000080); + durable_ = true; + + return this; + } + + // optional .pulsar.proto.MessageIdData start_message_id = 9; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasStartMessageId() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getStartMessageId() { + return startMessageId_; + } + public Builder setStartMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + startMessageId_ = value; + + bitField0_ |= 0x00000100; + return this; + } + public Builder setStartMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + startMessageId_ = builderForValue.build(); + + bitField0_ |= 0x00000100; + return this; + } + public Builder mergeStartMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000100) == 0x00000100) && + startMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + startMessageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(startMessageId_).mergeFrom(value).buildPartial(); + } else { + startMessageId_ = value; + } + + bitField0_ |= 0x00000100; + return this; + } + public Builder clearStartMessageId() { + startMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + // repeated .pulsar.proto.KeyValue metadata = 10; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + private void ensureMetadataIsMutable() { + if (!((bitField0_ & 0x00000200) == 0x00000200)) { + metadata_ = new java.util.ArrayList(metadata_); + bitField0_ |= 0x00000200; + } + } + + public java.util.List getMetadataList() { + return java.util.Collections.unmodifiableList(metadata_); + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.set(index, value); + + return this; + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.set(index, builderForValue.build()); + + return this; + } + public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(value); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(index, value); + + return this; + } + public Builder addMetadata( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(builderForValue.build()); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllMetadata( + java.lang.Iterable values) { + ensureMetadataIsMutable(); + super.addAll(values, metadata_); + + return this; + } + public Builder clearMetadata() { + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000200); + + return this; + } + public Builder removeMetadata(int index) { + ensureMetadataIsMutable(); + metadata_.remove(index); + + return this; + } + + // optional bool read_compacted = 11; + private boolean readCompacted_ ; + public boolean hasReadCompacted() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public boolean getReadCompacted() { + return readCompacted_; + } + public Builder setReadCompacted(boolean value) { + bitField0_ |= 0x00000400; + readCompacted_ = value; + + return this; + } + public Builder clearReadCompacted() { + bitField0_ = (bitField0_ & ~0x00000400); + readCompacted_ = false; + + return this; + } + + // optional .pulsar.proto.Schema schema = 12; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + public boolean hasSchema() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (value == null) { + throw new NullPointerException(); + } + schema_ = value; + + bitField0_ |= 0x00000800; + return this; + } + public Builder setSchema( + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { + schema_ = builderForValue.build(); + + bitField0_ |= 0x00000800; + return this; + } + public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (((bitField0_ & 0x00000800) == 0x00000800) && + schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { + schema_ = + org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); + } else { + schema_ = value; + } + + bitField0_ |= 0x00000800; + return this; + } + public Builder clearSchema() { + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000800); + return this; + } + + // optional .pulsar.proto.CommandSubscribe.InitialPosition initialPosition = 13 [default = Latest]; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; + public boolean hasInitialPosition() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition getInitialPosition() { + return initialPosition_; + } + public Builder setInitialPosition(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00001000; + initialPosition_ = value; + + return this; + } + public Builder clearInitialPosition() { + bitField0_ = (bitField0_ & ~0x00001000); + initialPosition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.InitialPosition.Latest; + + return this; + } + + // optional bool replicate_subscription_state = 14; + private boolean replicateSubscriptionState_ ; + public boolean hasReplicateSubscriptionState() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public boolean getReplicateSubscriptionState() { + return replicateSubscriptionState_; + } + public Builder setReplicateSubscriptionState(boolean value) { + bitField0_ |= 0x00002000; + replicateSubscriptionState_ = value; + + return this; + } + public Builder clearReplicateSubscriptionState() { + bitField0_ = (bitField0_ & ~0x00002000); + replicateSubscriptionState_ = false; + + return this; + } + + // optional bool force_topic_creation = 15 [default = true]; + private boolean forceTopicCreation_ = true; + public boolean hasForceTopicCreation() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public boolean getForceTopicCreation() { + return forceTopicCreation_; + } + public Builder setForceTopicCreation(boolean value) { + bitField0_ |= 0x00004000; + forceTopicCreation_ = value; + + return this; + } + public Builder clearForceTopicCreation() { + bitField0_ = (bitField0_ & ~0x00004000); + forceTopicCreation_ = true; + + return this; + } + + // optional uint64 start_message_rollback_duration_sec = 16 [default = 0]; + private long startMessageRollbackDurationSec_ ; + public boolean hasStartMessageRollbackDurationSec() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public long getStartMessageRollbackDurationSec() { + return startMessageRollbackDurationSec_; + } + public Builder setStartMessageRollbackDurationSec(long value) { + bitField0_ |= 0x00008000; + startMessageRollbackDurationSec_ = value; + + return this; + } + public Builder clearStartMessageRollbackDurationSec() { + bitField0_ = (bitField0_ & ~0x00008000); + startMessageRollbackDurationSec_ = 0L; + + return this; + } + + // optional .pulsar.proto.KeySharedMeta keySharedMeta = 17; + private org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); + public boolean hasKeySharedMeta() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta getKeySharedMeta() { + return keySharedMeta_; + } + public Builder setKeySharedMeta(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta value) { + if (value == null) { + throw new NullPointerException(); + } + keySharedMeta_ = value; + + bitField0_ |= 0x00010000; + return this; + } + public Builder setKeySharedMeta( + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.Builder builderForValue) { + keySharedMeta_ = builderForValue.build(); + + bitField0_ |= 0x00010000; + return this; + } + public Builder mergeKeySharedMeta(org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta value) { + if (((bitField0_ & 0x00010000) == 0x00010000) && + keySharedMeta_ != org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance()) { + keySharedMeta_ = + org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.newBuilder(keySharedMeta_).mergeFrom(value).buildPartial(); + } else { + keySharedMeta_ = value; + } + + bitField0_ |= 0x00010000; + return this; + } + public Builder clearKeySharedMeta() { + keySharedMeta_ = org.apache.pulsar.common.api.proto.PulsarApi.KeySharedMeta.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00010000); + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSubscribe) + } + + static { + defaultInstance = new CommandSubscribe(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSubscribe) + } + + public interface CommandPartitionedTopicMetadataOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string topic = 1; + boolean hasTopic(); + String getTopic(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + + // optional string original_principal = 3; + boolean hasOriginalPrincipal(); + String getOriginalPrincipal(); + + // optional string original_auth_data = 4; + boolean hasOriginalAuthData(); + String getOriginalAuthData(); + + // optional string original_auth_method = 5; + boolean hasOriginalAuthMethod(); + String getOriginalAuthMethod(); + } + public static final class CommandPartitionedTopicMetadata extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandPartitionedTopicMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandPartitionedTopicMetadata.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandPartitionedTopicMetadata(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandPartitionedTopicMetadata newObject(Handle handle) { + return new CommandPartitionedTopicMetadata(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandPartitionedTopicMetadata(boolean noInit) {} + + private static final CommandPartitionedTopicMetadata defaultInstance; + public static CommandPartitionedTopicMetadata getDefaultInstance() { + return defaultInstance; + } + + public CommandPartitionedTopicMetadata getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string topic = 1; + public static final int TOPIC_FIELD_NUMBER = 1; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + // optional string original_principal = 3; + public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 3; + private java.lang.Object originalPrincipal_; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + 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)) { + originalPrincipal_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { + java.lang.Object ref = originalPrincipal_; + 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); + originalPrincipal_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_data = 4; + public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 4; + private java.lang.Object originalAuthData_; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + 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)) { + originalAuthData_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { + java.lang.Object ref = originalAuthData_; + 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); + originalAuthData_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_method = 5; + public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 5; + private java.lang.Object originalAuthMethod_; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + 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)) { + originalAuthMethod_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { + java.lang.Object ref = originalAuthMethod_; + 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); + originalAuthMethod_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + topic_ = ""; + requestId_ = 0L; + originalPrincipal_ = ""; + originalAuthData_ = ""; + originalAuthMethod_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getOriginalAuthMethodBytes()); + } + } + + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getOriginalAuthMethodBytes()); + } + 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata 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.PulsarApi.CommandPartitionedTopicMetadata, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.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(); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + originalPrincipal_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + originalAuthData_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + originalAuthMethod_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.originalPrincipal_ = originalPrincipal_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.originalAuthData_ = originalAuthData_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.originalAuthMethod_ = originalAuthMethod_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance()) return this; + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasOriginalPrincipal()) { + setOriginalPrincipal(other.getOriginalPrincipal()); + } + if (other.hasOriginalAuthData()) { + setOriginalAuthData(other.getOriginalAuthData()); + } + if (other.hasOriginalAuthMethod()) { + setOriginalAuthMethod(other.getOriginalAuthMethod()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasTopic()) { + + return false; + } + if (!hasRequestId()) { + + 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; + topic_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + originalPrincipal_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + originalAuthData_ = input.readBytes(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + originalAuthMethod_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string topic = 1; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + topic_ = value; + + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // optional string original_principal = 3; + private java.lang.Object originalPrincipal_ = ""; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalPrincipal_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalPrincipal(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + originalPrincipal_ = value; + + return this; + } + public Builder clearOriginalPrincipal() { + bitField0_ = (bitField0_ & ~0x00000004); + originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); + + return this; + } + void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + originalPrincipal_ = value; + + } + + // optional string original_auth_data = 4; + private java.lang.Object originalAuthData_ = ""; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthData_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthData(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + originalAuthData_ = value; + + return this; + } + public Builder clearOriginalAuthData() { + bitField0_ = (bitField0_ & ~0x00000008); + originalAuthData_ = getDefaultInstance().getOriginalAuthData(); + + return this; + } + void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000008; + originalAuthData_ = value; + + } + + // optional string original_auth_method = 5; + private java.lang.Object originalAuthMethod_ = ""; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthMethod_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthMethod(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + originalAuthMethod_ = value; + + return this; + } + public Builder clearOriginalAuthMethod() { + bitField0_ = (bitField0_ & ~0x00000010); + originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); + + return this; + } + void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + originalAuthMethod_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPartitionedTopicMetadata) + } + + static { + defaultInstance = new CommandPartitionedTopicMetadata(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPartitionedTopicMetadata) + } + + public interface CommandPartitionedTopicMetadataResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional uint32 partitions = 1; + boolean hasPartitions(); + int getPartitions(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; + boolean hasResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandPartitionedTopicMetadataResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandPartitionedTopicMetadataResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandPartitionedTopicMetadataResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandPartitionedTopicMetadataResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandPartitionedTopicMetadataResponse newObject(Handle handle) { + return new CommandPartitionedTopicMetadataResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandPartitionedTopicMetadataResponse(boolean noInit) {} + + private static final CommandPartitionedTopicMetadataResponse defaultInstance; + public static CommandPartitionedTopicMetadataResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandPartitionedTopicMetadataResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public enum LookupType + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + Success(0, 0), + Failed(1, 1), + ; + + public static final int Success_VALUE = 0; + public static final int Failed_VALUE = 1; + + + public final int getNumber() { return value; } + + public static LookupType valueOf(int value) { + switch (value) { + case 0: return Success; + case 1: return Failed; + 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 LookupType findValueByNumber(int number) { + return LookupType.valueOf(number); + } + }; + + private final int value; + + private LookupType(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType) + } + + private int bitField0_; + // optional uint32 partitions = 1; + public static final int PARTITIONS_FIELD_NUMBER = 1; + private int partitions_; + public boolean hasPartitions() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getPartitions() { + return partitions_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; + public static final int RESPONSE_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType response_; + public boolean hasResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse() { + return response_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + partitions_ = 0; + requestId_ = 0L; + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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.writeUInt32(1, partitions_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(3, response_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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 + .computeUInt32Size(1, partitions_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(3, response_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse 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.PulsarApi.CommandPartitionedTopicMetadataResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.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(); + partitions_ = 0; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.partitions_ = partitions_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.response_ = response_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance()) return this; + if (other.hasPartitions()) { + setPartitions(other.getPartitions()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasResponse()) { + setResponse(other.getResponse()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + partitions_ = input.readUInt32(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + case 24: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + response_ = value; + } + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // optional uint32 partitions = 1; + private int partitions_ ; + public boolean hasPartitions() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public int getPartitions() { + return partitions_; + } + public Builder setPartitions(int value) { + bitField0_ |= 0x00000001; + partitions_ = value; + + return this; + } + public Builder clearPartitions() { + bitField0_ = (bitField0_ & ~0x00000001); + partitions_ = 0; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse.LookupType response = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; + public boolean hasResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType getResponse() { + return response_; + } + public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + response_ = value; + + return this; + } + public Builder clearResponse() { + bitField0_ = (bitField0_ & ~0x00000004); + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.LookupType.Success; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse) + } + + static { + defaultInstance = new CommandPartitionedTopicMetadataResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPartitionedTopicMetadataResponse) + } + + public interface CommandLookupTopicOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string topic = 1; + boolean hasTopic(); + String getTopic(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + + // optional bool authoritative = 3 [default = false]; + boolean hasAuthoritative(); + boolean getAuthoritative(); + + // optional string original_principal = 4; + boolean hasOriginalPrincipal(); + String getOriginalPrincipal(); + + // optional string original_auth_data = 5; + boolean hasOriginalAuthData(); + String getOriginalAuthData(); + + // optional string original_auth_method = 6; + boolean hasOriginalAuthMethod(); + String getOriginalAuthMethod(); + } + public static final class CommandLookupTopic extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandLookupTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandLookupTopic.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandLookupTopic(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandLookupTopic newObject(Handle handle) { + return new CommandLookupTopic(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandLookupTopic(boolean noInit) {} + + private static final CommandLookupTopic defaultInstance; + public static CommandLookupTopic getDefaultInstance() { + return defaultInstance; + } + + public CommandLookupTopic getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string topic = 1; + public static final int TOPIC_FIELD_NUMBER = 1; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + // optional bool authoritative = 3 [default = false]; + public static final int AUTHORITATIVE_FIELD_NUMBER = 3; + private boolean authoritative_; + public boolean hasAuthoritative() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public boolean getAuthoritative() { + return authoritative_; + } + + // optional string original_principal = 4; + public static final int ORIGINAL_PRINCIPAL_FIELD_NUMBER = 4; + private java.lang.Object originalPrincipal_; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + 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)) { + originalPrincipal_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalPrincipalBytes() { + java.lang.Object ref = originalPrincipal_; + 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); + originalPrincipal_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_data = 5; + public static final int ORIGINAL_AUTH_DATA_FIELD_NUMBER = 5; + private java.lang.Object originalAuthData_; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + 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)) { + originalAuthData_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthDataBytes() { + java.lang.Object ref = originalAuthData_; + 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); + originalAuthData_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string original_auth_method = 6; + public static final int ORIGINAL_AUTH_METHOD_FIELD_NUMBER = 6; + private java.lang.Object originalAuthMethod_; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + 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)) { + originalAuthMethod_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginalAuthMethodBytes() { + java.lang.Object ref = originalAuthMethod_; + 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); + originalAuthMethod_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + topic_ = ""; + requestId_ = 0L; + authoritative_ = false; + originalPrincipal_ = ""; + originalAuthData_ = ""; + originalAuthMethod_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBool(3, authoritative_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeBytes(6, getOriginalAuthMethodBytes()); + } + } + + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(3, authoritative_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, getOriginalPrincipalBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getOriginalAuthDataBytes()); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(6, getOriginalAuthMethodBytes()); + } + 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic 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.PulsarApi.CommandLookupTopic, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.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(); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + authoritative_ = false; + bitField0_ = (bitField0_ & ~0x00000004); + originalPrincipal_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + originalAuthData_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + originalAuthMethod_ = ""; + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic result = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.authoritative_ = authoritative_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.originalPrincipal_ = originalPrincipal_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.originalAuthData_ = originalAuthData_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.originalAuthMethod_ = originalAuthMethod_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance()) return this; + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasAuthoritative()) { + setAuthoritative(other.getAuthoritative()); + } + if (other.hasOriginalPrincipal()) { + setOriginalPrincipal(other.getOriginalPrincipal()); + } + if (other.hasOriginalAuthData()) { + setOriginalAuthData(other.getOriginalAuthData()); + } + if (other.hasOriginalAuthMethod()) { + setOriginalAuthMethod(other.getOriginalAuthMethod()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasTopic()) { + + return false; + } + if (!hasRequestId()) { + + 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; + topic_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + authoritative_ = input.readBool(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + originalPrincipal_ = input.readBytes(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + originalAuthData_ = input.readBytes(); + break; + } + case 50: { + bitField0_ |= 0x00000020; + originalAuthMethod_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string topic = 1; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + topic_ = value; + + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // optional bool authoritative = 3 [default = false]; + private boolean authoritative_ ; + public boolean hasAuthoritative() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public boolean getAuthoritative() { + return authoritative_; + } + public Builder setAuthoritative(boolean value) { + bitField0_ |= 0x00000004; + authoritative_ = value; + + return this; + } + public Builder clearAuthoritative() { + bitField0_ = (bitField0_ & ~0x00000004); + authoritative_ = false; + + return this; + } + + // optional string original_principal = 4; + private java.lang.Object originalPrincipal_ = ""; + public boolean hasOriginalPrincipal() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getOriginalPrincipal() { + java.lang.Object ref = originalPrincipal_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalPrincipal_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalPrincipal(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + originalPrincipal_ = value; + + return this; + } + public Builder clearOriginalPrincipal() { + bitField0_ = (bitField0_ & ~0x00000008); + originalPrincipal_ = getDefaultInstance().getOriginalPrincipal(); + + return this; + } + void setOriginalPrincipal(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000008; + originalPrincipal_ = value; + + } + + // optional string original_auth_data = 5; + private java.lang.Object originalAuthData_ = ""; + public boolean hasOriginalAuthData() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getOriginalAuthData() { + java.lang.Object ref = originalAuthData_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthData_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthData(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + originalAuthData_ = value; + + return this; + } + public Builder clearOriginalAuthData() { + bitField0_ = (bitField0_ & ~0x00000010); + originalAuthData_ = getDefaultInstance().getOriginalAuthData(); + + return this; + } + void setOriginalAuthData(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + originalAuthData_ = value; + + } + + // optional string original_auth_method = 6; + private java.lang.Object originalAuthMethod_ = ""; + public boolean hasOriginalAuthMethod() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public String getOriginalAuthMethod() { + java.lang.Object ref = originalAuthMethod_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + originalAuthMethod_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setOriginalAuthMethod(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + originalAuthMethod_ = value; + + return this; + } + public Builder clearOriginalAuthMethod() { + bitField0_ = (bitField0_ & ~0x00000020); + originalAuthMethod_ = getDefaultInstance().getOriginalAuthMethod(); + + return this; + } + void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000020; + originalAuthMethod_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandLookupTopic) + } + + static { + defaultInstance = new CommandLookupTopic(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandLookupTopic) + } + + public interface CommandLookupTopicResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional string brokerServiceUrl = 1; + boolean hasBrokerServiceUrl(); + String getBrokerServiceUrl(); + + // optional string brokerServiceUrlTls = 2; + boolean hasBrokerServiceUrlTls(); + String getBrokerServiceUrlTls(); + + // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; + boolean hasResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse(); + + // required uint64 request_id = 4; + boolean hasRequestId(); + long getRequestId(); + + // optional bool authoritative = 5 [default = false]; + boolean hasAuthoritative(); + boolean getAuthoritative(); + + // optional .pulsar.proto.ServerError error = 6; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 7; + boolean hasMessage(); + String getMessage(); + + // optional bool proxy_through_service_url = 8 [default = false]; + boolean hasProxyThroughServiceUrl(); + boolean getProxyThroughServiceUrl(); + } + public static final class CommandLookupTopicResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandLookupTopicResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandLookupTopicResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandLookupTopicResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandLookupTopicResponse newObject(Handle handle) { + return new CommandLookupTopicResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandLookupTopicResponse(boolean noInit) {} + + private static final CommandLookupTopicResponse defaultInstance; + public static CommandLookupTopicResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandLookupTopicResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public enum LookupType + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + Redirect(0, 0), + Connect(1, 1), + Failed(2, 2), + ; + + public static final int Redirect_VALUE = 0; + public static final int Connect_VALUE = 1; + public static final int Failed_VALUE = 2; + + + public final int getNumber() { return value; } + + public static LookupType valueOf(int value) { + switch (value) { + case 0: return Redirect; + case 1: return Connect; + case 2: return Failed; + 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 LookupType findValueByNumber(int number) { + return LookupType.valueOf(number); + } + }; + + private final int value; + + private LookupType(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandLookupTopicResponse.LookupType) + } + + private int bitField0_; + // optional string brokerServiceUrl = 1; + public static final int BROKERSERVICEURL_FIELD_NUMBER = 1; + private java.lang.Object brokerServiceUrl_; + public boolean hasBrokerServiceUrl() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getBrokerServiceUrl() { + java.lang.Object ref = brokerServiceUrl_; + 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)) { + brokerServiceUrl_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getBrokerServiceUrlBytes() { + java.lang.Object ref = brokerServiceUrl_; + 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); + brokerServiceUrl_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string brokerServiceUrlTls = 2; + public static final int BROKERSERVICEURLTLS_FIELD_NUMBER = 2; + private java.lang.Object brokerServiceUrlTls_; + public boolean hasBrokerServiceUrlTls() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getBrokerServiceUrlTls() { + java.lang.Object ref = brokerServiceUrlTls_; + 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)) { + brokerServiceUrlTls_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getBrokerServiceUrlTlsBytes() { + java.lang.Object ref = brokerServiceUrlTls_; + 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); + brokerServiceUrlTls_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; + public static final int RESPONSE_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType response_; + public boolean hasResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse() { + return response_; + } + + // required uint64 request_id = 4; + public static final int REQUEST_ID_FIELD_NUMBER = 4; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getRequestId() { + return requestId_; + } + + // optional bool authoritative = 5 [default = false]; + public static final int AUTHORITATIVE_FIELD_NUMBER = 5; + private boolean authoritative_; + public boolean hasAuthoritative() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getAuthoritative() { + return authoritative_; + } + + // optional .pulsar.proto.ServerError error = 6; + public static final int ERROR_FIELD_NUMBER = 6; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 7; + public static final int MESSAGE_FIELD_NUMBER = 7; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bool proxy_through_service_url = 8 [default = false]; + public static final int PROXY_THROUGH_SERVICE_URL_FIELD_NUMBER = 8; + private boolean proxyThroughServiceUrl_; + public boolean hasProxyThroughServiceUrl() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public boolean getProxyThroughServiceUrl() { + return proxyThroughServiceUrl_; + } + + private void initFields() { + brokerServiceUrl_ = ""; + brokerServiceUrlTls_ = ""; + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; + requestId_ = 0L; + authoritative_ = false; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + proxyThroughServiceUrl_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, getBrokerServiceUrlBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getBrokerServiceUrlTlsBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(3, response_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, requestId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, authoritative_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeEnum(6, error_.getNumber()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBytes(7, getMessageBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeBool(8, proxyThroughServiceUrl_); + } + } + + 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, getBrokerServiceUrlBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getBrokerServiceUrlTlsBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(3, response_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, requestId_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(5, authoritative_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(6, error_.getNumber()); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(7, getMessageBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(8, proxyThroughServiceUrl_); + } + 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse 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.PulsarApi.CommandLookupTopicResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.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(); + brokerServiceUrl_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + brokerServiceUrlTls_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; + bitField0_ = (bitField0_ & ~0x00000004); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + authoritative_ = false; + bitField0_ = (bitField0_ & ~0x00000010); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000020); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000040); + proxyThroughServiceUrl_ = false; + bitField0_ = (bitField0_ & ~0x00000080); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.brokerServiceUrl_ = brokerServiceUrl_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.brokerServiceUrlTls_ = brokerServiceUrlTls_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.response_ = response_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.authoritative_ = authoritative_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.proxyThroughServiceUrl_ = proxyThroughServiceUrl_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance()) return this; + if (other.hasBrokerServiceUrl()) { + setBrokerServiceUrl(other.getBrokerServiceUrl()); + } + if (other.hasBrokerServiceUrlTls()) { + setBrokerServiceUrlTls(other.getBrokerServiceUrlTls()); + } + if (other.hasResponse()) { + setResponse(other.getResponse()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasAuthoritative()) { + setAuthoritative(other.getAuthoritative()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + if (other.hasProxyThroughServiceUrl()) { + setProxyThroughServiceUrl(other.getProxyThroughServiceUrl()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + brokerServiceUrl_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + brokerServiceUrlTls_ = input.readBytes(); + break; + } + case 24: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + response_ = value; + } + break; + } + case 32: { + bitField0_ |= 0x00000008; + requestId_ = input.readUInt64(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + authoritative_ = input.readBool(); + break; + } + case 48: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000020; + error_ = value; + } + break; + } + case 58: { + bitField0_ |= 0x00000040; + message_ = input.readBytes(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + proxyThroughServiceUrl_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // optional string brokerServiceUrl = 1; + private java.lang.Object brokerServiceUrl_ = ""; + public boolean hasBrokerServiceUrl() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getBrokerServiceUrl() { + java.lang.Object ref = brokerServiceUrl_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + brokerServiceUrl_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setBrokerServiceUrl(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + brokerServiceUrl_ = value; + + return this; + } + public Builder clearBrokerServiceUrl() { + bitField0_ = (bitField0_ & ~0x00000001); + brokerServiceUrl_ = getDefaultInstance().getBrokerServiceUrl(); + + return this; + } + void setBrokerServiceUrl(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + brokerServiceUrl_ = value; + + } + + // optional string brokerServiceUrlTls = 2; + private java.lang.Object brokerServiceUrlTls_ = ""; + public boolean hasBrokerServiceUrlTls() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getBrokerServiceUrlTls() { + java.lang.Object ref = brokerServiceUrlTls_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + brokerServiceUrlTls_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setBrokerServiceUrlTls(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + brokerServiceUrlTls_ = value; + + return this; + } + public Builder clearBrokerServiceUrlTls() { + bitField0_ = (bitField0_ & ~0x00000002); + brokerServiceUrlTls_ = getDefaultInstance().getBrokerServiceUrlTls(); + + return this; + } + void setBrokerServiceUrlTls(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + brokerServiceUrlTls_ = value; + + } + + // optional .pulsar.proto.CommandLookupTopicResponse.LookupType response = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; + public boolean hasResponse() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType getResponse() { + return response_; + } + public Builder setResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + response_ = value; + + return this; + } + public Builder clearResponse() { + bitField0_ = (bitField0_ & ~0x00000004); + response_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.LookupType.Redirect; + + return this; + } + + // required uint64 request_id = 4; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000008; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000008); + requestId_ = 0L; + + return this; + } + + // optional bool authoritative = 5 [default = false]; + private boolean authoritative_ ; + public boolean hasAuthoritative() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getAuthoritative() { + return authoritative_; + } + public Builder setAuthoritative(boolean value) { + bitField0_ |= 0x00000010; + authoritative_ = value; + + return this; + } + public Builder clearAuthoritative() { + bitField0_ = (bitField0_ & ~0x00000010); + authoritative_ = false; + + return this; + } + + // optional .pulsar.proto.ServerError error = 6; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000020; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000020); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 7; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000040; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000040); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000040; + message_ = value; + + } + + // optional bool proxy_through_service_url = 8 [default = false]; + private boolean proxyThroughServiceUrl_ ; + public boolean hasProxyThroughServiceUrl() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public boolean getProxyThroughServiceUrl() { + return proxyThroughServiceUrl_; + } + public Builder setProxyThroughServiceUrl(boolean value) { + bitField0_ |= 0x00000080; + proxyThroughServiceUrl_ = value; + + return this; + } + public Builder clearProxyThroughServiceUrl() { + bitField0_ = (bitField0_ & ~0x00000080); + proxyThroughServiceUrl_ = false; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandLookupTopicResponse) + } + + static { + defaultInstance = new CommandLookupTopicResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandLookupTopicResponse) + } + + public interface CommandProducerOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string topic = 1; + boolean hasTopic(); + String getTopic(); + + // required uint64 producer_id = 2; + boolean hasProducerId(); + long getProducerId(); + + // required uint64 request_id = 3; + boolean hasRequestId(); + long getRequestId(); + + // optional string producer_name = 4; + boolean hasProducerName(); + String getProducerName(); + + // optional bool encrypted = 5 [default = false]; + boolean hasEncrypted(); + boolean getEncrypted(); + + // repeated .pulsar.proto.KeyValue metadata = 6; + java.util.List + getMetadataList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index); + int getMetadataCount(); + + // optional .pulsar.proto.Schema schema = 7; + boolean hasSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); + + // optional uint64 epoch = 8 [default = 0]; + boolean hasEpoch(); + long getEpoch(); + + // optional bool user_provided_producer_name = 9 [default = true]; + boolean hasUserProvidedProducerName(); + boolean getUserProvidedProducerName(); + } + public static final class CommandProducer extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandProducer.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandProducer(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandProducer newObject(Handle handle) { + return new CommandProducer(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandProducer(boolean noInit) {} + + private static final CommandProducer defaultInstance; + public static CommandProducer getDefaultInstance() { + return defaultInstance; + } + + public CommandProducer getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string topic = 1; + public static final int TOPIC_FIELD_NUMBER = 1; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required uint64 producer_id = 2; + public static final int PRODUCER_ID_FIELD_NUMBER = 2; + private long producerId_; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getProducerId() { + return producerId_; + } + + // required uint64 request_id = 3; + public static final int REQUEST_ID_FIELD_NUMBER = 3; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getRequestId() { + return requestId_; + } + + // optional string producer_name = 4; + public static final int PRODUCER_NAME_FIELD_NUMBER = 4; + private java.lang.Object producerName_; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + 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)) { + producerName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { + java.lang.Object ref = producerName_; + 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); + producerName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bool encrypted = 5 [default = false]; + public static final int ENCRYPTED_FIELD_NUMBER = 5; + private boolean encrypted_; + public boolean hasEncrypted() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getEncrypted() { + return encrypted_; + } + + // repeated .pulsar.proto.KeyValue metadata = 6; + public static final int METADATA_FIELD_NUMBER = 6; + private java.util.List metadata_; + public java.util.List getMetadataList() { + return metadata_; + } + public java.util.List + getMetadataOrBuilderList() { + return metadata_; + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValueOrBuilder getMetadataOrBuilder( + int index) { + return metadata_.get(index); + } + + // optional .pulsar.proto.Schema schema = 7; + public static final int SCHEMA_FIELD_NUMBER = 7; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; + public boolean hasSchema() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + + // optional uint64 epoch = 8 [default = 0]; + public static final int EPOCH_FIELD_NUMBER = 8; + private long epoch_; + public boolean hasEpoch() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public long getEpoch() { + return epoch_; + } + + // optional bool user_provided_producer_name = 9 [default = true]; + public static final int USER_PROVIDED_PRODUCER_NAME_FIELD_NUMBER = 9; + private boolean userProvidedProducerName_; + public boolean hasUserProvidedProducerName() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public boolean getUserProvidedProducerName() { + return userProvidedProducerName_; + } + + private void initFields() { + topic_ = ""; + producerId_ = 0L; + requestId_ = 0L; + producerName_ = ""; + encrypted_ = false; + metadata_ = java.util.Collections.emptyList(); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + epoch_ = 0L; + userProvidedProducerName_ = true; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasProducerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSchema()) { + if (!getSchema().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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, producerId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, requestId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBool(5, encrypted_); + } + for (int i = 0; i < metadata_.size(); i++) { + output.writeMessage(6, metadata_.get(i)); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeMessage(7, schema_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeUInt64(8, epoch_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeBool(9, userProvidedProducerName_); + } + } + + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, producerId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, requestId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(5, encrypted_); + } + for (int i = 0; i < metadata_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(6, metadata_.get(i)); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(7, schema_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(8, epoch_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(9, userProvidedProducerName_); + } + 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer 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.PulsarApi.CommandProducer, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.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(); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + producerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + producerName_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + encrypted_ = false; + bitField0_ = (bitField0_ & ~0x00000010); + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000020); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000040); + epoch_ = 0L; + bitField0_ = (bitField0_ & ~0x00000080); + userProvidedProducerName_ = true; + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.producerId_ = producerId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.producerName_ = producerName_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.encrypted_ = encrypted_; + if (((bitField0_ & 0x00000020) == 0x00000020)) { + metadata_ = java.util.Collections.unmodifiableList(metadata_); + bitField0_ = (bitField0_ & ~0x00000020); + } + result.metadata_ = metadata_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000020; + } + result.schema_ = schema_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000040; + } + result.epoch_ = epoch_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000080; + } + result.userProvidedProducerName_ = userProvidedProducerName_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance()) return this; + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasProducerId()) { + setProducerId(other.getProducerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasProducerName()) { + setProducerName(other.getProducerName()); + } + if (other.hasEncrypted()) { + setEncrypted(other.getEncrypted()); + } + if (!other.metadata_.isEmpty()) { + if (metadata_.isEmpty()) { + metadata_ = other.metadata_; + bitField0_ = (bitField0_ & ~0x00000020); + } else { + ensureMetadataIsMutable(); + metadata_.addAll(other.metadata_); + } + + } + if (other.hasSchema()) { + mergeSchema(other.getSchema()); + } + if (other.hasEpoch()) { + setEpoch(other.getEpoch()); + } + if (other.hasUserProvidedProducerName()) { + setUserProvidedProducerName(other.getUserProvidedProducerName()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasTopic()) { + + return false; + } + if (!hasProducerId()) { + + return false; + } + if (!hasRequestId()) { + + return false; + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + + return false; + } + } + if (hasSchema()) { + if (!getSchema().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; + topic_ = input.readBytes(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + producerId_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + requestId_ = input.readUInt64(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + producerName_ = input.readBytes(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + encrypted_ = input.readBool(); + break; + } + case 50: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMetadata(subBuilder.buildPartial()); + break; + } + case 58: { + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); + if (hasSchema()) { + subBuilder.mergeFrom(getSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + epoch_ = input.readUInt64(); + break; + } + case 72: { + bitField0_ |= 0x00000100; + userProvidedProducerName_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // required string topic = 1; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + topic_ = value; + + } + + // required uint64 producer_id = 2; + private long producerId_ ; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getProducerId() { + return producerId_; + } + public Builder setProducerId(long value) { + bitField0_ |= 0x00000002; + producerId_ = value; + + return this; + } + public Builder clearProducerId() { + bitField0_ = (bitField0_ & ~0x00000002); + producerId_ = 0L; + + return this; + } + + // required uint64 request_id = 3; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000004; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000004); + requestId_ = 0L; + + return this; + } + + // optional string producer_name = 4; + private java.lang.Object producerName_ = ""; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + producerName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProducerName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + producerName_ = value; + + return this; + } + public Builder clearProducerName() { + bitField0_ = (bitField0_ & ~0x00000008); + producerName_ = getDefaultInstance().getProducerName(); + + return this; + } + void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000008; + producerName_ = value; + + } + + // optional bool encrypted = 5 [default = false]; + private boolean encrypted_ ; + public boolean hasEncrypted() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public boolean getEncrypted() { + return encrypted_; + } + public Builder setEncrypted(boolean value) { + bitField0_ |= 0x00000010; + encrypted_ = value; + + return this; + } + public Builder clearEncrypted() { + bitField0_ = (bitField0_ & ~0x00000010); + encrypted_ = false; + + return this; + } + + // repeated .pulsar.proto.KeyValue metadata = 6; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + private void ensureMetadataIsMutable() { + if (!((bitField0_ & 0x00000020) == 0x00000020)) { + metadata_ = new java.util.ArrayList(metadata_); + bitField0_ |= 0x00000020; + } + } + + public java.util.List getMetadataList() { + return java.util.Collections.unmodifiableList(metadata_); + } + public int getMetadataCount() { + return metadata_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyValue getMetadata(int index) { + return metadata_.get(index); + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.set(index, value); + + return this; + } + public Builder setMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.set(index, builderForValue.build()); + + return this; + } + public Builder addMetadata(org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(value); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(index, value); + + return this; + } + public Builder addMetadata( + org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(builderForValue.build()); + + return this; + } + public Builder addMetadata( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyValue.Builder builderForValue) { + ensureMetadataIsMutable(); + metadata_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllMetadata( + java.lang.Iterable values) { + ensureMetadataIsMutable(); + super.addAll(values, metadata_); + + return this; + } + public Builder clearMetadata() { + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000020); + + return this; + } + public Builder removeMetadata(int index) { + ensureMetadataIsMutable(); + metadata_.remove(index); + + return this; + } + + // optional .pulsar.proto.Schema schema = 7; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + public boolean hasSchema() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (value == null) { + throw new NullPointerException(); + } + schema_ = value; + + bitField0_ |= 0x00000040; + return this; + } + public Builder setSchema( + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { + schema_ = builderForValue.build(); + + bitField0_ |= 0x00000040; + return this; + } + public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (((bitField0_ & 0x00000040) == 0x00000040) && + schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { + schema_ = + org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); + } else { + schema_ = value; + } + + bitField0_ |= 0x00000040; + return this; + } + public Builder clearSchema() { + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000040); + return this; + } + + // optional uint64 epoch = 8 [default = 0]; + private long epoch_ ; + public boolean hasEpoch() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public long getEpoch() { + return epoch_; + } + public Builder setEpoch(long value) { + bitField0_ |= 0x00000080; + epoch_ = value; + + return this; + } + public Builder clearEpoch() { + bitField0_ = (bitField0_ & ~0x00000080); + epoch_ = 0L; + + return this; + } + + // optional bool user_provided_producer_name = 9 [default = true]; + private boolean userProvidedProducerName_ = true; + public boolean hasUserProvidedProducerName() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public boolean getUserProvidedProducerName() { + return userProvidedProducerName_; + } + public Builder setUserProvidedProducerName(boolean value) { + bitField0_ |= 0x00000100; + userProvidedProducerName_ = value; + + return this; + } + public Builder clearUserProvidedProducerName() { + bitField0_ = (bitField0_ & ~0x00000100); + userProvidedProducerName_ = true; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandProducer) + } + + static { + defaultInstance = new CommandProducer(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandProducer) + } + + public interface CommandSendOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 producer_id = 1; + boolean hasProducerId(); + long getProducerId(); + + // required uint64 sequence_id = 2; + boolean hasSequenceId(); + long getSequenceId(); + + // optional int32 num_messages = 3 [default = 1]; + boolean hasNumMessages(); + int getNumMessages(); + + // optional uint64 txnid_least_bits = 4 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 5 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional uint64 highest_sequence_id = 6 [default = 0]; + boolean hasHighestSequenceId(); + long getHighestSequenceId(); + } + public static final class CommandSend extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSendOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSend.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSend(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSend newObject(Handle handle) { + return new CommandSend(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSend(boolean noInit) {} + + private static final CommandSend defaultInstance; + public static CommandSend getDefaultInstance() { + return defaultInstance; + } + + public CommandSend getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 producer_id = 1; + public static final int PRODUCER_ID_FIELD_NUMBER = 1; + private long producerId_; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + + // required uint64 sequence_id = 2; + public static final int SEQUENCE_ID_FIELD_NUMBER = 2; + private long sequenceId_; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + + // optional int32 num_messages = 3 [default = 1]; + public static final int NUM_MESSAGES_FIELD_NUMBER = 3; + private int numMessages_; + public boolean hasNumMessages() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getNumMessages() { + return numMessages_; + } + + // optional uint64 txnid_least_bits = 4 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 4; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 5 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 5; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional uint64 highest_sequence_id = 6 [default = 0]; + public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 6; + private long highestSequenceId_; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + + private void initFields() { + producerId_ = 0L; + sequenceId_ = 0L; + numMessages_ = 1; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + highestSequenceId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProducerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSequenceId()) { + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt32(3, numMessages_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, txnidLeastBits_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeUInt64(5, txnidMostBits_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeUInt64(6, highestSequenceId_); + } + } + + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt32Size(3, numMessages_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, txnidLeastBits_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(5, txnidMostBits_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(6, highestSequenceId_); + } + 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend 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.PulsarApi.CommandSend, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.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(); + producerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + numMessages_ = 1; + bitField0_ = (bitField0_ & ~0x00000004); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000010); + highestSequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.producerId_ = producerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.sequenceId_ = sequenceId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.numMessages_ = numMessages_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.highestSequenceId_ = highestSequenceId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance()) return this; + if (other.hasProducerId()) { + setProducerId(other.getProducerId()); + } + if (other.hasSequenceId()) { + setSequenceId(other.getSequenceId()); + } + if (other.hasNumMessages()) { + setNumMessages(other.getNumMessages()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasHighestSequenceId()) { + setHighestSequenceId(other.getHighestSequenceId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasProducerId()) { + + return false; + } + if (!hasSequenceId()) { + + 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; + producerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + sequenceId_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + numMessages_ = input.readInt32(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 40: { + bitField0_ |= 0x00000010; + txnidMostBits_ = input.readUInt64(); + break; + } + case 48: { + bitField0_ |= 0x00000020; + highestSequenceId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 producer_id = 1; + private long producerId_ ; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + public Builder setProducerId(long value) { + bitField0_ |= 0x00000001; + producerId_ = value; + + return this; + } + public Builder clearProducerId() { + bitField0_ = (bitField0_ & ~0x00000001); + producerId_ = 0L; + + return this; + } + + // required uint64 sequence_id = 2; + private long sequenceId_ ; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + public Builder setSequenceId(long value) { + bitField0_ |= 0x00000002; + sequenceId_ = value; + + return this; + } + public Builder clearSequenceId() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceId_ = 0L; + + return this; + } + + // optional int32 num_messages = 3 [default = 1]; + private int numMessages_ = 1; + public boolean hasNumMessages() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getNumMessages() { + return numMessages_; + } + public Builder setNumMessages(int value) { + bitField0_ |= 0x00000004; + numMessages_ = value; + + return this; + } + public Builder clearNumMessages() { + bitField0_ = (bitField0_ & ~0x00000004); + numMessages_ = 1; + + return this; + } + + // optional uint64 txnid_least_bits = 4 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000008; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000008); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 5 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000010; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000010); + txnidMostBits_ = 0L; + + return this; + } + + // optional uint64 highest_sequence_id = 6 [default = 0]; + private long highestSequenceId_ ; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + public Builder setHighestSequenceId(long value) { + bitField0_ |= 0x00000020; + highestSequenceId_ = value; + + return this; + } + public Builder clearHighestSequenceId() { + bitField0_ = (bitField0_ & ~0x00000020); + highestSequenceId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSend) + } + + static { + defaultInstance = new CommandSend(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSend) + } + + public interface CommandSendReceiptOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 producer_id = 1; + boolean hasProducerId(); + long getProducerId(); + + // required uint64 sequence_id = 2; + boolean hasSequenceId(); + long getSequenceId(); + + // optional .pulsar.proto.MessageIdData message_id = 3; + boolean hasMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); + + // optional uint64 highest_sequence_id = 4 [default = 0]; + boolean hasHighestSequenceId(); + long getHighestSequenceId(); + } + public static final class CommandSendReceipt extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSendReceiptOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSendReceipt.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSendReceipt(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSendReceipt newObject(Handle handle) { + return new CommandSendReceipt(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSendReceipt(boolean noInit) {} + + private static final CommandSendReceipt defaultInstance; + public static CommandSendReceipt getDefaultInstance() { + return defaultInstance; + } + + public CommandSendReceipt getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 producer_id = 1; + public static final int PRODUCER_ID_FIELD_NUMBER = 1; + private long producerId_; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + + // required uint64 sequence_id = 2; + public static final int SEQUENCE_ID_FIELD_NUMBER = 2; + private long sequenceId_; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + + // optional .pulsar.proto.MessageIdData message_id = 3; + public static final int MESSAGE_ID_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; + public boolean hasMessageId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + + // optional uint64 highest_sequence_id = 4 [default = 0]; + public static final int HIGHEST_SEQUENCE_ID_FIELD_NUMBER = 4; + private long highestSequenceId_; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + + private void initFields() { + producerId_ = 0L; + sequenceId_ = 0L; + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + highestSequenceId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProducerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSequenceId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasMessageId()) { + 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.writeUInt64(1, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, messageId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, highestSequenceId_); + } + } + + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, messageId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, highestSequenceId_); + } + 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt 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.PulsarApi.CommandSendReceipt, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceiptOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.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(); + producerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000004); + highestSequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.producerId_ = producerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.sequenceId_ = sequenceId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.messageId_ = messageId_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.highestSequenceId_ = highestSequenceId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance()) return this; + if (other.hasProducerId()) { + setProducerId(other.getProducerId()); + } + if (other.hasSequenceId()) { + setSequenceId(other.getSequenceId()); + } + if (other.hasMessageId()) { + mergeMessageId(other.getMessageId()); + } + if (other.hasHighestSequenceId()) { + setHighestSequenceId(other.getHighestSequenceId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasProducerId()) { + + return false; + } + if (!hasSequenceId()) { + + return false; + } + if (hasMessageId()) { + 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 8: { + bitField0_ |= 0x00000001; + producerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + sequenceId_ = input.readUInt64(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasMessageId()) { + subBuilder.mergeFrom(getMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + highestSequenceId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 producer_id = 1; + private long producerId_ ; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + public Builder setProducerId(long value) { + bitField0_ |= 0x00000001; + producerId_ = value; + + return this; + } + public Builder clearProducerId() { + bitField0_ = (bitField0_ & ~0x00000001); + producerId_ = 0L; + + return this; + } + + // required uint64 sequence_id = 2; + private long sequenceId_ ; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + public Builder setSequenceId(long value) { + bitField0_ |= 0x00000002; + sequenceId_ = value; + + return this; + } + public Builder clearSequenceId() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceId_ = 0L; + + return this; + } + + // optional .pulsar.proto.MessageIdData message_id = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasMessageId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + messageId_ = value; + + bitField0_ |= 0x00000004; + return this; + } + public Builder setMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + messageId_ = builderForValue.build(); + + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + messageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); + } else { + messageId_ = value; + } + + bitField0_ |= 0x00000004; + return this; + } + public Builder clearMessageId() { + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + // optional uint64 highest_sequence_id = 4 [default = 0]; + private long highestSequenceId_ ; + public boolean hasHighestSequenceId() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getHighestSequenceId() { + return highestSequenceId_; + } + public Builder setHighestSequenceId(long value) { + bitField0_ |= 0x00000008; + highestSequenceId_ = value; + + return this; + } + public Builder clearHighestSequenceId() { + bitField0_ = (bitField0_ & ~0x00000008); + highestSequenceId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSendReceipt) + } + + static { + defaultInstance = new CommandSendReceipt(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSendReceipt) + } + + public interface CommandSendErrorOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 producer_id = 1; + boolean hasProducerId(); + long getProducerId(); + + // required uint64 sequence_id = 2; + boolean hasSequenceId(); + long getSequenceId(); + + // required .pulsar.proto.ServerError error = 3; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // required string message = 4; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandSendError extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSendErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSendError.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSendError(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSendError newObject(Handle handle) { + return new CommandSendError(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSendError(boolean noInit) {} + + private static final CommandSendError defaultInstance; + public static CommandSendError getDefaultInstance() { + return defaultInstance; + } + + public CommandSendError getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 producer_id = 1; + public static final int PRODUCER_ID_FIELD_NUMBER = 1; + private long producerId_; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + + // required uint64 sequence_id = 2; + public static final int SEQUENCE_ID_FIELD_NUMBER = 2; + private long sequenceId_; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + + // required .pulsar.proto.ServerError error = 3; + public static final int ERROR_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // required string message = 4; + public static final int MESSAGE_FIELD_NUMBER = 4; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + producerId_ = 0L; + sequenceId_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProducerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSequenceId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasError()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMessage()) { + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(3, error_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getMessageBytes()); + } + } + + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, sequenceId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(3, error_.getNumber()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, getMessageBytes()); + } + 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError 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.PulsarApi.CommandSendError, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSendErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.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(); + producerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + sequenceId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000004); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.producerId_ = producerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.sequenceId_ = sequenceId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance()) return this; + if (other.hasProducerId()) { + setProducerId(other.getProducerId()); + } + if (other.hasSequenceId()) { + setSequenceId(other.getSequenceId()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasProducerId()) { + + return false; + } + if (!hasSequenceId()) { + + return false; + } + if (!hasError()) { + + return false; + } + if (!hasMessage()) { + + 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; + producerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + sequenceId_ = input.readUInt64(); + break; + } + case 24: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + error_ = value; + } + break; + } + case 34: { + bitField0_ |= 0x00000008; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 producer_id = 1; + private long producerId_ ; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + public Builder setProducerId(long value) { + bitField0_ |= 0x00000001; + producerId_ = value; + + return this; + } + public Builder clearProducerId() { + bitField0_ = (bitField0_ & ~0x00000001); + producerId_ = 0L; + + return this; + } + + // required uint64 sequence_id = 2; + private long sequenceId_ ; + public boolean hasSequenceId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getSequenceId() { + return sequenceId_; + } + public Builder setSequenceId(long value) { + bitField0_ |= 0x00000002; + sequenceId_ = value; + + return this; + } + public Builder clearSequenceId() { + bitField0_ = (bitField0_ & ~0x00000002); + sequenceId_ = 0L; + + return this; + } + + // required .pulsar.proto.ServerError error = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // required string message = 4; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000008); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000008; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSendError) + } + + static { + defaultInstance = new CommandSendError(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSendError) + } + + public interface CommandMessageOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required .pulsar.proto.MessageIdData message_id = 2; + boolean hasMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); + + // optional uint32 redelivery_count = 3 [default = 0]; + boolean hasRedeliveryCount(); + int getRedeliveryCount(); + + // repeated int64 ack_set = 4; + java.util.List getAckSetList(); + int getAckSetCount(); + long getAckSet(int index); + } + public static final class CommandMessage extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandMessageOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandMessage.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandMessage(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandMessage newObject(Handle handle) { + return new CommandMessage(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandMessage(boolean noInit) {} + + private static final CommandMessage defaultInstance; + public static CommandMessage getDefaultInstance() { + return defaultInstance; + } + + public CommandMessage getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required .pulsar.proto.MessageIdData message_id = 2; + public static final int MESSAGE_ID_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; + public boolean hasMessageId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + + // optional uint32 redelivery_count = 3 [default = 0]; + public static final int REDELIVERY_COUNT_FIELD_NUMBER = 3; + private int redeliveryCount_; + public boolean hasRedeliveryCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getRedeliveryCount() { + return redeliveryCount_; + } + + // repeated int64 ack_set = 4; + public static final int ACK_SET_FIELD_NUMBER = 4; + private java.util.List ackSet_; + public java.util.List + getAckSetList() { + return ackSet_; + } + public int getAckSetCount() { + return ackSet_.size(); + } + public long getAckSet(int index) { + return ackSet_.get(index); + } + + private void initFields() { + consumerId_ = 0L; + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + redeliveryCount_ = 0; + ackSet_ = java.util.Collections.emptyList();; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + 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.writeUInt64(1, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, messageId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt32(3, redeliveryCount_); + } + for (int i = 0; i < ackSet_.size(); i++) { + output.writeInt64(4, ackSet_.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 + .computeUInt64Size(1, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, messageId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt32Size(3, redeliveryCount_); + } + { + int dataSize = 0; + for (int i = 0; i < ackSet_.size(); i++) { + dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt64SizeNoTag(ackSet_.get(i)); + } + size += dataSize; + size += 1 * getAckSetList().size(); + } + 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage 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.PulsarApi.CommandMessage, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandMessageOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000002); + redeliveryCount_ = 0; + bitField0_ = (bitField0_ & ~0x00000004); + ackSet_ = java.util.Collections.emptyList();; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage result = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.messageId_ = messageId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.redeliveryCount_ = redeliveryCount_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + ackSet_ = java.util.Collections.unmodifiableList(ackSet_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.ackSet_ = ackSet_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasMessageId()) { + mergeMessageId(other.getMessageId()); + } + if (other.hasRedeliveryCount()) { + setRedeliveryCount(other.getRedeliveryCount()); + } + if (!other.ackSet_.isEmpty()) { + if (ackSet_.isEmpty()) { + ackSet_ = other.ackSet_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureAckSetIsMutable(); + ackSet_.addAll(other.ackSet_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + 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 8: { + bitField0_ |= 0x00000001; + consumerId_ = input.readUInt64(); + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasMessageId()) { + subBuilder.mergeFrom(getMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + redeliveryCount_ = input.readUInt32(); + break; + } + case 32: { + ensureAckSetIsMutable(); + ackSet_.add(input.readInt64()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required .pulsar.proto.MessageIdData message_id = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasMessageId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + messageId_ = value; + + bitField0_ |= 0x00000002; + return this; + } + public Builder setMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + messageId_ = builderForValue.build(); + + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + messageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); + } else { + messageId_ = value; + } + + bitField0_ |= 0x00000002; + return this; + } + public Builder clearMessageId() { + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + // optional uint32 redelivery_count = 3 [default = 0]; + private int redeliveryCount_ ; + public boolean hasRedeliveryCount() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public int getRedeliveryCount() { + return redeliveryCount_; + } + public Builder setRedeliveryCount(int value) { + bitField0_ |= 0x00000004; + redeliveryCount_ = value; + + return this; + } + public Builder clearRedeliveryCount() { + bitField0_ = (bitField0_ & ~0x00000004); + redeliveryCount_ = 0; + + return this; + } + + // repeated int64 ack_set = 4; + private java.util.List ackSet_ = java.util.Collections.emptyList();; + private void ensureAckSetIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + ackSet_ = new java.util.ArrayList(ackSet_); + bitField0_ |= 0x00000008; + } + } + public java.util.List + getAckSetList() { + return java.util.Collections.unmodifiableList(ackSet_); + } + public int getAckSetCount() { + return ackSet_.size(); + } + public long getAckSet(int index) { + return ackSet_.get(index); + } + public Builder setAckSet( + int index, long value) { + ensureAckSetIsMutable(); + ackSet_.set(index, value); + + return this; + } + public Builder addAckSet(long value) { + ensureAckSetIsMutable(); + ackSet_.add(value); + + return this; + } + public Builder addAllAckSet( + java.lang.Iterable values) { + ensureAckSetIsMutable(); + super.addAll(values, ackSet_); + + return this; + } + public Builder clearAckSet() { + ackSet_ = java.util.Collections.emptyList();; + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandMessage) + } + + static { + defaultInstance = new CommandMessage(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandMessage) + } + + public interface CommandAckOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required .pulsar.proto.CommandAck.AckType ack_type = 2; + boolean hasAckType(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType(); + + // repeated .pulsar.proto.MessageIdData message_id = 3; + java.util.List + getMessageIdList(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index); + int getMessageIdCount(); + + // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; + boolean hasValidationError(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError(); + + // repeated .pulsar.proto.KeyLongValue properties = 5; + java.util.List + getPropertiesList(); + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index); + int getPropertiesCount(); + + // optional uint64 txnid_least_bits = 6 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 7 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + } + public static final class CommandAck extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAckOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAck.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAck(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAck newObject(Handle handle) { + return new CommandAck(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAck(boolean noInit) {} + + private static final CommandAck defaultInstance; + public static CommandAck getDefaultInstance() { + return defaultInstance; + } + + public CommandAck getDefaultInstanceForType() { + return defaultInstance; + } + + public enum AckType + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + Individual(0, 0), + Cumulative(1, 1), + ; + + public static final int Individual_VALUE = 0; + public static final int Cumulative_VALUE = 1; + + + public final int getNumber() { return value; } + + public static AckType valueOf(int value) { + switch (value) { + case 0: return Individual; + case 1: return Cumulative; + 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 AckType findValueByNumber(int number) { + return AckType.valueOf(number); + } + }; + + private final int value; + + private AckType(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandAck.AckType) + } + + public enum ValidationError + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + UncompressedSizeCorruption(0, 0), + DecompressionError(1, 1), + ChecksumMismatch(2, 2), + BatchDeSerializeError(3, 3), + DecryptionError(4, 4), + ; + + public static final int UncompressedSizeCorruption_VALUE = 0; + public static final int DecompressionError_VALUE = 1; + public static final int ChecksumMismatch_VALUE = 2; + public static final int BatchDeSerializeError_VALUE = 3; + public static final int DecryptionError_VALUE = 4; + + + public final int getNumber() { return value; } + + public static ValidationError valueOf(int value) { + switch (value) { + case 0: return UncompressedSizeCorruption; + case 1: return DecompressionError; + case 2: return ChecksumMismatch; + case 3: return BatchDeSerializeError; + case 4: return DecryptionError; + 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 ValidationError findValueByNumber(int number) { + return ValidationError.valueOf(number); + } + }; + + private final int value; + + private ValidationError(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandAck.ValidationError) + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required .pulsar.proto.CommandAck.AckType ack_type = 2; + public static final int ACK_TYPE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType ackType_; + public boolean hasAckType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType() { + return ackType_; + } + + // repeated .pulsar.proto.MessageIdData message_id = 3; + public static final int MESSAGE_ID_FIELD_NUMBER = 3; + private java.util.List messageId_; + public java.util.List getMessageIdList() { + return messageId_; + } + public java.util.List + getMessageIdOrBuilderList() { + return messageId_; + } + public int getMessageIdCount() { + return messageId_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index) { + return messageId_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder getMessageIdOrBuilder( + int index) { + return messageId_.get(index); + } + + // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; + public static final int VALIDATION_ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError validationError_; + public boolean hasValidationError() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError() { + return validationError_; + } + + // repeated .pulsar.proto.KeyLongValue properties = 5; + public static final int PROPERTIES_FIELD_NUMBER = 5; + private java.util.List properties_; + public java.util.List getPropertiesList() { + return properties_; + } + public java.util.List + getPropertiesOrBuilderList() { + return properties_; + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index) { + return properties_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValueOrBuilder getPropertiesOrBuilder( + int index) { + return properties_.get(index); + } + + // optional uint64 txnid_least_bits = 6 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 6; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 7 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 7; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + private void initFields() { + consumerId_ = 0L; + ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; + messageId_ = java.util.Collections.emptyList(); + validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; + properties_ = java.util.Collections.emptyList(); + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAckType()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getMessageIdCount(); i++) { + if (!getMessageId(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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.writeUInt64(1, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, ackType_.getNumber()); + } + for (int i = 0; i < messageId_.size(); i++) { + output.writeMessage(3, messageId_.get(i)); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(4, validationError_.getNumber()); + } + for (int i = 0; i < properties_.size(); i++) { + output.writeMessage(5, properties_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(6, txnidLeastBits_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeUInt64(7, txnidMostBits_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, ackType_.getNumber()); + } + for (int i = 0; i < messageId_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, messageId_.get(i)); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, validationError_.getNumber()); + } + for (int i = 0; i < properties_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(5, properties_.get(i)); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(6, txnidLeastBits_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(7, txnidMostBits_); + } + 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck 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.PulsarApi.CommandAck, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAckOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; + bitField0_ = (bitField0_ & ~0x00000002); + messageId_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; + bitField0_ = (bitField0_ & ~0x00000008); + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000020); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000040); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.ackType_ = ackType_; + if (((bitField0_ & 0x00000004) == 0x00000004)) { + messageId_ = java.util.Collections.unmodifiableList(messageId_); + bitField0_ = (bitField0_ & ~0x00000004); + } + result.messageId_ = messageId_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000004; + } + result.validationError_ = validationError_; + if (((bitField0_ & 0x00000010) == 0x00000010)) { + properties_ = java.util.Collections.unmodifiableList(properties_); + bitField0_ = (bitField0_ & ~0x00000010); + } + result.properties_ = properties_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000008; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000010; + } + result.txnidMostBits_ = txnidMostBits_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasAckType()) { + setAckType(other.getAckType()); + } + if (!other.messageId_.isEmpty()) { + if (messageId_.isEmpty()) { + messageId_ = other.messageId_; + bitField0_ = (bitField0_ & ~0x00000004); + } else { + ensureMessageIdIsMutable(); + messageId_.addAll(other.messageId_); + } + + } + if (other.hasValidationError()) { + setValidationError(other.getValidationError()); + } + if (!other.properties_.isEmpty()) { + if (properties_.isEmpty()) { + properties_ = other.properties_; + bitField0_ = (bitField0_ & ~0x00000010); + } else { + ensurePropertiesIsMutable(); + properties_.addAll(other.properties_); + } + + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasAckType()) { + + return false; + } + for (int i = 0; i < getMessageIdCount(); i++) { + if (!getMessageId(i).isInitialized()) { + + return false; + } + } + for (int i = 0; i < getPropertiesCount(); i++) { + if (!getProperties(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 8: { + bitField0_ |= 0x00000001; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType value = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + ackType_ = value; + } + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMessageId(subBuilder.buildPartial()); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError value = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + validationError_ = value; + } + break; + } + case 42: { + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addProperties(subBuilder.buildPartial()); + break; + } + case 48: { + bitField0_ |= 0x00000020; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 56: { + bitField0_ |= 0x00000040; + txnidMostBits_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required .pulsar.proto.CommandAck.AckType ack_type = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; + public boolean hasAckType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType getAckType() { + return ackType_; + } + public Builder setAckType(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + ackType_ = value; + + return this; + } + public Builder clearAckType() { + bitField0_ = (bitField0_ & ~0x00000002); + ackType_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType.Individual; + + return this; + } + + // repeated .pulsar.proto.MessageIdData message_id = 3; + private java.util.List messageId_ = + java.util.Collections.emptyList(); + private void ensureMessageIdIsMutable() { + if (!((bitField0_ & 0x00000004) == 0x00000004)) { + messageId_ = new java.util.ArrayList(messageId_); + bitField0_ |= 0x00000004; + } + } + + public java.util.List getMessageIdList() { + return java.util.Collections.unmodifiableList(messageId_); + } + public int getMessageIdCount() { + return messageId_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(int index) { + return messageId_.get(index); + } + public Builder setMessageId( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdIsMutable(); + messageId_.set(index, value); + + return this; + } + public Builder setMessageId( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdIsMutable(); + messageId_.set(index, builderForValue.build()); + + return this; + } + public Builder addMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdIsMutable(); + messageId_.add(value); + + return this; + } + public Builder addMessageId( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdIsMutable(); + messageId_.add(index, value); + + return this; + } + public Builder addMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdIsMutable(); + messageId_.add(builderForValue.build()); + + return this; + } + public Builder addMessageId( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdIsMutable(); + messageId_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllMessageId( + java.lang.Iterable values) { + ensureMessageIdIsMutable(); + super.addAll(values, messageId_); + + return this; + } + public Builder clearMessageId() { + messageId_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000004); + + return this; + } + public Builder removeMessageId(int index) { + ensureMessageIdIsMutable(); + messageId_.remove(index); + + return this; + } + + // optional .pulsar.proto.CommandAck.ValidationError validation_error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; + public boolean hasValidationError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError getValidationError() { + return validationError_; + } + public Builder setValidationError(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + validationError_ = value; + + return this; + } + public Builder clearValidationError() { + bitField0_ = (bitField0_ & ~0x00000008); + validationError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.ValidationError.UncompressedSizeCorruption; + + return this; + } + + // repeated .pulsar.proto.KeyLongValue properties = 5; + private java.util.List properties_ = + java.util.Collections.emptyList(); + private void ensurePropertiesIsMutable() { + if (!((bitField0_ & 0x00000010) == 0x00000010)) { + properties_ = new java.util.ArrayList(properties_); + bitField0_ |= 0x00000010; + } + } + + public java.util.List getPropertiesList() { + return java.util.Collections.unmodifiableList(properties_); + } + public int getPropertiesCount() { + return properties_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue getProperties(int index) { + return properties_.get(index); + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.set(index, value); + + return this; + } + public Builder setProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.set(index, builderForValue.build()); + + return this; + } + public Builder addProperties(org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(value); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePropertiesIsMutable(); + properties_.add(index, value); + + return this; + } + public Builder addProperties( + org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(builderForValue.build()); + + return this; + } + public Builder addProperties( + int index, org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue.Builder builderForValue) { + ensurePropertiesIsMutable(); + properties_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllProperties( + java.lang.Iterable values) { + ensurePropertiesIsMutable(); + super.addAll(values, properties_); + + return this; + } + public Builder clearProperties() { + properties_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000010); + + return this; + } + public Builder removeProperties(int index) { + ensurePropertiesIsMutable(); + properties_.remove(index); + + return this; + } + + // optional uint64 txnid_least_bits = 6 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000020; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000020); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 7 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000040; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000040); + txnidMostBits_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAck) + } + + static { + defaultInstance = new CommandAck(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAck) + } + + public interface CommandAckResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandAckResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAckResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAckResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAckResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAckResponse newObject(Handle handle) { + return new CommandAckResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAckResponse(boolean noInit) {} + + private static final CommandAckResponse defaultInstance; + public static CommandAckResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandAckResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + consumerId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse 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.PulsarApi.CommandAckResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAckResponse) + } + + static { + defaultInstance = new CommandAckResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAckResponse) + } + + public interface CommandActiveConsumerChangeOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // optional bool is_active = 2 [default = false]; + boolean hasIsActive(); + boolean getIsActive(); + } + public static final class CommandActiveConsumerChange extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandActiveConsumerChangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandActiveConsumerChange.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandActiveConsumerChange(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandActiveConsumerChange newObject(Handle handle) { + return new CommandActiveConsumerChange(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandActiveConsumerChange(boolean noInit) {} + + private static final CommandActiveConsumerChange defaultInstance; + public static CommandActiveConsumerChange getDefaultInstance() { + return defaultInstance; + } + + public CommandActiveConsumerChange getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // optional bool is_active = 2 [default = false]; + public static final int IS_ACTIVE_FIELD_NUMBER = 2; + private boolean isActive_; + public boolean hasIsActive() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public boolean getIsActive() { + return isActive_; + } + + private void initFields() { + consumerId_ = 0L; + isActive_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBool(2, isActive_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(2, isActive_); + } + 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange 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.PulsarApi.CommandActiveConsumerChange, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChangeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + isActive_ = false; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange result = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.isActive_ = isActive_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasIsActive()) { + setIsActive(other.getIsActive()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + isActive_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // optional bool is_active = 2 [default = false]; + private boolean isActive_ ; + public boolean hasIsActive() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public boolean getIsActive() { + return isActive_; + } + public Builder setIsActive(boolean value) { + bitField0_ |= 0x00000002; + isActive_ = value; + + return this; + } + public Builder clearIsActive() { + bitField0_ = (bitField0_ & ~0x00000002); + isActive_ = false; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandActiveConsumerChange) + } + + static { + defaultInstance = new CommandActiveConsumerChange(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandActiveConsumerChange) + } + + public interface CommandFlowOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint32 messagePermits = 2; + boolean hasMessagePermits(); + int getMessagePermits(); + } + public static final class CommandFlow extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandFlowOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandFlow.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandFlow(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandFlow newObject(Handle handle) { + return new CommandFlow(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandFlow(boolean noInit) {} + + private static final CommandFlow defaultInstance; + public static CommandFlow getDefaultInstance() { + return defaultInstance; + } + + public CommandFlow getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint32 messagePermits = 2; + public static final int MESSAGEPERMITS_FIELD_NUMBER = 2; + private int messagePermits_; + public boolean hasMessagePermits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getMessagePermits() { + return messagePermits_; + } + + private void initFields() { + consumerId_ = 0L; + messagePermits_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMessagePermits()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt32(2, messagePermits_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt32Size(2, messagePermits_); + } + 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow 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.PulsarApi.CommandFlow, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandFlowOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + messagePermits_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow result = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.messagePermits_ = messagePermits_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasMessagePermits()) { + setMessagePermits(other.getMessagePermits()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasMessagePermits()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + messagePermits_ = input.readUInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required uint32 messagePermits = 2; + private int messagePermits_ ; + public boolean hasMessagePermits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getMessagePermits() { + return messagePermits_; + } + public Builder setMessagePermits(int value) { + bitField0_ |= 0x00000002; + messagePermits_ = value; + + return this; + } + public Builder clearMessagePermits() { + bitField0_ = (bitField0_ & ~0x00000002); + messagePermits_ = 0; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandFlow) + } + + static { + defaultInstance = new CommandFlow(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandFlow) + } + + public interface CommandUnsubscribeOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + } + public static final class CommandUnsubscribe extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandUnsubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandUnsubscribe.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandUnsubscribe(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandUnsubscribe newObject(Handle handle) { + return new CommandUnsubscribe(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandUnsubscribe(boolean noInit) {} + + private static final CommandUnsubscribe defaultInstance; + public static CommandUnsubscribe getDefaultInstance() { + return defaultInstance; + } + + public CommandUnsubscribe getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + private void initFields() { + consumerId_ = 0L; + requestId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe 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.PulsarApi.CommandUnsubscribe, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribeOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe result = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasRequestId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandUnsubscribe) + } + + static { + defaultInstance = new CommandUnsubscribe(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandUnsubscribe) + } + + public interface CommandSeekOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.MessageIdData message_id = 3; + boolean hasMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId(); + + // optional uint64 message_publish_time = 4; + boolean hasMessagePublishTime(); + long getMessagePublishTime(); + } + public static final class CommandSeek extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSeekOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSeek.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSeek(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSeek newObject(Handle handle) { + return new CommandSeek(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSeek(boolean noInit) {} + + private static final CommandSeek defaultInstance; + public static CommandSeek getDefaultInstance() { + return defaultInstance; + } + + public CommandSeek getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.MessageIdData message_id = 3; + public static final int MESSAGE_ID_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_; + public boolean hasMessageId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + + // optional uint64 message_publish_time = 4; + public static final int MESSAGE_PUBLISH_TIME_FIELD_NUMBER = 4; + private long messagePublishTime_; + public boolean hasMessagePublishTime() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getMessagePublishTime() { + return messagePublishTime_; + } + + private void initFields() { + consumerId_ = 0L; + requestId_ = 0L; + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + messagePublishTime_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasMessageId()) { + 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.writeUInt64(1, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, messageId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt64(4, messagePublishTime_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, messageId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, messagePublishTime_); + } + 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek 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.PulsarApi.CommandSeek, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSeekOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000004); + messagePublishTime_ = 0L; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.messageId_ = messageId_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.messagePublishTime_ = messagePublishTime_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasMessageId()) { + mergeMessageId(other.getMessageId()); + } + if (other.hasMessagePublishTime()) { + setMessagePublishTime(other.getMessagePublishTime()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasRequestId()) { + + return false; + } + if (hasMessageId()) { + 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 8: { + bitField0_ |= 0x00000001; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasMessageId()) { + subBuilder.mergeFrom(getMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + messagePublishTime_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.MessageIdData message_id = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasMessageId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageId() { + return messageId_; + } + public Builder setMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + messageId_ = value; + + bitField0_ |= 0x00000004; + return this; + } + public Builder setMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + messageId_ = builderForValue.build(); + + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + messageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + messageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(messageId_).mergeFrom(value).buildPartial(); + } else { + messageId_ = value; + } + + bitField0_ |= 0x00000004; + return this; + } + public Builder clearMessageId() { + messageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + // optional uint64 message_publish_time = 4; + private long messagePublishTime_ ; + public boolean hasMessagePublishTime() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public long getMessagePublishTime() { + return messagePublishTime_; + } + public Builder setMessagePublishTime(long value) { + bitField0_ |= 0x00000008; + messagePublishTime_ = value; + + return this; + } + public Builder clearMessagePublishTime() { + bitField0_ = (bitField0_ & ~0x00000008); + messagePublishTime_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSeek) + } + + static { + defaultInstance = new CommandSeek(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSeek) + } + + public interface CommandReachedEndOfTopicOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + } + public static final class CommandReachedEndOfTopic extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandReachedEndOfTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandReachedEndOfTopic.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandReachedEndOfTopic(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandReachedEndOfTopic newObject(Handle handle) { + return new CommandReachedEndOfTopic(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandReachedEndOfTopic(boolean noInit) {} + + private static final CommandReachedEndOfTopic defaultInstance; + public static CommandReachedEndOfTopic getDefaultInstance() { + return defaultInstance; + } + + public CommandReachedEndOfTopic getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + private void initFields() { + consumerId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + 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, consumerId_); + } + } + + 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, consumerId_); + } + 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic 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.PulsarApi.CommandReachedEndOfTopic, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopicOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic result = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandReachedEndOfTopic) + } + + static { + defaultInstance = new CommandReachedEndOfTopic(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandReachedEndOfTopic) + } + + public interface CommandCloseProducerOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 producer_id = 1; + boolean hasProducerId(); + long getProducerId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + } + public static final class CommandCloseProducer extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandCloseProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandCloseProducer.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandCloseProducer(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandCloseProducer newObject(Handle handle) { + return new CommandCloseProducer(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandCloseProducer(boolean noInit) {} + + private static final CommandCloseProducer defaultInstance; + public static CommandCloseProducer getDefaultInstance() { + return defaultInstance; + } + + public CommandCloseProducer getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 producer_id = 1; + public static final int PRODUCER_ID_FIELD_NUMBER = 1; + private long producerId_; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + private void initFields() { + producerId_ = 0L; + requestId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasProducerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + } + + 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, producerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer 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.PulsarApi.CommandCloseProducer, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.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(); + producerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.producerId_ = producerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance()) return this; + if (other.hasProducerId()) { + setProducerId(other.getProducerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasProducerId()) { + + return false; + } + if (!hasRequestId()) { + + 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; + producerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 producer_id = 1; + private long producerId_ ; + public boolean hasProducerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getProducerId() { + return producerId_; + } + public Builder setProducerId(long value) { + bitField0_ |= 0x00000001; + producerId_ = value; + + return this; + } + public Builder clearProducerId() { + bitField0_ = (bitField0_ & ~0x00000001); + producerId_ = 0L; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandCloseProducer) + } + + static { + defaultInstance = new CommandCloseProducer(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandCloseProducer) + } + + public interface CommandCloseConsumerOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + } + public static final class CommandCloseConsumer extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandCloseConsumerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandCloseConsumer.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandCloseConsumer(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandCloseConsumer newObject(Handle handle) { + return new CommandCloseConsumer(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandCloseConsumer(boolean noInit) {} + + private static final CommandCloseConsumer defaultInstance; + public static CommandCloseConsumer getDefaultInstance() { + return defaultInstance; + } + + public CommandCloseConsumer getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + private void initFields() { + consumerId_ = 0L; + requestId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer 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.PulsarApi.CommandCloseConsumer, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumerOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer result = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasRequestId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandCloseConsumer) + } + + static { + defaultInstance = new CommandCloseConsumer(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandCloseConsumer) + } + + public interface CommandRedeliverUnacknowledgedMessagesOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // repeated .pulsar.proto.MessageIdData message_ids = 2; + java.util.List + getMessageIdsList(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index); + int getMessageIdsCount(); + } + public static final class CommandRedeliverUnacknowledgedMessages extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandRedeliverUnacknowledgedMessagesOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandRedeliverUnacknowledgedMessages.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandRedeliverUnacknowledgedMessages(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandRedeliverUnacknowledgedMessages newObject(Handle handle) { + return new CommandRedeliverUnacknowledgedMessages(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandRedeliverUnacknowledgedMessages(boolean noInit) {} + + private static final CommandRedeliverUnacknowledgedMessages defaultInstance; + public static CommandRedeliverUnacknowledgedMessages getDefaultInstance() { + return defaultInstance; + } + + public CommandRedeliverUnacknowledgedMessages getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // repeated .pulsar.proto.MessageIdData message_ids = 2; + public static final int MESSAGE_IDS_FIELD_NUMBER = 2; + private java.util.List messageIds_; + public java.util.List getMessageIdsList() { + return messageIds_; + } + public java.util.List + getMessageIdsOrBuilderList() { + return messageIds_; + } + public int getMessageIdsCount() { + return messageIds_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index) { + return messageIds_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdDataOrBuilder getMessageIdsOrBuilder( + int index) { + return messageIds_.get(index); + } + + private void initFields() { + consumerId_ = 0L; + messageIds_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getMessageIdsCount(); i++) { + if (!getMessageIds(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.writeUInt64(1, consumerId_); + } + for (int i = 0; i < messageIds_.size(); i++) { + output.writeMessage(2, messageIds_.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 + .computeUInt64Size(1, consumerId_); + } + for (int i = 0; i < messageIds_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, messageIds_.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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages 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.PulsarApi.CommandRedeliverUnacknowledgedMessages, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessagesOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + messageIds_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages result = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + messageIds_ = java.util.Collections.unmodifiableList(messageIds_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.messageIds_ = messageIds_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (!other.messageIds_.isEmpty()) { + if (messageIds_.isEmpty()) { + messageIds_ = other.messageIds_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureMessageIdsIsMutable(); + messageIds_.addAll(other.messageIds_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + for (int i = 0; i < getMessageIdsCount(); i++) { + if (!getMessageIds(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 8: { + bitField0_ |= 0x00000001; + consumerId_ = input.readUInt64(); + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addMessageIds(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // repeated .pulsar.proto.MessageIdData message_ids = 2; + private java.util.List messageIds_ = + java.util.Collections.emptyList(); + private void ensureMessageIdsIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + messageIds_ = new java.util.ArrayList(messageIds_); + bitField0_ |= 0x00000002; + } + } + + public java.util.List getMessageIdsList() { + return java.util.Collections.unmodifiableList(messageIds_); + } + public int getMessageIdsCount() { + return messageIds_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getMessageIds(int index) { + return messageIds_.get(index); + } + public Builder setMessageIds( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdsIsMutable(); + messageIds_.set(index, value); + + return this; + } + public Builder setMessageIds( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdsIsMutable(); + messageIds_.set(index, builderForValue.build()); + + return this; + } + public Builder addMessageIds(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdsIsMutable(); + messageIds_.add(value); + + return this; + } + public Builder addMessageIds( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessageIdsIsMutable(); + messageIds_.add(index, value); + + return this; + } + public Builder addMessageIds( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdsIsMutable(); + messageIds_.add(builderForValue.build()); + + return this; + } + public Builder addMessageIds( + int index, org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + ensureMessageIdsIsMutable(); + messageIds_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllMessageIds( + java.lang.Iterable values) { + ensureMessageIdsIsMutable(); + super.addAll(values, messageIds_); + + return this; + } + public Builder clearMessageIds() { + messageIds_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000002); + + return this; + } + public Builder removeMessageIds(int index) { + ensureMessageIdsIsMutable(); + messageIds_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandRedeliverUnacknowledgedMessages) + } + + static { + defaultInstance = new CommandRedeliverUnacknowledgedMessages(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandRedeliverUnacknowledgedMessages) + } + + public interface CommandSuccessOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.Schema schema = 2; + boolean hasSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); + } + public static final class CommandSuccess extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandSuccess.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandSuccess(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandSuccess newObject(Handle handle) { + return new CommandSuccess(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandSuccess(boolean noInit) {} + + private static final CommandSuccess defaultInstance; + public static CommandSuccess getDefaultInstance() { + return defaultInstance; + } + + public CommandSuccess getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.Schema schema = 2; + public static final int SCHEMA_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; + public boolean hasSchema() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + + private void initFields() { + requestId_ = 0L; + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasSchema()) { + if (!getSchema().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.writeUInt64(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, schema_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, schema_); + } + 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess 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.PulsarApi.CommandSuccess, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess result = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.schema_ = schema_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasSchema()) { + mergeSchema(other.getSchema()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (hasSchema()) { + if (!getSchema().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 8: { + bitField0_ |= 0x00000001; + requestId_ = input.readUInt64(); + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); + if (hasSchema()) { + subBuilder.mergeFrom(getSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.Schema schema = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + public boolean hasSchema() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (value == null) { + throw new NullPointerException(); + } + schema_ = value; + + bitField0_ |= 0x00000002; + return this; + } + public Builder setSchema( + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { + schema_ = builderForValue.build(); + + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { + schema_ = + org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); + } else { + schema_ = value; + } + + bitField0_ |= 0x00000002; + return this; + } + public Builder clearSchema() { + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandSuccess) + } + + static { + defaultInstance = new CommandSuccess(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandSuccess) + } + + public interface CommandProducerSuccessOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required string producer_name = 2; + boolean hasProducerName(); + String getProducerName(); + + // optional int64 last_sequence_id = 3 [default = -1]; + boolean hasLastSequenceId(); + long getLastSequenceId(); + + // optional bytes schema_version = 4; + boolean hasSchemaVersion(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); + } + public static final class CommandProducerSuccess extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandProducerSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandProducerSuccess.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandProducerSuccess(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandProducerSuccess newObject(Handle handle) { + return new CommandProducerSuccess(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandProducerSuccess(boolean noInit) {} + + private static final CommandProducerSuccess defaultInstance; + public static CommandProducerSuccess getDefaultInstance() { + return defaultInstance; + } + + public CommandProducerSuccess getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required string producer_name = 2; + public static final int PRODUCER_NAME_FIELD_NUMBER = 2; + private java.lang.Object producerName_; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + 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)) { + producerName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getProducerNameBytes() { + java.lang.Object ref = producerName_; + 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); + producerName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional int64 last_sequence_id = 3 [default = -1]; + public static final int LAST_SEQUENCE_ID_FIELD_NUMBER = 3; + private long lastSequenceId_; + public boolean hasLastSequenceId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getLastSequenceId() { + return lastSequenceId_; + } + + // optional bytes schema_version = 4; + public static final int SCHEMA_VERSION_FIELD_NUMBER = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + + private void initFields() { + requestId_ = 0L; + producerName_ = ""; + lastSequenceId_ = -1L; + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasProducerName()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeInt64(3, lastSequenceId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, schemaVersion_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getProducerNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeInt64Size(3, lastSequenceId_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, schemaVersion_); + } + 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess 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.PulsarApi.CommandProducerSuccess, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccessOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + producerName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + lastSequenceId_ = -1L; + bitField0_ = (bitField0_ & ~0x00000004); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess result = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.producerName_ = producerName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.lastSequenceId_ = lastSequenceId_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.schemaVersion_ = schemaVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasProducerName()) { + setProducerName(other.getProducerName()); + } + if (other.hasLastSequenceId()) { + setLastSequenceId(other.getLastSequenceId()); + } + if (other.hasSchemaVersion()) { + setSchemaVersion(other.getSchemaVersion()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasProducerName()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + producerName_ = input.readBytes(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + lastSequenceId_ = input.readInt64(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + schemaVersion_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required string producer_name = 2; + private java.lang.Object producerName_ = ""; + public boolean hasProducerName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getProducerName() { + java.lang.Object ref = producerName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + producerName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setProducerName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + producerName_ = value; + + return this; + } + public Builder clearProducerName() { + bitField0_ = (bitField0_ & ~0x00000002); + producerName_ = getDefaultInstance().getProducerName(); + + return this; + } + void setProducerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + producerName_ = value; + + } + + // optional int64 last_sequence_id = 3 [default = -1]; + private long lastSequenceId_ = -1L; + public boolean hasLastSequenceId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getLastSequenceId() { + return lastSequenceId_; + } + public Builder setLastSequenceId(long value) { + bitField0_ |= 0x00000004; + lastSequenceId_ = value; + + return this; + } + public Builder clearLastSequenceId() { + bitField0_ = (bitField0_ & ~0x00000004); + lastSequenceId_ = -1L; + + return this; + } + + // optional bytes schema_version = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + schemaVersion_ = value; + + return this; + } + public Builder clearSchemaVersion() { + bitField0_ = (bitField0_ & ~0x00000008); + schemaVersion_ = getDefaultInstance().getSchemaVersion(); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandProducerSuccess) + } + + static { + defaultInstance = new CommandProducerSuccess(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandProducerSuccess) + } + + public interface CommandErrorOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required .pulsar.proto.ServerError error = 2; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // required string message = 3; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandError extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandError.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandError(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandError newObject(Handle handle) { + return new CommandError(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandError(boolean noInit) {} + + private static final CommandError defaultInstance; + public static CommandError getDefaultInstance() { + return defaultInstance; + } + + public CommandError getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required .pulsar.proto.ServerError error = 2; + public static final int ERROR_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // required string message = 3; + public static final int MESSAGE_FIELD_NUMBER = 3; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasError()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMessage()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, error_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, error_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, getMessageBytes()); + } + 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError 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.PulsarApi.CommandError, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandErrorOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandError.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000002); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandError build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandError buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandError buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandError result = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandError other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasError()) { + + return false; + } + if (!hasMessage()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + error_ = value; + } + break; + } + case 26: { + bitField0_ |= 0x00000004; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required .pulsar.proto.ServerError error = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000002); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // required string message = 3; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000004); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandError) + } + + static { + defaultInstance = new CommandError(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandError) + } + + public interface CommandPingOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + } + public static final class CommandPing extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandPingOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandPing.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandPing(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandPing newObject(Handle handle) { + return new CommandPing(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandPing(boolean noInit) {} + + private static final CommandPing defaultInstance; + public static CommandPing getDefaultInstance() { + return defaultInstance; + } + + public CommandPing getDefaultInstanceForType() { + return defaultInstance; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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(); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing 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.PulsarApi.CommandPing, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPingOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.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(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.RECYCLER.get(); + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance()) return this; + return this; + } + + public final boolean isInitialized() { + 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; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPing) + } + + static { + defaultInstance = new CommandPing(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPing) + } + + public interface CommandPongOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + } + public static final class CommandPong extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandPongOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandPong.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandPong(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandPong newObject(Handle handle) { + return new CommandPong(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandPong(boolean noInit) {} + + private static final CommandPong defaultInstance; + public static CommandPong getDefaultInstance() { + return defaultInstance; + } + + public CommandPong getDefaultInstanceForType() { + return defaultInstance; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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(); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong 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.PulsarApi.CommandPong, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandPongOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.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(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong result = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.RECYCLER.get(); + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance()) return this; + return this; + } + + public final boolean isInitialized() { + 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; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandPong) + } + + static { + defaultInstance = new CommandPong(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandPong) + } + + public interface CommandConsumerStatsOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required uint64 consumer_id = 4; + boolean hasConsumerId(); + long getConsumerId(); + } + public static final class CommandConsumerStats extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandConsumerStatsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandConsumerStats.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandConsumerStats(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandConsumerStats newObject(Handle handle) { + return new CommandConsumerStats(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandConsumerStats(boolean noInit) {} + + private static final CommandConsumerStats defaultInstance; + public static CommandConsumerStats getDefaultInstance() { + return defaultInstance; + } + + public CommandConsumerStats getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required uint64 consumer_id = 4; + public static final int CONSUMER_ID_FIELD_NUMBER = 4; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getConsumerId() { + return consumerId_; + } + + private void initFields() { + requestId_ = 0L; + consumerId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasConsumerId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(4, consumerId_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(4, consumerId_); + } + 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats 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.PulsarApi.CommandConsumerStats, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.consumerId_ = consumerId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasConsumerId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000002; + consumerId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required uint64 consumer_id = 4; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000002; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000002); + consumerId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStats) + } + + static { + defaultInstance = new CommandConsumerStats(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStats) + } + + public interface CommandConsumerStatsResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.ServerError error_code = 2; + boolean hasErrorCode(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); + + // optional string error_message = 3; + boolean hasErrorMessage(); + String getErrorMessage(); + + // optional double msgRateOut = 4; + boolean hasMsgRateOut(); + double getMsgRateOut(); + + // optional double msgThroughputOut = 5; + boolean hasMsgThroughputOut(); + double getMsgThroughputOut(); + + // optional double msgRateRedeliver = 6; + boolean hasMsgRateRedeliver(); + double getMsgRateRedeliver(); + + // optional string consumerName = 7; + boolean hasConsumerName(); + String getConsumerName(); + + // optional uint64 availablePermits = 8; + boolean hasAvailablePermits(); + long getAvailablePermits(); + + // optional uint64 unackedMessages = 9; + boolean hasUnackedMessages(); + long getUnackedMessages(); + + // optional bool blockedConsumerOnUnackedMsgs = 10; + boolean hasBlockedConsumerOnUnackedMsgs(); + boolean getBlockedConsumerOnUnackedMsgs(); + + // optional string address = 11; + boolean hasAddress(); + String getAddress(); + + // optional string connectedSince = 12; + boolean hasConnectedSince(); + String getConnectedSince(); + + // optional string type = 13; + boolean hasType(); + String getType(); + + // optional double msgRateExpired = 14; + boolean hasMsgRateExpired(); + double getMsgRateExpired(); + + // optional uint64 msgBacklog = 15; + boolean hasMsgBacklog(); + long getMsgBacklog(); + } + public static final class CommandConsumerStatsResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandConsumerStatsResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandConsumerStatsResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandConsumerStatsResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandConsumerStatsResponse newObject(Handle handle) { + return new CommandConsumerStatsResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandConsumerStatsResponse(boolean noInit) {} + + private static final CommandConsumerStatsResponse defaultInstance; + public static CommandConsumerStatsResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandConsumerStatsResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.ServerError error_code = 2; + public static final int ERROR_CODE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + + // optional string error_message = 3; + public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; + private java.lang.Object errorMessage_; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + 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)) { + errorMessage_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { + java.lang.Object ref = errorMessage_; + 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); + errorMessage_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional double msgRateOut = 4; + public static final int MSGRATEOUT_FIELD_NUMBER = 4; + private double msgRateOut_; + public boolean hasMsgRateOut() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public double getMsgRateOut() { + return msgRateOut_; + } + + // optional double msgThroughputOut = 5; + public static final int MSGTHROUGHPUTOUT_FIELD_NUMBER = 5; + private double msgThroughputOut_; + public boolean hasMsgThroughputOut() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public double getMsgThroughputOut() { + return msgThroughputOut_; + } + + // optional double msgRateRedeliver = 6; + public static final int MSGRATEREDELIVER_FIELD_NUMBER = 6; + private double msgRateRedeliver_; + public boolean hasMsgRateRedeliver() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public double getMsgRateRedeliver() { + return msgRateRedeliver_; + } + + // optional string consumerName = 7; + public static final int CONSUMERNAME_FIELD_NUMBER = 7; + private java.lang.Object consumerName_; + public boolean hasConsumerName() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getConsumerName() { + java.lang.Object ref = consumerName_; + 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)) { + consumerName_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConsumerNameBytes() { + java.lang.Object ref = consumerName_; + 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); + consumerName_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional uint64 availablePermits = 8; + public static final int AVAILABLEPERMITS_FIELD_NUMBER = 8; + private long availablePermits_; + public boolean hasAvailablePermits() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public long getAvailablePermits() { + return availablePermits_; + } + + // optional uint64 unackedMessages = 9; + public static final int UNACKEDMESSAGES_FIELD_NUMBER = 9; + private long unackedMessages_; + public boolean hasUnackedMessages() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public long getUnackedMessages() { + return unackedMessages_; + } + + // optional bool blockedConsumerOnUnackedMsgs = 10; + public static final int BLOCKEDCONSUMERONUNACKEDMSGS_FIELD_NUMBER = 10; + private boolean blockedConsumerOnUnackedMsgs_; + public boolean hasBlockedConsumerOnUnackedMsgs() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public boolean getBlockedConsumerOnUnackedMsgs() { + return blockedConsumerOnUnackedMsgs_; + } + + // optional string address = 11; + public static final int ADDRESS_FIELD_NUMBER = 11; + private java.lang.Object address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public String getAddress() { + java.lang.Object ref = address_; + 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)) { + address_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getAddressBytes() { + java.lang.Object ref = address_; + 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); + address_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string connectedSince = 12; + public static final int CONNECTEDSINCE_FIELD_NUMBER = 12; + private java.lang.Object connectedSince_; + public boolean hasConnectedSince() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public String getConnectedSince() { + java.lang.Object ref = connectedSince_; + 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)) { + connectedSince_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getConnectedSinceBytes() { + java.lang.Object ref = connectedSince_; + 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); + connectedSince_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional string type = 13; + public static final int TYPE_FIELD_NUMBER = 13; + private java.lang.Object type_; + public boolean hasType() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public String getType() { + java.lang.Object ref = type_; + 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)) { + type_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTypeBytes() { + java.lang.Object ref = type_; + 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); + type_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional double msgRateExpired = 14; + public static final int MSGRATEEXPIRED_FIELD_NUMBER = 14; + private double msgRateExpired_; + public boolean hasMsgRateExpired() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public double getMsgRateExpired() { + return msgRateExpired_; + } + + // optional uint64 msgBacklog = 15; + public static final int MSGBACKLOG_FIELD_NUMBER = 15; + private long msgBacklog_; + public boolean hasMsgBacklog() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public long getMsgBacklog() { + return msgBacklog_; + } + + private void initFields() { + requestId_ = 0L; + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + errorMessage_ = ""; + msgRateOut_ = 0D; + msgThroughputOut_ = 0D; + msgRateRedeliver_ = 0D; + consumerName_ = ""; + availablePermits_ = 0L; + unackedMessages_ = 0L; + blockedConsumerOnUnackedMsgs_ = false; + address_ = ""; + connectedSince_ = ""; + type_ = ""; + msgRateExpired_ = 0D; + msgBacklog_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeDouble(4, msgRateOut_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeDouble(5, msgThroughputOut_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeDouble(6, msgRateRedeliver_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeBytes(7, getConsumerNameBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeUInt64(8, availablePermits_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeUInt64(9, unackedMessages_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBool(10, blockedConsumerOnUnackedMsgs_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeBytes(11, getAddressBytes()); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeBytes(12, getConnectedSinceBytes()); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeBytes(13, getTypeBytes()); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeDouble(14, msgRateExpired_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + output.writeUInt64(15, msgBacklog_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeDoubleSize(4, msgRateOut_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeDoubleSize(5, msgThroughputOut_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeDoubleSize(6, msgRateRedeliver_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(7, getConsumerNameBytes()); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(8, availablePermits_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(9, unackedMessages_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBoolSize(10, blockedConsumerOnUnackedMsgs_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(11, getAddressBytes()); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(12, getConnectedSinceBytes()); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(13, getTypeBytes()); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeDoubleSize(14, msgRateExpired_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(15, msgBacklog_); + } + 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse 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.PulsarApi.CommandConsumerStatsResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000002); + errorMessage_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + msgRateOut_ = 0D; + bitField0_ = (bitField0_ & ~0x00000008); + msgThroughputOut_ = 0D; + bitField0_ = (bitField0_ & ~0x00000010); + msgRateRedeliver_ = 0D; + bitField0_ = (bitField0_ & ~0x00000020); + consumerName_ = ""; + bitField0_ = (bitField0_ & ~0x00000040); + availablePermits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000080); + unackedMessages_ = 0L; + bitField0_ = (bitField0_ & ~0x00000100); + blockedConsumerOnUnackedMsgs_ = false; + bitField0_ = (bitField0_ & ~0x00000200); + address_ = ""; + bitField0_ = (bitField0_ & ~0x00000400); + connectedSince_ = ""; + bitField0_ = (bitField0_ & ~0x00000800); + type_ = ""; + bitField0_ = (bitField0_ & ~0x00001000); + msgRateExpired_ = 0D; + bitField0_ = (bitField0_ & ~0x00002000); + msgBacklog_ = 0L; + bitField0_ = (bitField0_ & ~0x00004000); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.errorCode_ = errorCode_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.errorMessage_ = errorMessage_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.msgRateOut_ = msgRateOut_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.msgThroughputOut_ = msgThroughputOut_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.msgRateRedeliver_ = msgRateRedeliver_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.consumerName_ = consumerName_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.availablePermits_ = availablePermits_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.unackedMessages_ = unackedMessages_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.blockedConsumerOnUnackedMsgs_ = blockedConsumerOnUnackedMsgs_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000400; + } + result.address_ = address_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000800; + } + result.connectedSince_ = connectedSince_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00001000; + } + result.type_ = type_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00002000; + } + result.msgRateExpired_ = msgRateExpired_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00004000; + } + result.msgBacklog_ = msgBacklog_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasErrorCode()) { + setErrorCode(other.getErrorCode()); + } + if (other.hasErrorMessage()) { + setErrorMessage(other.getErrorMessage()); + } + if (other.hasMsgRateOut()) { + setMsgRateOut(other.getMsgRateOut()); + } + if (other.hasMsgThroughputOut()) { + setMsgThroughputOut(other.getMsgThroughputOut()); + } + if (other.hasMsgRateRedeliver()) { + setMsgRateRedeliver(other.getMsgRateRedeliver()); + } + if (other.hasConsumerName()) { + setConsumerName(other.getConsumerName()); + } + if (other.hasAvailablePermits()) { + setAvailablePermits(other.getAvailablePermits()); + } + if (other.hasUnackedMessages()) { + setUnackedMessages(other.getUnackedMessages()); + } + if (other.hasBlockedConsumerOnUnackedMsgs()) { + setBlockedConsumerOnUnackedMsgs(other.getBlockedConsumerOnUnackedMsgs()); + } + if (other.hasAddress()) { + setAddress(other.getAddress()); + } + if (other.hasConnectedSince()) { + setConnectedSince(other.getConnectedSince()); + } + if (other.hasType()) { + setType(other.getType()); + } + if (other.hasMsgRateExpired()) { + setMsgRateExpired(other.getMsgRateExpired()); + } + if (other.hasMsgBacklog()) { + setMsgBacklog(other.getMsgBacklog()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + errorCode_ = value; + } + break; + } + case 26: { + bitField0_ |= 0x00000004; + errorMessage_ = input.readBytes(); + break; + } + case 33: { + bitField0_ |= 0x00000008; + msgRateOut_ = input.readDouble(); + break; + } + case 41: { + bitField0_ |= 0x00000010; + msgThroughputOut_ = input.readDouble(); + break; + } + case 49: { + bitField0_ |= 0x00000020; + msgRateRedeliver_ = input.readDouble(); + break; + } + case 58: { + bitField0_ |= 0x00000040; + consumerName_ = input.readBytes(); + break; + } + case 64: { + bitField0_ |= 0x00000080; + availablePermits_ = input.readUInt64(); + break; + } + case 72: { + bitField0_ |= 0x00000100; + unackedMessages_ = input.readUInt64(); + break; + } + case 80: { + bitField0_ |= 0x00000200; + blockedConsumerOnUnackedMsgs_ = input.readBool(); + break; + } + case 90: { + bitField0_ |= 0x00000400; + address_ = input.readBytes(); + break; + } + case 98: { + bitField0_ |= 0x00000800; + connectedSince_ = input.readBytes(); + break; + } + case 106: { + bitField0_ |= 0x00001000; + type_ = input.readBytes(); + break; + } + case 113: { + bitField0_ |= 0x00002000; + msgRateExpired_ = input.readDouble(); + break; + } + case 120: { + bitField0_ |= 0x00004000; + msgBacklog_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error_code = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + errorCode_ = value; + + return this; + } + public Builder clearErrorCode() { + bitField0_ = (bitField0_ & ~0x00000002); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string error_message = 3; + private java.lang.Object errorMessage_ = ""; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + errorMessage_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setErrorMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + errorMessage_ = value; + + return this; + } + public Builder clearErrorMessage() { + bitField0_ = (bitField0_ & ~0x00000004); + errorMessage_ = getDefaultInstance().getErrorMessage(); + + return this; + } + void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + errorMessage_ = value; + + } + + // optional double msgRateOut = 4; + private double msgRateOut_ ; + public boolean hasMsgRateOut() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public double getMsgRateOut() { + return msgRateOut_; + } + public Builder setMsgRateOut(double value) { + bitField0_ |= 0x00000008; + msgRateOut_ = value; + + return this; + } + public Builder clearMsgRateOut() { + bitField0_ = (bitField0_ & ~0x00000008); + msgRateOut_ = 0D; + + return this; + } + + // optional double msgThroughputOut = 5; + private double msgThroughputOut_ ; + public boolean hasMsgThroughputOut() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public double getMsgThroughputOut() { + return msgThroughputOut_; + } + public Builder setMsgThroughputOut(double value) { + bitField0_ |= 0x00000010; + msgThroughputOut_ = value; + + return this; + } + public Builder clearMsgThroughputOut() { + bitField0_ = (bitField0_ & ~0x00000010); + msgThroughputOut_ = 0D; + + return this; + } + + // optional double msgRateRedeliver = 6; + private double msgRateRedeliver_ ; + public boolean hasMsgRateRedeliver() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public double getMsgRateRedeliver() { + return msgRateRedeliver_; + } + public Builder setMsgRateRedeliver(double value) { + bitField0_ |= 0x00000020; + msgRateRedeliver_ = value; + + return this; + } + public Builder clearMsgRateRedeliver() { + bitField0_ = (bitField0_ & ~0x00000020); + msgRateRedeliver_ = 0D; + + return this; + } + + // optional string consumerName = 7; + private java.lang.Object consumerName_ = ""; + public boolean hasConsumerName() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public String getConsumerName() { + java.lang.Object ref = consumerName_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + consumerName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConsumerName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000040; + consumerName_ = value; + + return this; + } + public Builder clearConsumerName() { + bitField0_ = (bitField0_ & ~0x00000040); + consumerName_ = getDefaultInstance().getConsumerName(); + + return this; + } + void setConsumerName(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000040; + consumerName_ = value; + + } + + // optional uint64 availablePermits = 8; + private long availablePermits_ ; + public boolean hasAvailablePermits() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public long getAvailablePermits() { + return availablePermits_; + } + public Builder setAvailablePermits(long value) { + bitField0_ |= 0x00000080; + availablePermits_ = value; + + return this; + } + public Builder clearAvailablePermits() { + bitField0_ = (bitField0_ & ~0x00000080); + availablePermits_ = 0L; + + return this; + } + + // optional uint64 unackedMessages = 9; + private long unackedMessages_ ; + public boolean hasUnackedMessages() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public long getUnackedMessages() { + return unackedMessages_; + } + public Builder setUnackedMessages(long value) { + bitField0_ |= 0x00000100; + unackedMessages_ = value; + + return this; + } + public Builder clearUnackedMessages() { + bitField0_ = (bitField0_ & ~0x00000100); + unackedMessages_ = 0L; + + return this; + } + + // optional bool blockedConsumerOnUnackedMsgs = 10; + private boolean blockedConsumerOnUnackedMsgs_ ; + public boolean hasBlockedConsumerOnUnackedMsgs() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public boolean getBlockedConsumerOnUnackedMsgs() { + return blockedConsumerOnUnackedMsgs_; + } + public Builder setBlockedConsumerOnUnackedMsgs(boolean value) { + bitField0_ |= 0x00000200; + blockedConsumerOnUnackedMsgs_ = value; + + return this; + } + public Builder clearBlockedConsumerOnUnackedMsgs() { + bitField0_ = (bitField0_ & ~0x00000200); + blockedConsumerOnUnackedMsgs_ = false; + + return this; + } + + // optional string address = 11; + private java.lang.Object address_ = ""; + public boolean hasAddress() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public String getAddress() { + java.lang.Object ref = address_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + address_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setAddress(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000400; + address_ = value; + + return this; + } + public Builder clearAddress() { + bitField0_ = (bitField0_ & ~0x00000400); + address_ = getDefaultInstance().getAddress(); + + return this; + } + void setAddress(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000400; + address_ = value; + + } + + // optional string connectedSince = 12; + private java.lang.Object connectedSince_ = ""; + public boolean hasConnectedSince() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public String getConnectedSince() { + java.lang.Object ref = connectedSince_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + connectedSince_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConnectedSince(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000800; + connectedSince_ = value; + + return this; + } + public Builder clearConnectedSince() { + bitField0_ = (bitField0_ & ~0x00000800); + connectedSince_ = getDefaultInstance().getConnectedSince(); + + return this; + } + void setConnectedSince(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000800; + connectedSince_ = value; + + } + + // optional string type = 13; + private java.lang.Object type_ = ""; + public boolean hasType() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public String getType() { + java.lang.Object ref = type_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + type_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setType(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00001000; + type_ = value; + + return this; + } + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00001000); + type_ = getDefaultInstance().getType(); + + return this; + } + void setType(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00001000; + type_ = value; + + } + + // optional double msgRateExpired = 14; + private double msgRateExpired_ ; + public boolean hasMsgRateExpired() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public double getMsgRateExpired() { + return msgRateExpired_; + } + public Builder setMsgRateExpired(double value) { + bitField0_ |= 0x00002000; + msgRateExpired_ = value; + + return this; + } + public Builder clearMsgRateExpired() { + bitField0_ = (bitField0_ & ~0x00002000); + msgRateExpired_ = 0D; + + return this; + } + + // optional uint64 msgBacklog = 15; + private long msgBacklog_ ; + public boolean hasMsgBacklog() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public long getMsgBacklog() { + return msgBacklog_; + } + public Builder setMsgBacklog(long value) { + bitField0_ |= 0x00004000; + msgBacklog_ = value; + + return this; + } + public Builder clearMsgBacklog() { + bitField0_ = (bitField0_ & ~0x00004000); + msgBacklog_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConsumerStatsResponse) + } + + static { + defaultInstance = new CommandConsumerStatsResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConsumerStatsResponse) + } + + public interface CommandGetLastMessageIdOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 consumer_id = 1; + boolean hasConsumerId(); + long getConsumerId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + } + public static final class CommandGetLastMessageId extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetLastMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetLastMessageId.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetLastMessageId(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetLastMessageId newObject(Handle handle) { + return new CommandGetLastMessageId(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetLastMessageId(boolean noInit) {} + + private static final CommandGetLastMessageId defaultInstance; + public static CommandGetLastMessageId getDefaultInstance() { + return defaultInstance; + } + + public CommandGetLastMessageId getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 consumer_id = 1; + public static final int CONSUMER_ID_FIELD_NUMBER = 1; + private long consumerId_; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + private void initFields() { + consumerId_ = 0L; + requestId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConsumerId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + } + + 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, consumerId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId 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.PulsarApi.CommandGetLastMessageId, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.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(); + consumerId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.consumerId_ = consumerId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance()) return this; + if (other.hasConsumerId()) { + setConsumerId(other.getConsumerId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasConsumerId()) { + + return false; + } + if (!hasRequestId()) { + + 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; + consumerId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 consumer_id = 1; + private long consumerId_ ; + public boolean hasConsumerId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getConsumerId() { + return consumerId_; + } + public Builder setConsumerId(long value) { + bitField0_ |= 0x00000001; + consumerId_ = value; + + return this; + } + public Builder clearConsumerId() { + bitField0_ = (bitField0_ & ~0x00000001); + consumerId_ = 0L; + + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetLastMessageId) + } + + static { + defaultInstance = new CommandGetLastMessageId(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetLastMessageId) + } + + public interface CommandGetLastMessageIdResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required .pulsar.proto.MessageIdData last_message_id = 1; + boolean hasLastMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId(); + + // required uint64 request_id = 2; + boolean hasRequestId(); + long getRequestId(); + } + public static final class CommandGetLastMessageIdResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetLastMessageIdResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetLastMessageIdResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetLastMessageIdResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetLastMessageIdResponse newObject(Handle handle) { + return new CommandGetLastMessageIdResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetLastMessageIdResponse(boolean noInit) {} + + private static final CommandGetLastMessageIdResponse defaultInstance; + public static CommandGetLastMessageIdResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandGetLastMessageIdResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required .pulsar.proto.MessageIdData last_message_id = 1; + public static final int LAST_MESSAGE_ID_FIELD_NUMBER = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData lastMessageId_; + public boolean hasLastMessageId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId() { + return lastMessageId_; + } + + // required uint64 request_id = 2; + public static final int REQUEST_ID_FIELD_NUMBER = 2; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + + private void initFields() { + lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + requestId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasLastMessageId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!getLastMessageId().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.writeMessage(1, lastMessageId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, requestId_); + } + } + + 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 + .computeMessageSize(1, lastMessageId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, requestId_); + } + 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse 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.PulsarApi.CommandGetLastMessageIdResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.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(); + lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.lastMessageId_ = lastMessageId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.requestId_ = requestId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance()) return this; + if (other.hasLastMessageId()) { + mergeLastMessageId(other.getLastMessageId()); + } + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasLastMessageId()) { + + return false; + } + if (!hasRequestId()) { + + return false; + } + if (!getLastMessageId().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: { + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(); + if (hasLastMessageId()) { + subBuilder.mergeFrom(getLastMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setLastMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + requestId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required .pulsar.proto.MessageIdData last_message_id = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + public boolean hasLastMessageId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData getLastMessageId() { + return lastMessageId_; + } + public Builder setLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (value == null) { + throw new NullPointerException(); + } + lastMessageId_ = value; + + bitField0_ |= 0x00000001; + return this; + } + public Builder setLastMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.Builder builderForValue) { + lastMessageId_ = builderForValue.build(); + + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData value) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + lastMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance()) { + lastMessageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.newBuilder(lastMessageId_).mergeFrom(value).buildPartial(); + } else { + lastMessageId_ = value; + } + + bitField0_ |= 0x00000001; + return this; + } + public Builder clearLastMessageId() { + lastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + // required uint64 request_id = 2; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000002; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000002); + requestId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetLastMessageIdResponse) + } + + static { + defaultInstance = new CommandGetLastMessageIdResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetLastMessageIdResponse) + } + + public interface CommandGetTopicsOfNamespaceOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required string namespace = 2; + boolean hasNamespace(); + String getNamespace(); + + // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; + boolean hasMode(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode(); + } + public static final class CommandGetTopicsOfNamespace extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetTopicsOfNamespaceOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetTopicsOfNamespace.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetTopicsOfNamespace(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetTopicsOfNamespace newObject(Handle handle) { + return new CommandGetTopicsOfNamespace(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetTopicsOfNamespace(boolean noInit) {} + + private static final CommandGetTopicsOfNamespace defaultInstance; + public static CommandGetTopicsOfNamespace getDefaultInstance() { + return defaultInstance; + } + + public CommandGetTopicsOfNamespace getDefaultInstanceForType() { + return defaultInstance; + } + + public enum Mode + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + PERSISTENT(0, 0), + NON_PERSISTENT(1, 1), + ALL(2, 2), + ; + + public static final int PERSISTENT_VALUE = 0; + public static final int NON_PERSISTENT_VALUE = 1; + public static final int ALL_VALUE = 2; + + + public final int getNumber() { return value; } + + public static Mode valueOf(int value) { + switch (value) { + case 0: return PERSISTENT; + case 1: return NON_PERSISTENT; + case 2: return ALL; + 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 Mode findValueByNumber(int number) { + return Mode.valueOf(number); + } + }; + + private final int value; + + private Mode(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.CommandGetTopicsOfNamespace.Mode) + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required string namespace = 2; + public static final int NAMESPACE_FIELD_NUMBER = 2; + private java.lang.Object namespace_; + public boolean hasNamespace() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getNamespace() { + java.lang.Object ref = namespace_; + 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)) { + namespace_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getNamespaceBytes() { + java.lang.Object ref = namespace_; + 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); + namespace_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; + public static final int MODE_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode mode_; + public boolean hasMode() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode() { + return mode_; + } + + private void initFields() { + requestId_ = 0L; + namespace_ = ""; + mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasNamespace()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getNamespaceBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeEnum(3, mode_.getNumber()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getNamespaceBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(3, mode_.getNumber()); + } + 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace 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.PulsarApi.CommandGetTopicsOfNamespace, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + namespace_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.namespace_ = namespace_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.mode_ = mode_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasNamespace()) { + setNamespace(other.getNamespace()); + } + if (other.hasMode()) { + setMode(other.getMode()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasNamespace()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + namespace_ = input.readBytes(); + break; + } + case 24: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode value = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000004; + mode_ = value; + } + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required string namespace = 2; + private java.lang.Object namespace_ = ""; + public boolean hasNamespace() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getNamespace() { + java.lang.Object ref = namespace_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + namespace_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setNamespace(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + namespace_ = value; + + return this; + } + public Builder clearNamespace() { + bitField0_ = (bitField0_ & ~0x00000002); + namespace_ = getDefaultInstance().getNamespace(); + + return this; + } + void setNamespace(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + namespace_ = value; + + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespace.Mode mode = 3 [default = PERSISTENT]; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; + public boolean hasMode() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode getMode() { + return mode_; + } + public Builder setMode(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + mode_ = value; + + return this; + } + public Builder clearMode() { + bitField0_ = (bitField0_ & ~0x00000004); + mode_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Mode.PERSISTENT; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetTopicsOfNamespace) + } + + static { + defaultInstance = new CommandGetTopicsOfNamespace(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetTopicsOfNamespace) + } + + public interface CommandGetTopicsOfNamespaceResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // repeated string topics = 2; + java.util.List getTopicsList(); + int getTopicsCount(); + String getTopics(int index); + } + public static final class CommandGetTopicsOfNamespaceResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetTopicsOfNamespaceResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetTopicsOfNamespaceResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetTopicsOfNamespaceResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetTopicsOfNamespaceResponse newObject(Handle handle) { + return new CommandGetTopicsOfNamespaceResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetTopicsOfNamespaceResponse(boolean noInit) {} + + private static final CommandGetTopicsOfNamespaceResponse defaultInstance; + public static CommandGetTopicsOfNamespaceResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandGetTopicsOfNamespaceResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // repeated string topics = 2; + public static final int TOPICS_FIELD_NUMBER = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList topics_; + public java.util.List + getTopicsList() { + return topics_; + } + public int getTopicsCount() { + return topics_.size(); + } + public String getTopics(int index) { + return topics_.get(index); + } + + private void initFields() { + requestId_ = 0L; + topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + for (int i = 0; i < topics_.size(); i++) { + output.writeBytes(2, topics_.getByteString(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 + .computeUInt64Size(1, requestId_); + } + { + int dataSize = 0; + for (int i = 0; i < topics_.size(); i++) { + dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSizeNoTag(topics_.getByteString(i)); + } + size += dataSize; + size += 1 * getTopicsList().size(); + } + 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse 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.PulsarApi.CommandGetTopicsOfNamespaceResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((bitField0_ & 0x00000002) == 0x00000002)) { + topics_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( + topics_); + bitField0_ = (bitField0_ & ~0x00000002); + } + result.topics_ = topics_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (!other.topics_.isEmpty()) { + if (topics_.isEmpty()) { + topics_ = other.topics_; + bitField0_ = (bitField0_ & ~0x00000002); + } else { + ensureTopicsIsMutable(); + topics_.addAll(other.topics_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 18: { + ensureTopicsIsMutable(); + topics_.add(input.readBytes()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // repeated string topics = 2; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + private void ensureTopicsIsMutable() { + if (!((bitField0_ & 0x00000002) == 0x00000002)) { + topics_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(topics_); + bitField0_ |= 0x00000002; + } + } + public java.util.List + getTopicsList() { + return java.util.Collections.unmodifiableList(topics_); + } + public int getTopicsCount() { + return topics_.size(); + } + public String getTopics(int index) { + return topics_.get(index); + } + public Builder setTopics( + int index, String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureTopicsIsMutable(); + topics_.set(index, value); + + return this; + } + public Builder addTopics(String value) { + if (value == null) { + throw new NullPointerException(); + } + ensureTopicsIsMutable(); + topics_.add(value); + + return this; + } + public Builder addAllTopics( + java.lang.Iterable values) { + ensureTopicsIsMutable(); + super.addAll(values, topics_); + + return this; + } + public Builder clearTopics() { + topics_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + + return this; + } + void addTopics(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + ensureTopicsIsMutable(); + topics_.add(value); + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetTopicsOfNamespaceResponse) + } + + static { + defaultInstance = new CommandGetTopicsOfNamespaceResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetTopicsOfNamespaceResponse) + } + + public interface CommandGetSchemaOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required string topic = 2; + boolean hasTopic(); + String getTopic(); + + // optional bytes schema_version = 3; + boolean hasSchemaVersion(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); + } + public static final class CommandGetSchema extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetSchema.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetSchema(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetSchema newObject(Handle handle) { + return new CommandGetSchema(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetSchema(boolean noInit) {} + + private static final CommandGetSchema defaultInstance; + public static CommandGetSchema getDefaultInstance() { + return defaultInstance; + } + + public CommandGetSchema getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required string topic = 2; + public static final int TOPIC_FIELD_NUMBER = 2; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bytes schema_version = 3; + public static final int SCHEMA_VERSION_FIELD_NUMBER = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + + private void initFields() { + requestId_ = 0L; + topic_ = ""; + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTopic()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getTopicBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, schemaVersion_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getTopicBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, schemaVersion_); + } + 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema 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.PulsarApi.CommandGetSchema, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.schemaVersion_ = schemaVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasSchemaVersion()) { + setSchemaVersion(other.getSchemaVersion()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasTopic()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + topic_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + schemaVersion_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required string topic = 2; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000002); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + topic_ = value; + + } + + // optional bytes schema_version = 3; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + schemaVersion_ = value; + + return this; + } + public Builder clearSchemaVersion() { + bitField0_ = (bitField0_ & ~0x00000004); + schemaVersion_ = getDefaultInstance().getSchemaVersion(); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetSchema) + } + + static { + defaultInstance = new CommandGetSchema(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetSchema) + } + + public interface CommandGetSchemaResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.ServerError error_code = 2; + boolean hasErrorCode(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); + + // optional string error_message = 3; + boolean hasErrorMessage(); + String getErrorMessage(); + + // optional .pulsar.proto.Schema schema = 4; + boolean hasSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); + + // optional bytes schema_version = 5; + boolean hasSchemaVersion(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); + } + public static final class CommandGetSchemaResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetSchemaResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetSchemaResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetSchemaResponse newObject(Handle handle) { + return new CommandGetSchemaResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetSchemaResponse(boolean noInit) {} + + private static final CommandGetSchemaResponse defaultInstance; + public static CommandGetSchemaResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandGetSchemaResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.ServerError error_code = 2; + public static final int ERROR_CODE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + + // optional string error_message = 3; + public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; + private java.lang.Object errorMessage_; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + 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)) { + errorMessage_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { + java.lang.Object ref = errorMessage_; + 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); + errorMessage_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.Schema schema = 4; + public static final int SCHEMA_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; + public boolean hasSchema() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + + // optional bytes schema_version = 5; + public static final int SCHEMA_VERSION_FIELD_NUMBER = 5; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + + private void initFields() { + requestId_ = 0L; + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + errorMessage_ = ""; + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasSchema()) { + if (!getSchema().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.writeUInt64(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, schema_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, schemaVersion_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, schema_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, schemaVersion_); + } + 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse 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.PulsarApi.CommandGetSchemaResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000002); + errorMessage_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000008); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.errorCode_ = errorCode_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.errorMessage_ = errorMessage_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.schema_ = schema_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.schemaVersion_ = schemaVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasErrorCode()) { + setErrorCode(other.getErrorCode()); + } + if (other.hasErrorMessage()) { + setErrorMessage(other.getErrorMessage()); + } + if (other.hasSchema()) { + mergeSchema(other.getSchema()); + } + if (other.hasSchemaVersion()) { + setSchemaVersion(other.getSchemaVersion()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (hasSchema()) { + if (!getSchema().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 8: { + bitField0_ |= 0x00000001; + requestId_ = input.readUInt64(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + errorCode_ = value; + } + break; + } + case 26: { + bitField0_ |= 0x00000004; + errorMessage_ = input.readBytes(); + break; + } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); + if (hasSchema()) { + subBuilder.mergeFrom(getSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 42: { + bitField0_ |= 0x00000010; + schemaVersion_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error_code = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + errorCode_ = value; + + return this; + } + public Builder clearErrorCode() { + bitField0_ = (bitField0_ & ~0x00000002); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string error_message = 3; + private java.lang.Object errorMessage_ = ""; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + errorMessage_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setErrorMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + errorMessage_ = value; + + return this; + } + public Builder clearErrorMessage() { + bitField0_ = (bitField0_ & ~0x00000004); + errorMessage_ = getDefaultInstance().getErrorMessage(); + + return this; + } + void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + errorMessage_ = value; + + } + + // optional .pulsar.proto.Schema schema = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + public boolean hasSchema() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (value == null) { + throw new NullPointerException(); + } + schema_ = value; + + bitField0_ |= 0x00000008; + return this; + } + public Builder setSchema( + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { + schema_ = builderForValue.build(); + + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { + schema_ = + org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); + } else { + schema_ = value; + } + + bitField0_ |= 0x00000008; + return this; + } + public Builder clearSchema() { + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + // optional bytes schema_version = 5; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + schemaVersion_ = value; + + return this; + } + public Builder clearSchemaVersion() { + bitField0_ = (bitField0_ & ~0x00000010); + schemaVersion_ = getDefaultInstance().getSchemaVersion(); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetSchemaResponse) + } + + static { + defaultInstance = new CommandGetSchemaResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetSchemaResponse) + } + + public interface CommandGetOrCreateSchemaOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // required string topic = 2; + boolean hasTopic(); + String getTopic(); + + // required .pulsar.proto.Schema schema = 3; + boolean hasSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema(); + } + public static final class CommandGetOrCreateSchema extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetOrCreateSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetOrCreateSchema.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetOrCreateSchema(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetOrCreateSchema newObject(Handle handle) { + return new CommandGetOrCreateSchema(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetOrCreateSchema(boolean noInit) {} + + private static final CommandGetOrCreateSchema defaultInstance; + public static CommandGetOrCreateSchema getDefaultInstance() { + return defaultInstance; + } + + public CommandGetOrCreateSchema getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // required string topic = 2; + public static final int TOPIC_FIELD_NUMBER = 2; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required .pulsar.proto.Schema schema = 3; + public static final int SCHEMA_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_; + public boolean hasSchema() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + + private void initFields() { + requestId_ = 0L; + topic_ = ""; + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSchema()) { + memoizedIsInitialized = 0; + return false; + } + if (!getSchema().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.writeUInt64(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getTopicBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, schema_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getTopicBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, schema_); + } + 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema 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.PulsarApi.CommandGetOrCreateSchema, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.schema_ = schema_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasSchema()) { + mergeSchema(other.getSchema()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (!hasTopic()) { + + return false; + } + if (!hasSchema()) { + + return false; + } + if (!getSchema().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 8: { + bitField0_ |= 0x00000001; + requestId_ = input.readUInt64(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + topic_ = input.readBytes(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(); + if (hasSchema()) { + subBuilder.mergeFrom(getSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // required string topic = 2; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000002); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + topic_ = value; + + } + + // required .pulsar.proto.Schema schema = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.Schema schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + public boolean hasSchema() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Schema getSchema() { + return schema_; + } + public Builder setSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (value == null) { + throw new NullPointerException(); + } + schema_ = value; + + bitField0_ |= 0x00000004; + return this; + } + public Builder setSchema( + org.apache.pulsar.common.api.proto.PulsarApi.Schema.Builder builderForValue) { + schema_ = builderForValue.build(); + + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeSchema(org.apache.pulsar.common.api.proto.PulsarApi.Schema value) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + schema_ != org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance()) { + schema_ = + org.apache.pulsar.common.api.proto.PulsarApi.Schema.newBuilder(schema_).mergeFrom(value).buildPartial(); + } else { + schema_ = value; + } + + bitField0_ |= 0x00000004; + return this; + } + public Builder clearSchema() { + schema_ = org.apache.pulsar.common.api.proto.PulsarApi.Schema.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetOrCreateSchema) + } + + static { + defaultInstance = new CommandGetOrCreateSchema(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetOrCreateSchema) + } + + public interface CommandGetOrCreateSchemaResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional .pulsar.proto.ServerError error_code = 2; + boolean hasErrorCode(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode(); + + // optional string error_message = 3; + boolean hasErrorMessage(); + String getErrorMessage(); + + // optional bytes schema_version = 4; + boolean hasSchemaVersion(); + org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion(); + } + public static final class CommandGetOrCreateSchemaResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandGetOrCreateSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandGetOrCreateSchemaResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandGetOrCreateSchemaResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandGetOrCreateSchemaResponse newObject(Handle handle) { + return new CommandGetOrCreateSchemaResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandGetOrCreateSchemaResponse(boolean noInit) {} + + private static final CommandGetOrCreateSchemaResponse defaultInstance; + public static CommandGetOrCreateSchemaResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandGetOrCreateSchemaResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional .pulsar.proto.ServerError error_code = 2; + public static final int ERROR_CODE_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + + // optional string error_message = 3; + public static final int ERROR_MESSAGE_FIELD_NUMBER = 3; + private java.lang.Object errorMessage_; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + 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)) { + errorMessage_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getErrorMessageBytes() { + java.lang.Object ref = errorMessage_; + 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); + errorMessage_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional bytes schema_version = 4; + public static final int SCHEMA_VERSION_FIELD_NUMBER = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + + private void initFields() { + requestId_ = 0L; + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + errorMessage_ = ""; + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, schemaVersion_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(2, errorCode_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(3, getErrorMessageBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, schemaVersion_); + } + 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse 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.PulsarApi.CommandGetOrCreateSchemaResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000002); + errorMessage_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.errorCode_ = errorCode_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.errorMessage_ = errorMessage_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.schemaVersion_ = schemaVersion_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasErrorCode()) { + setErrorCode(other.getErrorCode()); + } + if (other.hasErrorMessage()) { + setErrorMessage(other.getErrorMessage()); + } + if (other.hasSchemaVersion()) { + setSchemaVersion(other.getSchemaVersion()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000002; + errorCode_ = value; + } + break; + } + case 26: { + bitField0_ |= 0x00000004; + errorMessage_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + schemaVersion_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error_code = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasErrorCode() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getErrorCode() { + return errorCode_; + } + public Builder setErrorCode(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + errorCode_ = value; + + return this; + } + public Builder clearErrorCode() { + bitField0_ = (bitField0_ & ~0x00000002); + errorCode_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string error_message = 3; + private java.lang.Object errorMessage_ = ""; + public boolean hasErrorMessage() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getErrorMessage() { + java.lang.Object ref = errorMessage_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + errorMessage_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setErrorMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + errorMessage_ = value; + + return this; + } + public Builder clearErrorMessage() { + bitField0_ = (bitField0_ & ~0x00000004); + errorMessage_ = getDefaultInstance().getErrorMessage(); + + return this; + } + void setErrorMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000004; + errorMessage_ = value; + + } + + // optional bytes schema_version = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString schemaVersion_ = org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString.EMPTY; + public boolean hasSchemaVersion() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSchemaVersion() { + return schemaVersion_; + } + public Builder setSchemaVersion(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + schemaVersion_ = value; + + return this; + } + public Builder clearSchemaVersion() { + bitField0_ = (bitField0_ & ~0x00000008); + schemaVersion_ = getDefaultInstance().getSchemaVersion(); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandGetOrCreateSchemaResponse) + } + + static { + defaultInstance = new CommandGetOrCreateSchemaResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandGetOrCreateSchemaResponse) + } + + public interface CommandNewTxnOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txn_ttl_seconds = 2 [default = 0]; + boolean hasTxnTtlSeconds(); + long getTxnTtlSeconds(); + + // optional uint64 tc_id = 3 [default = 0]; + boolean hasTcId(); + long getTcId(); + } + public static final class CommandNewTxn extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandNewTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandNewTxn.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandNewTxn(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandNewTxn newObject(Handle handle) { + return new CommandNewTxn(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandNewTxn(boolean noInit) {} + + private static final CommandNewTxn defaultInstance; + public static CommandNewTxn getDefaultInstance() { + return defaultInstance; + } + + public CommandNewTxn getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txn_ttl_seconds = 2 [default = 0]; + public static final int TXN_TTL_SECONDS_FIELD_NUMBER = 2; + private long txnTtlSeconds_; + public boolean hasTxnTtlSeconds() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnTtlSeconds() { + return txnTtlSeconds_; + } + + // optional uint64 tc_id = 3 [default = 0]; + public static final int TC_ID_FIELD_NUMBER = 3; + private long tcId_; + public boolean hasTcId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTcId() { + return tcId_; + } + + private void initFields() { + requestId_ = 0L; + txnTtlSeconds_ = 0L; + tcId_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnTtlSeconds_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, tcId_); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnTtlSeconds_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, tcId_); + } + 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn 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.PulsarApi.CommandNewTxn, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnTtlSeconds_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + tcId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnTtlSeconds_ = txnTtlSeconds_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.tcId_ = tcId_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnTtlSeconds()) { + setTxnTtlSeconds(other.getTxnTtlSeconds()); + } + if (other.hasTcId()) { + setTcId(other.getTcId()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnTtlSeconds_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + tcId_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txn_ttl_seconds = 2 [default = 0]; + private long txnTtlSeconds_ ; + public boolean hasTxnTtlSeconds() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnTtlSeconds() { + return txnTtlSeconds_; + } + public Builder setTxnTtlSeconds(long value) { + bitField0_ |= 0x00000002; + txnTtlSeconds_ = value; + + return this; + } + public Builder clearTxnTtlSeconds() { + bitField0_ = (bitField0_ & ~0x00000002); + txnTtlSeconds_ = 0L; + + return this; + } + + // optional uint64 tc_id = 3 [default = 0]; + private long tcId_ ; + public boolean hasTcId() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTcId() { + return tcId_; + } + public Builder setTcId(long value) { + bitField0_ |= 0x00000004; + tcId_ = value; + + return this; + } + public Builder clearTcId() { + bitField0_ = (bitField0_ & ~0x00000004); + tcId_ = 0L; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandNewTxn) + } + + static { + defaultInstance = new CommandNewTxn(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandNewTxn) + } + + public interface CommandNewTxnResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandNewTxnResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandNewTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandNewTxnResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandNewTxnResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandNewTxnResponse newObject(Handle handle) { + return new CommandNewTxnResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandNewTxnResponse(boolean noInit) {} + + private static final CommandNewTxnResponse defaultInstance; + public static CommandNewTxnResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandNewTxnResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse 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.PulsarApi.CommandNewTxnResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandNewTxnResponse) + } + + static { + defaultInstance = new CommandNewTxnResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandNewTxnResponse) + } + + public interface CommandAddPartitionToTxnOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // repeated string partitions = 4; + java.util.List getPartitionsList(); + int getPartitionsCount(); + String getPartitions(int index); + } + public static final class CommandAddPartitionToTxn extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAddPartitionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAddPartitionToTxn.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAddPartitionToTxn(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAddPartitionToTxn newObject(Handle handle) { + return new CommandAddPartitionToTxn(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAddPartitionToTxn(boolean noInit) {} + + private static final CommandAddPartitionToTxn defaultInstance; + public static CommandAddPartitionToTxn getDefaultInstance() { + return defaultInstance; + } + + public CommandAddPartitionToTxn getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // repeated string partitions = 4; + public static final int PARTITIONS_FIELD_NUMBER = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList partitions_; + public java.util.List + getPartitionsList() { + return partitions_; + } + public int getPartitionsCount() { + return partitions_.size(); + } + public String getPartitions(int index) { + return partitions_.get(index); + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + for (int i = 0; i < partitions_.size(); i++) { + output.writeBytes(4, partitions_.getByteString(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 + .computeUInt64Size(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + { + int dataSize = 0; + for (int i = 0; i < partitions_.size(); i++) { + dataSize += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSizeNoTag(partitions_.getByteString(i)); + } + size += dataSize; + size += 1 * getPartitionsList().size(); + } + 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn 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.PulsarApi.CommandAddPartitionToTxn, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + partitions_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.UnmodifiableLazyStringList( + partitions_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.partitions_ = partitions_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (!other.partitions_.isEmpty()) { + if (partitions_.isEmpty()) { + partitions_ = other.partitions_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensurePartitionsIsMutable(); + partitions_.addAll(other.partitions_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 34: { + ensurePartitionsIsMutable(); + partitions_.add(input.readBytes()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // repeated string partitions = 4; + private org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringList partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + private void ensurePartitionsIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + partitions_ = new org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList(partitions_); + bitField0_ |= 0x00000008; + } + } + public java.util.List + getPartitionsList() { + return java.util.Collections.unmodifiableList(partitions_); + } + public int getPartitionsCount() { + return partitions_.size(); + } + public String getPartitions(int index) { + return partitions_.get(index); + } + public Builder setPartitions( + int index, String value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePartitionsIsMutable(); + partitions_.set(index, value); + + return this; + } + public Builder addPartitions(String value) { + if (value == null) { + throw new NullPointerException(); + } + ensurePartitionsIsMutable(); + partitions_.add(value); + + return this; + } + public Builder addAllPartitions( + java.lang.Iterable values) { + ensurePartitionsIsMutable(); + super.addAll(values, partitions_); + + return this; + } + public Builder clearPartitions() { + partitions_ = org.apache.pulsar.shaded.com.google.protobuf.v241.LazyStringArrayList.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + void addPartitions(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + ensurePartitionsIsMutable(); + partitions_.add(value); + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddPartitionToTxn) + } + + static { + defaultInstance = new CommandAddPartitionToTxn(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddPartitionToTxn) + } + + public interface CommandAddPartitionToTxnResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandAddPartitionToTxnResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAddPartitionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAddPartitionToTxnResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAddPartitionToTxnResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAddPartitionToTxnResponse newObject(Handle handle) { + return new CommandAddPartitionToTxnResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAddPartitionToTxnResponse(boolean noInit) {} + + private static final CommandAddPartitionToTxnResponse defaultInstance; + public static CommandAddPartitionToTxnResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandAddPartitionToTxnResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse 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.PulsarApi.CommandAddPartitionToTxnResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddPartitionToTxnResponse) + } + + static { + defaultInstance = new CommandAddPartitionToTxnResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddPartitionToTxnResponse) + } + + public interface SubscriptionOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required string topic = 1; + boolean hasTopic(); + String getTopic(); + + // required string subscription = 2; + boolean hasSubscription(); + String getSubscription(); + } + public static final class Subscription extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements SubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use Subscription.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private Subscription(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected Subscription newObject(Handle handle) { + return new Subscription(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private Subscription(boolean noInit) {} + + private static final Subscription defaultInstance; + public static Subscription getDefaultInstance() { + return defaultInstance; + } + + public Subscription getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required string topic = 1; + public static final int TOPIC_FIELD_NUMBER = 1; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // required string subscription = 2; + public static final int SUBSCRIPTION_FIELD_NUMBER = 2; + private java.lang.Object subscription_; + public boolean hasSubscription() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSubscription() { + java.lang.Object ref = subscription_; + 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)) { + subscription_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getSubscriptionBytes() { + java.lang.Object ref = subscription_; + 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); + subscription_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + topic_ = ""; + subscription_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTopic()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasSubscription()) { + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSubscriptionBytes()); + } + } + + 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, getTopicBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(2, getSubscriptionBytes()); + } + 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription 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.PulsarApi.Subscription, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.SubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.Subscription.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(); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + subscription_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription build() { + org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.Subscription buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.Subscription result = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.subscription_ = subscription_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.Subscription other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance()) return this; + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasSubscription()) { + setSubscription(other.getSubscription()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasTopic()) { + + return false; + } + if (!hasSubscription()) { + + 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; + topic_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + subscription_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string topic = 1; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000001); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000001; + topic_ = value; + + } + + // required string subscription = 2; + private java.lang.Object subscription_ = ""; + public boolean hasSubscription() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSubscription() { + java.lang.Object ref = subscription_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + subscription_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setSubscription(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + subscription_ = value; + + return this; + } + public Builder clearSubscription() { + bitField0_ = (bitField0_ & ~0x00000002); + subscription_ = getDefaultInstance().getSubscription(); + + return this; + } + void setSubscription(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000002; + subscription_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.Subscription) + } + + static { + defaultInstance = new Subscription(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.Subscription) + } + + public interface CommandAddSubscriptionToTxnOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // repeated .pulsar.proto.Subscription subscription = 4; + java.util.List + getSubscriptionList(); + org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index); + int getSubscriptionCount(); + } + public static final class CommandAddSubscriptionToTxn extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAddSubscriptionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAddSubscriptionToTxn.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAddSubscriptionToTxn(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAddSubscriptionToTxn newObject(Handle handle) { + return new CommandAddSubscriptionToTxn(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAddSubscriptionToTxn(boolean noInit) {} + + private static final CommandAddSubscriptionToTxn defaultInstance; + public static CommandAddSubscriptionToTxn getDefaultInstance() { + return defaultInstance; + } + + public CommandAddSubscriptionToTxn getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // repeated .pulsar.proto.Subscription subscription = 4; + public static final int SUBSCRIPTION_FIELD_NUMBER = 4; + private java.util.List subscription_; + public java.util.List getSubscriptionList() { + return subscription_; + } + public java.util.List + getSubscriptionOrBuilderList() { + return subscription_; + } + public int getSubscriptionCount() { + return subscription_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index) { + return subscription_.get(index); + } + public org.apache.pulsar.common.api.proto.PulsarApi.SubscriptionOrBuilder getSubscriptionOrBuilder( + int index) { + return subscription_.get(index); + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + subscription_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + for (int i = 0; i < getSubscriptionCount(); i++) { + if (!getSubscription(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.writeUInt64(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + for (int i = 0; i < subscription_.size(); i++) { + output.writeMessage(4, subscription_.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 + .computeUInt64Size(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + for (int i = 0; i < subscription_.size(); i++) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, subscription_.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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn 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.PulsarApi.CommandAddSubscriptionToTxn, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + subscription_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + subscription_ = java.util.Collections.unmodifiableList(subscription_); + bitField0_ = (bitField0_ & ~0x00000008); + } + result.subscription_ = subscription_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (!other.subscription_.isEmpty()) { + if (subscription_.isEmpty()) { + subscription_ = other.subscription_; + bitField0_ = (bitField0_ & ~0x00000008); + } else { + ensureSubscriptionIsMutable(); + subscription_.addAll(other.subscription_); + } + + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + for (int i = 0; i < getSubscriptionCount(); i++) { + if (!getSubscription(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 8: { + bitField0_ |= 0x00000001; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addSubscription(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // repeated .pulsar.proto.Subscription subscription = 4; + private java.util.List subscription_ = + java.util.Collections.emptyList(); + private void ensureSubscriptionIsMutable() { + if (!((bitField0_ & 0x00000008) == 0x00000008)) { + subscription_ = new java.util.ArrayList(subscription_); + bitField0_ |= 0x00000008; + } + } + + public java.util.List getSubscriptionList() { + return java.util.Collections.unmodifiableList(subscription_); + } + public int getSubscriptionCount() { + return subscription_.size(); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(int index) { + return subscription_.get(index); + } + public Builder setSubscription( + int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { + if (value == null) { + throw new NullPointerException(); + } + ensureSubscriptionIsMutable(); + subscription_.set(index, value); + + return this; + } + public Builder setSubscription( + int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { + ensureSubscriptionIsMutable(); + subscription_.set(index, builderForValue.build()); + + return this; + } + public Builder addSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { + if (value == null) { + throw new NullPointerException(); + } + ensureSubscriptionIsMutable(); + subscription_.add(value); + + return this; + } + public Builder addSubscription( + int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { + if (value == null) { + throw new NullPointerException(); + } + ensureSubscriptionIsMutable(); + subscription_.add(index, value); + + return this; + } + public Builder addSubscription( + org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { + ensureSubscriptionIsMutable(); + subscription_.add(builderForValue.build()); + + return this; + } + public Builder addSubscription( + int index, org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { + ensureSubscriptionIsMutable(); + subscription_.add(index, builderForValue.build()); + + return this; + } + public Builder addAllSubscription( + java.lang.Iterable values) { + ensureSubscriptionIsMutable(); + super.addAll(values, subscription_); + + return this; + } + public Builder clearSubscription() { + subscription_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000008); + + return this; + } + public Builder removeSubscription(int index) { + ensureSubscriptionIsMutable(); + subscription_.remove(index); + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddSubscriptionToTxn) + } + + static { + defaultInstance = new CommandAddSubscriptionToTxn(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddSubscriptionToTxn) + } + + public interface CommandAddSubscriptionToTxnResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandAddSubscriptionToTxnResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandAddSubscriptionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandAddSubscriptionToTxnResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandAddSubscriptionToTxnResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandAddSubscriptionToTxnResponse newObject(Handle handle) { + return new CommandAddSubscriptionToTxnResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandAddSubscriptionToTxnResponse(boolean noInit) {} + + private static final CommandAddSubscriptionToTxnResponse defaultInstance; + public static CommandAddSubscriptionToTxnResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandAddSubscriptionToTxnResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse 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.PulsarApi.CommandAddSubscriptionToTxnResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandAddSubscriptionToTxnResponse) + } + + static { + defaultInstance = new CommandAddSubscriptionToTxnResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandAddSubscriptionToTxnResponse) + } + + public interface CommandEndTxnOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.TxnAction txn_action = 4; + boolean hasTxnAction(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); + } + public static final class CommandEndTxn extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxn.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxn(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxn newObject(Handle handle) { + return new CommandEndTxn(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxn(boolean noInit) {} + + private static final CommandEndTxn defaultInstance; + public static CommandEndTxn getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxn getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.TxnAction txn_action = 4; + public static final int TXN_ACTION_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, txnAction_.getNumber()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, txnAction_.getNumber()); + } + 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn 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.PulsarApi.CommandEndTxn, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.txnAction_ = txnAction_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasTxnAction()) { + setTxnAction(other.getTxnAction()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + txnAction_ = value; + } + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.TxnAction txn_action = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + txnAction_ = value; + + return this; + } + public Builder clearTxnAction() { + bitField0_ = (bitField0_ & ~0x00000008); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxn) + } + + static { + defaultInstance = new CommandEndTxn(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxn) + } + + public interface CommandEndTxnResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandEndTxnResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxnResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxnResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxnResponse newObject(Handle handle) { + return new CommandEndTxnResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxnResponse(boolean noInit) {} + + private static final CommandEndTxnResponse defaultInstance; + public static CommandEndTxnResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxnResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse 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.PulsarApi.CommandEndTxnResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnResponse) + } + + static { + defaultInstance = new CommandEndTxnResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnResponse) + } + + public interface CommandEndTxnOnPartitionOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional string topic = 4; + boolean hasTopic(); + String getTopic(); + + // optional .pulsar.proto.TxnAction txn_action = 5; + boolean hasTxnAction(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); + } + public static final class CommandEndTxnOnPartition extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnOnPartitionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxnOnPartition.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxnOnPartition(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxnOnPartition newObject(Handle handle) { + return new CommandEndTxnOnPartition(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxnOnPartition(boolean noInit) {} + + private static final CommandEndTxnOnPartition defaultInstance; + public static CommandEndTxnOnPartition getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxnOnPartition getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional string topic = 4; + public static final int TOPIC_FIELD_NUMBER = 4; + private java.lang.Object topic_; + public boolean hasTopic() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getTopic() { + java.lang.Object ref = topic_; + 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)) { + topic_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getTopicBytes() { + java.lang.Object ref = topic_; + 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); + topic_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + // optional .pulsar.proto.TxnAction txn_action = 5; + public static final int TXN_ACTION_FIELD_NUMBER = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + topic_ = ""; + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, getTopicBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeEnum(5, txnAction_.getNumber()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(4, getTopicBytes()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(5, txnAction_.getNumber()); + } + 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition 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.PulsarApi.CommandEndTxnOnPartition, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + topic_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.topic_ = topic_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.txnAction_ = txnAction_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasTopic()) { + setTopic(other.getTopic()); + } + if (other.hasTxnAction()) { + setTxnAction(other.getTxnAction()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + topic_ = input.readBytes(); + break; + } + case 40: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000010; + txnAction_ = value; + } + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional string topic = 4; + private java.lang.Object topic_ = ""; + public boolean hasTopic() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public String getTopic() { + java.lang.Object ref = topic_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + topic_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTopic(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + topic_ = value; + + return this; + } + public Builder clearTopic() { + bitField0_ = (bitField0_ & ~0x00000008); + topic_ = getDefaultInstance().getTopic(); + + return this; + } + void setTopic(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000008; + topic_ = value; + + } + + // optional .pulsar.proto.TxnAction txn_action = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + txnAction_ = value; + + return this; + } + public Builder clearTxnAction() { + bitField0_ = (bitField0_ & ~0x00000010); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnPartition) + } + + static { + defaultInstance = new CommandEndTxnOnPartition(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnPartition) + } + + public interface CommandEndTxnOnPartitionResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandEndTxnOnPartitionResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnOnPartitionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxnOnPartitionResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxnOnPartitionResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxnOnPartitionResponse newObject(Handle handle) { + return new CommandEndTxnOnPartitionResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxnOnPartitionResponse(boolean noInit) {} + + private static final CommandEndTxnOnPartitionResponse defaultInstance; + public static CommandEndTxnOnPartitionResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxnOnPartitionResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse 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.PulsarApi.CommandEndTxnOnPartitionResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnPartitionResponse) + } + + static { + defaultInstance = new CommandEndTxnOnPartitionResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnPartitionResponse) + } + + public interface CommandEndTxnOnSubscriptionOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.Subscription subscription = 4; + boolean hasSubscription(); + org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription(); + + // optional .pulsar.proto.TxnAction txn_action = 5; + boolean hasTxnAction(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction(); + } + public static final class CommandEndTxnOnSubscription extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnOnSubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxnOnSubscription.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxnOnSubscription(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxnOnSubscription newObject(Handle handle) { + return new CommandEndTxnOnSubscription(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxnOnSubscription(boolean noInit) {} + + private static final CommandEndTxnOnSubscription defaultInstance; + public static CommandEndTxnOnSubscription getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxnOnSubscription getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.Subscription subscription = 4; + public static final int SUBSCRIPTION_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Subscription subscription_; + public boolean hasSubscription() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription() { + return subscription_; + } + + // optional .pulsar.proto.TxnAction txn_action = 5; + public static final int TXN_ACTION_FIELD_NUMBER = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + memoizedIsInitialized = 0; + return false; + } + if (hasSubscription()) { + if (!getSubscription().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.writeUInt64(1, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, subscription_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeEnum(5, txnAction_.getNumber()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, subscription_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(5, txnAction_.getNumber()); + } + 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription 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.PulsarApi.CommandEndTxnOnSubscription, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000008); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.subscription_ = subscription_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.txnAction_ = txnAction_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasSubscription()) { + mergeSubscription(other.getSubscription()); + } + if (other.hasTxnAction()) { + setTxnAction(other.getTxnAction()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + return false; + } + if (hasSubscription()) { + if (!getSubscription().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 8: { + bitField0_ |= 0x00000001; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(); + if (hasSubscription()) { + subBuilder.mergeFrom(getSubscription()); + } + input.readMessage(subBuilder, extensionRegistry); + setSubscription(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 40: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000010; + txnAction_ = value; + } + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.Subscription subscription = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.Subscription subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); + public boolean hasSubscription() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.Subscription getSubscription() { + return subscription_; + } + public Builder setSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { + if (value == null) { + throw new NullPointerException(); + } + subscription_ = value; + + bitField0_ |= 0x00000008; + return this; + } + public Builder setSubscription( + org.apache.pulsar.common.api.proto.PulsarApi.Subscription.Builder builderForValue) { + subscription_ = builderForValue.build(); + + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeSubscription(org.apache.pulsar.common.api.proto.PulsarApi.Subscription value) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + subscription_ != org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance()) { + subscription_ = + org.apache.pulsar.common.api.proto.PulsarApi.Subscription.newBuilder(subscription_).mergeFrom(value).buildPartial(); + } else { + subscription_ = value; + } + + bitField0_ |= 0x00000008; + return this; + } + public Builder clearSubscription() { + subscription_ = org.apache.pulsar.common.api.proto.PulsarApi.Subscription.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + // optional .pulsar.proto.TxnAction txn_action = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.TxnAction txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + public boolean hasTxnAction() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.TxnAction getTxnAction() { + return txnAction_; + } + public Builder setTxnAction(org.apache.pulsar.common.api.proto.PulsarApi.TxnAction value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + txnAction_ = value; + + return this; + } + public Builder clearTxnAction() { + bitField0_ = (bitField0_ & ~0x00000010); + txnAction_ = org.apache.pulsar.common.api.proto.PulsarApi.TxnAction.COMMIT; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnSubscription) + } + + static { + defaultInstance = new CommandEndTxnOnSubscription(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnSubscription) + } + + public interface CommandEndTxnOnSubscriptionResponseOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required uint64 request_id = 1; + boolean hasRequestId(); + long getRequestId(); + + // optional uint64 txnid_least_bits = 2 [default = 0]; + boolean hasTxnidLeastBits(); + long getTxnidLeastBits(); + + // optional uint64 txnid_most_bits = 3 [default = 0]; + boolean hasTxnidMostBits(); + long getTxnidMostBits(); + + // optional .pulsar.proto.ServerError error = 4; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError(); + + // optional string message = 5; + boolean hasMessage(); + String getMessage(); + } + public static final class CommandEndTxnOnSubscriptionResponse extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements CommandEndTxnOnSubscriptionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use CommandEndTxnOnSubscriptionResponse.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private CommandEndTxnOnSubscriptionResponse(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected CommandEndTxnOnSubscriptionResponse newObject(Handle handle) { + return new CommandEndTxnOnSubscriptionResponse(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private CommandEndTxnOnSubscriptionResponse(boolean noInit) {} + + private static final CommandEndTxnOnSubscriptionResponse defaultInstance; + public static CommandEndTxnOnSubscriptionResponse getDefaultInstance() { + return defaultInstance; + } + + public CommandEndTxnOnSubscriptionResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // required uint64 request_id = 1; + public static final int REQUEST_ID_FIELD_NUMBER = 1; + private long requestId_; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + public static final int TXNID_LEAST_BITS_FIELD_NUMBER = 2; + private long txnidLeastBits_; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + public static final int TXNID_MOST_BITS_FIELD_NUMBER = 3; + private long txnidMostBits_; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + + // optional .pulsar.proto.ServerError error = 4; + public static final int ERROR_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + + // optional string message = 5; + public static final int MESSAGE_FIELD_NUMBER = 5; + private java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + 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)) { + message_ = s; + } + return s; + } + } + private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getMessageBytes() { + java.lang.Object ref = message_; + 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); + message_ = b; + return b; + } else { + return (org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref; + } + } + + private void initFields() { + requestId_ = 0L; + txnidLeastBits_ = 0L; + txnidMostBits_ = 0L; + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + message_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRequestId()) { + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeEnum(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getMessageBytes()); + } + } + + 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, requestId_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(2, txnidLeastBits_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeUInt64Size(3, txnidMostBits_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeEnumSize(4, error_.getNumber()); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeBytesSize(5, getMessageBytes()); + } + 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse 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.PulsarApi.CommandEndTxnOnSubscriptionResponse, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponseOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.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(); + requestId_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + txnidLeastBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + txnidMostBits_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + bitField0_ = (bitField0_ & ~0x00000008); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse build() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse result = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.requestId_ = requestId_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.txnidLeastBits_ = txnidLeastBits_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.txnidMostBits_ = txnidMostBits_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance()) return this; + if (other.hasRequestId()) { + setRequestId(other.getRequestId()); + } + if (other.hasTxnidLeastBits()) { + setTxnidLeastBits(other.getTxnidLeastBits()); + } + if (other.hasTxnidMostBits()) { + setTxnidMostBits(other.getTxnidMostBits()); + } + if (other.hasError()) { + setError(other.getError()); + } + if (other.hasMessage()) { + setMessage(other.getMessage()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasRequestId()) { + + 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; + requestId_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + txnidLeastBits_ = input.readUInt64(); + break; + } + case 24: { + bitField0_ |= 0x00000004; + txnidMostBits_ = input.readUInt64(); + break; + } + case 32: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.ServerError value = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000008; + error_ = value; + } + break; + } + case 42: { + bitField0_ |= 0x00000010; + message_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 request_id = 1; + private long requestId_ ; + public boolean hasRequestId() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRequestId() { + return requestId_; + } + public Builder setRequestId(long value) { + bitField0_ |= 0x00000001; + requestId_ = value; + + return this; + } + public Builder clearRequestId() { + bitField0_ = (bitField0_ & ~0x00000001); + requestId_ = 0L; + + return this; + } + + // optional uint64 txnid_least_bits = 2 [default = 0]; + private long txnidLeastBits_ ; + public boolean hasTxnidLeastBits() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTxnidLeastBits() { + return txnidLeastBits_; + } + public Builder setTxnidLeastBits(long value) { + bitField0_ |= 0x00000002; + txnidLeastBits_ = value; + + return this; + } + public Builder clearTxnidLeastBits() { + bitField0_ = (bitField0_ & ~0x00000002); + txnidLeastBits_ = 0L; + + return this; + } + + // optional uint64 txnid_most_bits = 3 [default = 0]; + private long txnidMostBits_ ; + public boolean hasTxnidMostBits() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTxnidMostBits() { + return txnidMostBits_; + } + public Builder setTxnidMostBits(long value) { + bitField0_ |= 0x00000004; + txnidMostBits_ = value; + + return this; + } + public Builder clearTxnidMostBits() { + bitField0_ = (bitField0_ & ~0x00000004); + txnidMostBits_ = 0L; + + return this; + } + + // optional .pulsar.proto.ServerError error = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.ServerError error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + public boolean hasError() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.ServerError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.ServerError value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + error_ = value; + + return this; + } + public Builder clearError() { + bitField0_ = (bitField0_ & ~0x00000008); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.ServerError.UnknownError; + + return this; + } + + // optional string message = 5; + private java.lang.Object message_ = ""; + public boolean hasMessage() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMessage(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000010; + message_ = value; + + return this; + } + public Builder clearMessage() { + bitField0_ = (bitField0_ & ~0x00000010); + message_ = getDefaultInstance().getMessage(); + + return this; + } + void setMessage(org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString value) { + bitField0_ |= 0x00000010; + message_ = value; + + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandEndTxnOnSubscriptionResponse) + } + + static { + defaultInstance = new CommandEndTxnOnSubscriptionResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.CommandEndTxnOnSubscriptionResponse) + } + + public interface BaseCommandOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // required .pulsar.proto.BaseCommand.Type type = 1; + boolean hasType(); + org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType(); + + // optional .pulsar.proto.CommandConnect connect = 2; + boolean hasConnect(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect(); + + // optional .pulsar.proto.CommandConnected connected = 3; + boolean hasConnected(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected(); + + // optional .pulsar.proto.CommandSubscribe subscribe = 4; + boolean hasSubscribe(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe(); + + // optional .pulsar.proto.CommandProducer producer = 5; + boolean hasProducer(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer(); + + // optional .pulsar.proto.CommandSend send = 6; + boolean hasSend(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend(); + + // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; + boolean hasSendReceipt(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt(); + + // optional .pulsar.proto.CommandSendError send_error = 8; + boolean hasSendError(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError(); + + // optional .pulsar.proto.CommandMessage message = 9; + boolean hasMessage(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage(); + + // optional .pulsar.proto.CommandAck ack = 10; + boolean hasAck(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck(); + + // optional .pulsar.proto.CommandFlow flow = 11; + boolean hasFlow(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow(); + + // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; + boolean hasUnsubscribe(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe(); + + // optional .pulsar.proto.CommandSuccess success = 13; + boolean hasSuccess(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess(); + + // optional .pulsar.proto.CommandError error = 14; + boolean hasError(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError(); + + // optional .pulsar.proto.CommandCloseProducer close_producer = 15; + boolean hasCloseProducer(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer(); + + // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; + boolean hasCloseConsumer(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer(); + + // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; + boolean hasProducerSuccess(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess(); + + // optional .pulsar.proto.CommandPing ping = 18; + boolean hasPing(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing(); + + // optional .pulsar.proto.CommandPong pong = 19; + boolean hasPong(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong(); + + // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; + boolean hasRedeliverUnacknowledgedMessages(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages(); + + // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; + boolean hasPartitionMetadata(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata(); + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; + boolean hasPartitionMetadataResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse(); + + // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; + boolean hasLookupTopic(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic(); + + // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; + boolean hasLookupTopicResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse(); + + // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; + boolean hasConsumerStats(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats(); + + // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; + boolean hasConsumerStatsResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse(); + + // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; + boolean hasReachedEndOfTopic(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic(); + + // optional .pulsar.proto.CommandSeek seek = 28; + boolean hasSeek(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek(); + + // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; + boolean hasGetLastMessageId(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId(); + + // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; + boolean hasGetLastMessageIdResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse(); + + // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; + boolean hasActiveConsumerChange(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange(); + + // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; + boolean hasGetTopicsOfNamespace(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace(); + + // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; + boolean hasGetTopicsOfNamespaceResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse(); + + // optional .pulsar.proto.CommandGetSchema getSchema = 34; + boolean hasGetSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema(); + + // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; + boolean hasGetSchemaResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse(); + + // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; + boolean hasAuthChallenge(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge(); + + // optional .pulsar.proto.CommandAuthResponse authResponse = 37; + boolean hasAuthResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse(); + + // optional .pulsar.proto.CommandAckResponse ackResponse = 38; + boolean hasAckResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse(); + + // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; + boolean hasGetOrCreateSchema(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema(); + + // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; + boolean hasGetOrCreateSchemaResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse(); + + // optional .pulsar.proto.CommandNewTxn newTxn = 50; + boolean hasNewTxn(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn(); + + // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; + boolean hasNewTxnResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse(); + + // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; + boolean hasAddPartitionToTxn(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn(); + + // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; + boolean hasAddPartitionToTxnResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse(); + + // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; + boolean hasAddSubscriptionToTxn(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn(); + + // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; + boolean hasAddSubscriptionToTxnResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse(); + + // optional .pulsar.proto.CommandEndTxn endTxn = 56; + boolean hasEndTxn(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn(); + + // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; + boolean hasEndTxnResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse(); + + // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; + boolean hasEndTxnOnPartition(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition(); + + // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; + boolean hasEndTxnOnPartitionResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse(); + + // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; + boolean hasEndTxnOnSubscription(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription(); + + // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; + boolean hasEndTxnOnSubscriptionResponse(); + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse(); + } + public static final class BaseCommand extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements BaseCommandOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use BaseCommand.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private BaseCommand(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected BaseCommand newObject(Handle handle) { + return new BaseCommand(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.bitField1_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private BaseCommand(boolean noInit) {} + + private static final BaseCommand defaultInstance; + public static BaseCommand getDefaultInstance() { + return defaultInstance; + } + + public BaseCommand getDefaultInstanceForType() { + return defaultInstance; + } + + public enum Type + implements org.apache.pulsar.shaded.com.google.protobuf.v241.Internal.EnumLite { + CONNECT(0, 2), + CONNECTED(1, 3), + SUBSCRIBE(2, 4), + PRODUCER(3, 5), + SEND(4, 6), + SEND_RECEIPT(5, 7), + SEND_ERROR(6, 8), + MESSAGE(7, 9), + ACK(8, 10), + FLOW(9, 11), + UNSUBSCRIBE(10, 12), + SUCCESS(11, 13), + ERROR(12, 14), + CLOSE_PRODUCER(13, 15), + CLOSE_CONSUMER(14, 16), + PRODUCER_SUCCESS(15, 17), + PING(16, 18), + PONG(17, 19), + REDELIVER_UNACKNOWLEDGED_MESSAGES(18, 20), + PARTITIONED_METADATA(19, 21), + PARTITIONED_METADATA_RESPONSE(20, 22), + LOOKUP(21, 23), + LOOKUP_RESPONSE(22, 24), + CONSUMER_STATS(23, 25), + CONSUMER_STATS_RESPONSE(24, 26), + REACHED_END_OF_TOPIC(25, 27), + SEEK(26, 28), + GET_LAST_MESSAGE_ID(27, 29), + GET_LAST_MESSAGE_ID_RESPONSE(28, 30), + ACTIVE_CONSUMER_CHANGE(29, 31), + GET_TOPICS_OF_NAMESPACE(30, 32), + GET_TOPICS_OF_NAMESPACE_RESPONSE(31, 33), + GET_SCHEMA(32, 34), + GET_SCHEMA_RESPONSE(33, 35), + AUTH_CHALLENGE(34, 36), + AUTH_RESPONSE(35, 37), + ACK_RESPONSE(36, 38), + GET_OR_CREATE_SCHEMA(37, 39), + GET_OR_CREATE_SCHEMA_RESPONSE(38, 40), + NEW_TXN(39, 50), + NEW_TXN_RESPONSE(40, 51), + ADD_PARTITION_TO_TXN(41, 52), + ADD_PARTITION_TO_TXN_RESPONSE(42, 53), + ADD_SUBSCRIPTION_TO_TXN(43, 54), + ADD_SUBSCRIPTION_TO_TXN_RESPONSE(44, 55), + END_TXN(45, 56), + END_TXN_RESPONSE(46, 57), + END_TXN_ON_PARTITION(47, 58), + END_TXN_ON_PARTITION_RESPONSE(48, 59), + END_TXN_ON_SUBSCRIPTION(49, 60), + END_TXN_ON_SUBSCRIPTION_RESPONSE(50, 61), + ; + + public static final int CONNECT_VALUE = 2; + public static final int CONNECTED_VALUE = 3; + public static final int SUBSCRIBE_VALUE = 4; + public static final int PRODUCER_VALUE = 5; + public static final int SEND_VALUE = 6; + public static final int SEND_RECEIPT_VALUE = 7; + public static final int SEND_ERROR_VALUE = 8; + public static final int MESSAGE_VALUE = 9; + public static final int ACK_VALUE = 10; + public static final int FLOW_VALUE = 11; + public static final int UNSUBSCRIBE_VALUE = 12; + public static final int SUCCESS_VALUE = 13; + public static final int ERROR_VALUE = 14; + public static final int CLOSE_PRODUCER_VALUE = 15; + public static final int CLOSE_CONSUMER_VALUE = 16; + public static final int PRODUCER_SUCCESS_VALUE = 17; + public static final int PING_VALUE = 18; + public static final int PONG_VALUE = 19; + public static final int REDELIVER_UNACKNOWLEDGED_MESSAGES_VALUE = 20; + public static final int PARTITIONED_METADATA_VALUE = 21; + public static final int PARTITIONED_METADATA_RESPONSE_VALUE = 22; + public static final int LOOKUP_VALUE = 23; + public static final int LOOKUP_RESPONSE_VALUE = 24; + public static final int CONSUMER_STATS_VALUE = 25; + public static final int CONSUMER_STATS_RESPONSE_VALUE = 26; + public static final int REACHED_END_OF_TOPIC_VALUE = 27; + public static final int SEEK_VALUE = 28; + public static final int GET_LAST_MESSAGE_ID_VALUE = 29; + public static final int GET_LAST_MESSAGE_ID_RESPONSE_VALUE = 30; + public static final int ACTIVE_CONSUMER_CHANGE_VALUE = 31; + public static final int GET_TOPICS_OF_NAMESPACE_VALUE = 32; + public static final int GET_TOPICS_OF_NAMESPACE_RESPONSE_VALUE = 33; + public static final int GET_SCHEMA_VALUE = 34; + public static final int GET_SCHEMA_RESPONSE_VALUE = 35; + public static final int AUTH_CHALLENGE_VALUE = 36; + public static final int AUTH_RESPONSE_VALUE = 37; + public static final int ACK_RESPONSE_VALUE = 38; + public static final int GET_OR_CREATE_SCHEMA_VALUE = 39; + public static final int GET_OR_CREATE_SCHEMA_RESPONSE_VALUE = 40; + public static final int NEW_TXN_VALUE = 50; + public static final int NEW_TXN_RESPONSE_VALUE = 51; + public static final int ADD_PARTITION_TO_TXN_VALUE = 52; + public static final int ADD_PARTITION_TO_TXN_RESPONSE_VALUE = 53; + public static final int ADD_SUBSCRIPTION_TO_TXN_VALUE = 54; + public static final int ADD_SUBSCRIPTION_TO_TXN_RESPONSE_VALUE = 55; + public static final int END_TXN_VALUE = 56; + public static final int END_TXN_RESPONSE_VALUE = 57; + public static final int END_TXN_ON_PARTITION_VALUE = 58; + public static final int END_TXN_ON_PARTITION_RESPONSE_VALUE = 59; + public static final int END_TXN_ON_SUBSCRIPTION_VALUE = 60; + public static final int END_TXN_ON_SUBSCRIPTION_RESPONSE_VALUE = 61; + + + public final int getNumber() { return value; } + + public static Type valueOf(int value) { + switch (value) { + case 2: return CONNECT; + case 3: return CONNECTED; + case 4: return SUBSCRIBE; + case 5: return PRODUCER; + case 6: return SEND; + case 7: return SEND_RECEIPT; + case 8: return SEND_ERROR; + case 9: return MESSAGE; + case 10: return ACK; + case 11: return FLOW; + case 12: return UNSUBSCRIBE; + case 13: return SUCCESS; + case 14: return ERROR; + case 15: return CLOSE_PRODUCER; + case 16: return CLOSE_CONSUMER; + case 17: return PRODUCER_SUCCESS; + case 18: return PING; + case 19: return PONG; + case 20: return REDELIVER_UNACKNOWLEDGED_MESSAGES; + case 21: return PARTITIONED_METADATA; + case 22: return PARTITIONED_METADATA_RESPONSE; + case 23: return LOOKUP; + case 24: return LOOKUP_RESPONSE; + case 25: return CONSUMER_STATS; + case 26: return CONSUMER_STATS_RESPONSE; + case 27: return REACHED_END_OF_TOPIC; + case 28: return SEEK; + case 29: return GET_LAST_MESSAGE_ID; + case 30: return GET_LAST_MESSAGE_ID_RESPONSE; + case 31: return ACTIVE_CONSUMER_CHANGE; + case 32: return GET_TOPICS_OF_NAMESPACE; + case 33: return GET_TOPICS_OF_NAMESPACE_RESPONSE; + case 34: return GET_SCHEMA; + case 35: return GET_SCHEMA_RESPONSE; + case 36: return AUTH_CHALLENGE; + case 37: return AUTH_RESPONSE; + case 38: return ACK_RESPONSE; + case 39: return GET_OR_CREATE_SCHEMA; + case 40: return GET_OR_CREATE_SCHEMA_RESPONSE; + case 50: return NEW_TXN; + case 51: return NEW_TXN_RESPONSE; + case 52: return ADD_PARTITION_TO_TXN; + case 53: return ADD_PARTITION_TO_TXN_RESPONSE; + case 54: return ADD_SUBSCRIPTION_TO_TXN; + case 55: return ADD_SUBSCRIPTION_TO_TXN_RESPONSE; + case 56: return END_TXN; + case 57: return END_TXN_RESPONSE; + case 58: return END_TXN_ON_PARTITION; + case 59: return END_TXN_ON_PARTITION_RESPONSE; + case 60: return END_TXN_ON_SUBSCRIPTION; + case 61: return END_TXN_ON_SUBSCRIPTION_RESPONSE; + 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 Type findValueByNumber(int number) { + return Type.valueOf(number); + } + }; + + private final int value; + + private Type(int index, int value) { + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:pulsar.proto.BaseCommand.Type) + } + + private int bitField0_; + private int bitField1_; + // required .pulsar.proto.BaseCommand.Type type = 1; + public static final int TYPE_FIELD_NUMBER = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type type_; + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType() { + return type_; + } + + // optional .pulsar.proto.CommandConnect connect = 2; + public static final int CONNECT_FIELD_NUMBER = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect connect_; + public boolean hasConnect() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect() { + return connect_; + } + + // optional .pulsar.proto.CommandConnected connected = 3; + public static final int CONNECTED_FIELD_NUMBER = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected connected_; + public boolean hasConnected() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected() { + return connected_; + } + + // optional .pulsar.proto.CommandSubscribe subscribe = 4; + public static final int SUBSCRIBE_FIELD_NUMBER = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe subscribe_; + public boolean hasSubscribe() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe() { + return subscribe_; + } + + // optional .pulsar.proto.CommandProducer producer = 5; + public static final int PRODUCER_FIELD_NUMBER = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer producer_; + public boolean hasProducer() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer() { + return producer_; + } + + // optional .pulsar.proto.CommandSend send = 6; + public static final int SEND_FIELD_NUMBER = 6; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend send_; + public boolean hasSend() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend() { + return send_; + } + + // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; + public static final int SEND_RECEIPT_FIELD_NUMBER = 7; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt sendReceipt_; + public boolean hasSendReceipt() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt() { + return sendReceipt_; + } + + // optional .pulsar.proto.CommandSendError send_error = 8; + public static final int SEND_ERROR_FIELD_NUMBER = 8; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError sendError_; + public boolean hasSendError() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError() { + return sendError_; + } + + // optional .pulsar.proto.CommandMessage message = 9; + public static final int MESSAGE_FIELD_NUMBER = 9; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage() { + return message_; + } + + // optional .pulsar.proto.CommandAck ack = 10; + public static final int ACK_FIELD_NUMBER = 10; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck ack_; + public boolean hasAck() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck() { + return ack_; + } + + // optional .pulsar.proto.CommandFlow flow = 11; + public static final int FLOW_FIELD_NUMBER = 11; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow flow_; + public boolean hasFlow() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow() { + return flow_; + } + + // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; + public static final int UNSUBSCRIBE_FIELD_NUMBER = 12; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe unsubscribe_; + public boolean hasUnsubscribe() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe() { + return unsubscribe_; + } + + // optional .pulsar.proto.CommandSuccess success = 13; + public static final int SUCCESS_FIELD_NUMBER = 13; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess success_; + public boolean hasSuccess() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess() { + return success_; + } + + // optional .pulsar.proto.CommandError error = 14; + public static final int ERROR_FIELD_NUMBER = 14; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandError error_; + public boolean hasError() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError() { + return error_; + } + + // optional .pulsar.proto.CommandCloseProducer close_producer = 15; + public static final int CLOSE_PRODUCER_FIELD_NUMBER = 15; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer closeProducer_; + public boolean hasCloseProducer() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer() { + return closeProducer_; + } + + // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; + public static final int CLOSE_CONSUMER_FIELD_NUMBER = 16; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer closeConsumer_; + public boolean hasCloseConsumer() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer() { + return closeConsumer_; + } + + // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; + public static final int PRODUCER_SUCCESS_FIELD_NUMBER = 17; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess producerSuccess_; + public boolean hasProducerSuccess() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess() { + return producerSuccess_; + } + + // optional .pulsar.proto.CommandPing ping = 18; + public static final int PING_FIELD_NUMBER = 18; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing ping_; + public boolean hasPing() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing() { + return ping_; + } + + // optional .pulsar.proto.CommandPong pong = 19; + public static final int PONG_FIELD_NUMBER = 19; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong pong_; + public boolean hasPong() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong() { + return pong_; + } + + // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; + public static final int REDELIVERUNACKNOWLEDGEDMESSAGES_FIELD_NUMBER = 20; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages_; + public boolean hasRedeliverUnacknowledgedMessages() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages() { + return redeliverUnacknowledgedMessages_; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; + public static final int PARTITIONMETADATA_FIELD_NUMBER = 21; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata partitionMetadata_; + public boolean hasPartitionMetadata() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata() { + return partitionMetadata_; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; + public static final int PARTITIONMETADATARESPONSE_FIELD_NUMBER = 22; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse partitionMetadataResponse_; + public boolean hasPartitionMetadataResponse() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse() { + return partitionMetadataResponse_; + } + + // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; + public static final int LOOKUPTOPIC_FIELD_NUMBER = 23; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic lookupTopic_; + public boolean hasLookupTopic() { + return ((bitField0_ & 0x00400000) == 0x00400000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic() { + return lookupTopic_; + } + + // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; + public static final int LOOKUPTOPICRESPONSE_FIELD_NUMBER = 24; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse lookupTopicResponse_; + public boolean hasLookupTopicResponse() { + return ((bitField0_ & 0x00800000) == 0x00800000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse() { + return lookupTopicResponse_; + } + + // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; + public static final int CONSUMERSTATS_FIELD_NUMBER = 25; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_; + public boolean hasConsumerStats() { + return ((bitField0_ & 0x01000000) == 0x01000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() { + return consumerStats_; + } + + // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; + public static final int CONSUMERSTATSRESPONSE_FIELD_NUMBER = 26; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_; + public boolean hasConsumerStatsResponse() { + return ((bitField0_ & 0x02000000) == 0x02000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() { + return consumerStatsResponse_; + } + + // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; + public static final int REACHEDENDOFTOPIC_FIELD_NUMBER = 27; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic reachedEndOfTopic_; + public boolean hasReachedEndOfTopic() { + return ((bitField0_ & 0x04000000) == 0x04000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic() { + return reachedEndOfTopic_; + } + + // optional .pulsar.proto.CommandSeek seek = 28; + public static final int SEEK_FIELD_NUMBER = 28; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek seek_; + public boolean hasSeek() { + return ((bitField0_ & 0x08000000) == 0x08000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek() { + return seek_; + } + + // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; + public static final int GETLASTMESSAGEID_FIELD_NUMBER = 29; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getLastMessageId_; + public boolean hasGetLastMessageId() { + return ((bitField0_ & 0x10000000) == 0x10000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId() { + return getLastMessageId_; + } + + // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; + public static final int GETLASTMESSAGEIDRESPONSE_FIELD_NUMBER = 30; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getLastMessageIdResponse_; + public boolean hasGetLastMessageIdResponse() { + return ((bitField0_ & 0x20000000) == 0x20000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse() { + return getLastMessageIdResponse_; + } + + // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; + public static final int ACTIVE_CONSUMER_CHANGE_FIELD_NUMBER = 31; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange activeConsumerChange_; + public boolean hasActiveConsumerChange() { + return ((bitField0_ & 0x40000000) == 0x40000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange() { + return activeConsumerChange_; + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; + public static final int GETTOPICSOFNAMESPACE_FIELD_NUMBER = 32; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getTopicsOfNamespace_; + public boolean hasGetTopicsOfNamespace() { + return ((bitField0_ & 0x80000000) == 0x80000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace() { + return getTopicsOfNamespace_; + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; + public static final int GETTOPICSOFNAMESPACERESPONSE_FIELD_NUMBER = 33; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse_; + public boolean hasGetTopicsOfNamespaceResponse() { + return ((bitField1_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse() { + return getTopicsOfNamespaceResponse_; + } + + // optional .pulsar.proto.CommandGetSchema getSchema = 34; + public static final int GETSCHEMA_FIELD_NUMBER = 34; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getSchema_; + public boolean hasGetSchema() { + return ((bitField1_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema() { + return getSchema_; + } + + // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; + public static final int GETSCHEMARESPONSE_FIELD_NUMBER = 35; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getSchemaResponse_; + public boolean hasGetSchemaResponse() { + return ((bitField1_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse() { + return getSchemaResponse_; + } + + // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; + public static final int AUTHCHALLENGE_FIELD_NUMBER = 36; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge authChallenge_; + public boolean hasAuthChallenge() { + return ((bitField1_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge() { + return authChallenge_; + } + + // optional .pulsar.proto.CommandAuthResponse authResponse = 37; + public static final int AUTHRESPONSE_FIELD_NUMBER = 37; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse authResponse_; + public boolean hasAuthResponse() { + return ((bitField1_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse() { + return authResponse_; + } + + // optional .pulsar.proto.CommandAckResponse ackResponse = 38; + public static final int ACKRESPONSE_FIELD_NUMBER = 38; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse ackResponse_; + public boolean hasAckResponse() { + return ((bitField1_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse() { + return ackResponse_; + } + + // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; + public static final int GETORCREATESCHEMA_FIELD_NUMBER = 39; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getOrCreateSchema_; + public boolean hasGetOrCreateSchema() { + return ((bitField1_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema() { + return getOrCreateSchema_; + } + + // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; + public static final int GETORCREATESCHEMARESPONSE_FIELD_NUMBER = 40; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse_; + public boolean hasGetOrCreateSchemaResponse() { + return ((bitField1_ & 0x00000080) == 0x00000080); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse() { + return getOrCreateSchemaResponse_; + } + + // optional .pulsar.proto.CommandNewTxn newTxn = 50; + public static final int NEWTXN_FIELD_NUMBER = 50; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn newTxn_; + public boolean hasNewTxn() { + return ((bitField1_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn() { + return newTxn_; + } + + // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; + public static final int NEWTXNRESPONSE_FIELD_NUMBER = 51; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse newTxnResponse_; + public boolean hasNewTxnResponse() { + return ((bitField1_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse() { + return newTxnResponse_; + } + + // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; + public static final int ADDPARTITIONTOTXN_FIELD_NUMBER = 52; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn addPartitionToTxn_; + public boolean hasAddPartitionToTxn() { + return ((bitField1_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn() { + return addPartitionToTxn_; + } + + // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; + public static final int ADDPARTITIONTOTXNRESPONSE_FIELD_NUMBER = 53; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse addPartitionToTxnResponse_; + public boolean hasAddPartitionToTxnResponse() { + return ((bitField1_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse() { + return addPartitionToTxnResponse_; + } + + // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; + public static final int ADDSUBSCRIPTIONTOTXN_FIELD_NUMBER = 54; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn addSubscriptionToTxn_; + public boolean hasAddSubscriptionToTxn() { + return ((bitField1_ & 0x00001000) == 0x00001000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn() { + return addSubscriptionToTxn_; + } + + // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; + public static final int ADDSUBSCRIPTIONTOTXNRESPONSE_FIELD_NUMBER = 55; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse_; + public boolean hasAddSubscriptionToTxnResponse() { + return ((bitField1_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse() { + return addSubscriptionToTxnResponse_; + } + + // optional .pulsar.proto.CommandEndTxn endTxn = 56; + public static final int ENDTXN_FIELD_NUMBER = 56; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn endTxn_; + public boolean hasEndTxn() { + return ((bitField1_ & 0x00004000) == 0x00004000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn() { + return endTxn_; + } + + // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; + public static final int ENDTXNRESPONSE_FIELD_NUMBER = 57; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse endTxnResponse_; + public boolean hasEndTxnResponse() { + return ((bitField1_ & 0x00008000) == 0x00008000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse() { + return endTxnResponse_; + } + + // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; + public static final int ENDTXNONPARTITION_FIELD_NUMBER = 58; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition endTxnOnPartition_; + public boolean hasEndTxnOnPartition() { + return ((bitField1_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition() { + return endTxnOnPartition_; + } + + // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; + public static final int ENDTXNONPARTITIONRESPONSE_FIELD_NUMBER = 59; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse_; + public boolean hasEndTxnOnPartitionResponse() { + return ((bitField1_ & 0x00020000) == 0x00020000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse() { + return endTxnOnPartitionResponse_; + } + + // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; + public static final int ENDTXNONSUBSCRIPTION_FIELD_NUMBER = 60; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription endTxnOnSubscription_; + public boolean hasEndTxnOnSubscription() { + return ((bitField1_ & 0x00040000) == 0x00040000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription() { + return endTxnOnSubscription_; + } + + // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; + public static final int ENDTXNONSUBSCRIPTIONRESPONSE_FIELD_NUMBER = 61; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse_; + public boolean hasEndTxnOnSubscriptionResponse() { + return ((bitField1_ & 0x00080000) == 0x00080000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse() { + return endTxnOnSubscriptionResponse_; + } + + private void initFields() { + type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; + connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); + connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); + subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); + producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); + send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); + sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); + sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); + message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); + ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); + flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); + unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); + success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); + closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); + closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); + producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); + ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); + pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); + redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); + partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); + partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); + lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); + lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); + consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); + consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); + reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); + seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); + getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); + getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); + activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); + getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); + getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); + getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); + getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); + authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); + authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); + ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); + getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); + getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); + newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); + newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); + addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); + addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); + addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); + addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); + endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); + endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); + endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); + endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); + endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); + endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasType()) { + memoizedIsInitialized = 0; + return false; + } + if (hasConnect()) { + if (!getConnect().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasConnected()) { + if (!getConnected().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSubscribe()) { + if (!getSubscribe().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasProducer()) { + if (!getProducer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSend()) { + if (!getSend().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSendReceipt()) { + if (!getSendReceipt().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSendError()) { + if (!getSendError().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasMessage()) { + if (!getMessage().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAck()) { + if (!getAck().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasFlow()) { + if (!getFlow().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasUnsubscribe()) { + if (!getUnsubscribe().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSuccess()) { + if (!getSuccess().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasError()) { + if (!getError().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasCloseProducer()) { + if (!getCloseProducer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasCloseConsumer()) { + if (!getCloseConsumer().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasProducerSuccess()) { + if (!getProducerSuccess().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasRedeliverUnacknowledgedMessages()) { + if (!getRedeliverUnacknowledgedMessages().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasPartitionMetadata()) { + if (!getPartitionMetadata().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasPartitionMetadataResponse()) { + if (!getPartitionMetadataResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasLookupTopic()) { + if (!getLookupTopic().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasLookupTopicResponse()) { + if (!getLookupTopicResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasConsumerStats()) { + if (!getConsumerStats().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasConsumerStatsResponse()) { + if (!getConsumerStatsResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasReachedEndOfTopic()) { + if (!getReachedEndOfTopic().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasSeek()) { + if (!getSeek().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetLastMessageId()) { + if (!getGetLastMessageId().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetLastMessageIdResponse()) { + if (!getGetLastMessageIdResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasActiveConsumerChange()) { + if (!getActiveConsumerChange().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetTopicsOfNamespace()) { + if (!getGetTopicsOfNamespace().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetTopicsOfNamespaceResponse()) { + if (!getGetTopicsOfNamespaceResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetSchema()) { + if (!getGetSchema().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetSchemaResponse()) { + if (!getGetSchemaResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAckResponse()) { + if (!getAckResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetOrCreateSchema()) { + if (!getGetOrCreateSchema().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasGetOrCreateSchemaResponse()) { + if (!getGetOrCreateSchemaResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasNewTxn()) { + if (!getNewTxn().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasNewTxnResponse()) { + if (!getNewTxnResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAddPartitionToTxn()) { + if (!getAddPartitionToTxn().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAddPartitionToTxnResponse()) { + if (!getAddPartitionToTxnResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAddSubscriptionToTxn()) { + if (!getAddSubscriptionToTxn().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasAddSubscriptionToTxnResponse()) { + if (!getAddSubscriptionToTxnResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxn()) { + if (!getEndTxn().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxnResponse()) { + if (!getEndTxnResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxnOnPartition()) { + if (!getEndTxnOnPartition().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxnOnPartitionResponse()) { + if (!getEndTxnOnPartitionResponse().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxnOnSubscription()) { + if (!getEndTxnOnSubscription().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + if (hasEndTxnOnSubscriptionResponse()) { + if (!getEndTxnOnSubscriptionResponse().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.writeEnum(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, connect_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, connected_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, subscribe_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(5, producer_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeMessage(6, send_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeMessage(7, sendReceipt_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeMessage(8, sendError_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeMessage(9, message_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeMessage(10, ack_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeMessage(11, flow_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + output.writeMessage(12, unsubscribe_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + output.writeMessage(13, success_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + output.writeMessage(14, error_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + output.writeMessage(15, closeProducer_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + output.writeMessage(16, closeConsumer_); + } + if (((bitField0_ & 0x00010000) == 0x00010000)) { + output.writeMessage(17, producerSuccess_); + } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + output.writeMessage(18, ping_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + output.writeMessage(19, pong_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + output.writeMessage(20, redeliverUnacknowledgedMessages_); + } + if (((bitField0_ & 0x00100000) == 0x00100000)) { + output.writeMessage(21, partitionMetadata_); + } + if (((bitField0_ & 0x00200000) == 0x00200000)) { + output.writeMessage(22, partitionMetadataResponse_); + } + if (((bitField0_ & 0x00400000) == 0x00400000)) { + output.writeMessage(23, lookupTopic_); + } + if (((bitField0_ & 0x00800000) == 0x00800000)) { + output.writeMessage(24, lookupTopicResponse_); + } + if (((bitField0_ & 0x01000000) == 0x01000000)) { + output.writeMessage(25, consumerStats_); + } + if (((bitField0_ & 0x02000000) == 0x02000000)) { + output.writeMessage(26, consumerStatsResponse_); + } + if (((bitField0_ & 0x04000000) == 0x04000000)) { + output.writeMessage(27, reachedEndOfTopic_); + } + if (((bitField0_ & 0x08000000) == 0x08000000)) { + output.writeMessage(28, seek_); + } + if (((bitField0_ & 0x10000000) == 0x10000000)) { + output.writeMessage(29, getLastMessageId_); + } + if (((bitField0_ & 0x20000000) == 0x20000000)) { + output.writeMessage(30, getLastMessageIdResponse_); + } + if (((bitField0_ & 0x40000000) == 0x40000000)) { + output.writeMessage(31, activeConsumerChange_); + } + if (((bitField0_ & 0x80000000) == 0x80000000)) { + output.writeMessage(32, getTopicsOfNamespace_); + } + if (((bitField1_ & 0x00000001) == 0x00000001)) { + output.writeMessage(33, getTopicsOfNamespaceResponse_); + } + if (((bitField1_ & 0x00000002) == 0x00000002)) { + output.writeMessage(34, getSchema_); + } + if (((bitField1_ & 0x00000004) == 0x00000004)) { + output.writeMessage(35, getSchemaResponse_); + } + if (((bitField1_ & 0x00000008) == 0x00000008)) { + output.writeMessage(36, authChallenge_); + } + if (((bitField1_ & 0x00000010) == 0x00000010)) { + output.writeMessage(37, authResponse_); + } + if (((bitField1_ & 0x00000020) == 0x00000020)) { + output.writeMessage(38, ackResponse_); + } + if (((bitField1_ & 0x00000040) == 0x00000040)) { + output.writeMessage(39, getOrCreateSchema_); + } + if (((bitField1_ & 0x00000080) == 0x00000080)) { + output.writeMessage(40, getOrCreateSchemaResponse_); + } + if (((bitField1_ & 0x00000100) == 0x00000100)) { + output.writeMessage(50, newTxn_); + } + if (((bitField1_ & 0x00000200) == 0x00000200)) { + output.writeMessage(51, newTxnResponse_); + } + if (((bitField1_ & 0x00000400) == 0x00000400)) { + output.writeMessage(52, addPartitionToTxn_); + } + if (((bitField1_ & 0x00000800) == 0x00000800)) { + output.writeMessage(53, addPartitionToTxnResponse_); + } + if (((bitField1_ & 0x00001000) == 0x00001000)) { + output.writeMessage(54, addSubscriptionToTxn_); + } + if (((bitField1_ & 0x00002000) == 0x00002000)) { + output.writeMessage(55, addSubscriptionToTxnResponse_); + } + if (((bitField1_ & 0x00004000) == 0x00004000)) { + output.writeMessage(56, endTxn_); + } + if (((bitField1_ & 0x00008000) == 0x00008000)) { + output.writeMessage(57, endTxnResponse_); + } + if (((bitField1_ & 0x00010000) == 0x00010000)) { + output.writeMessage(58, endTxnOnPartition_); + } + if (((bitField1_ & 0x00020000) == 0x00020000)) { + output.writeMessage(59, endTxnOnPartitionResponse_); + } + if (((bitField1_ & 0x00040000) == 0x00040000)) { + output.writeMessage(60, endTxnOnSubscription_); + } + if (((bitField1_ & 0x00080000) == 0x00080000)) { + output.writeMessage(61, endTxnOnSubscriptionResponse_); + } + } + + 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 + .computeEnumSize(1, type_.getNumber()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(2, connect_); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(3, connected_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(4, subscribe_); + } + if (((bitField0_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(5, producer_); + } + if (((bitField0_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(6, send_); + } + if (((bitField0_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(7, sendReceipt_); + } + if (((bitField0_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(8, sendError_); + } + if (((bitField0_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(9, message_); + } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(10, ack_); + } + if (((bitField0_ & 0x00000400) == 0x00000400)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(11, flow_); + } + if (((bitField0_ & 0x00000800) == 0x00000800)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(12, unsubscribe_); + } + if (((bitField0_ & 0x00001000) == 0x00001000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(13, success_); + } + if (((bitField0_ & 0x00002000) == 0x00002000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(14, error_); + } + if (((bitField0_ & 0x00004000) == 0x00004000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(15, closeProducer_); + } + if (((bitField0_ & 0x00008000) == 0x00008000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(16, closeConsumer_); + } + if (((bitField0_ & 0x00010000) == 0x00010000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(17, producerSuccess_); + } + if (((bitField0_ & 0x00020000) == 0x00020000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(18, ping_); + } + if (((bitField0_ & 0x00040000) == 0x00040000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(19, pong_); + } + if (((bitField0_ & 0x00080000) == 0x00080000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(20, redeliverUnacknowledgedMessages_); + } + if (((bitField0_ & 0x00100000) == 0x00100000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(21, partitionMetadata_); + } + if (((bitField0_ & 0x00200000) == 0x00200000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(22, partitionMetadataResponse_); + } + if (((bitField0_ & 0x00400000) == 0x00400000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(23, lookupTopic_); + } + if (((bitField0_ & 0x00800000) == 0x00800000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(24, lookupTopicResponse_); + } + if (((bitField0_ & 0x01000000) == 0x01000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(25, consumerStats_); + } + if (((bitField0_ & 0x02000000) == 0x02000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(26, consumerStatsResponse_); + } + if (((bitField0_ & 0x04000000) == 0x04000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(27, reachedEndOfTopic_); + } + if (((bitField0_ & 0x08000000) == 0x08000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(28, seek_); + } + if (((bitField0_ & 0x10000000) == 0x10000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(29, getLastMessageId_); + } + if (((bitField0_ & 0x20000000) == 0x20000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(30, getLastMessageIdResponse_); + } + if (((bitField0_ & 0x40000000) == 0x40000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(31, activeConsumerChange_); + } + if (((bitField0_ & 0x80000000) == 0x80000000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(32, getTopicsOfNamespace_); + } + if (((bitField1_ & 0x00000001) == 0x00000001)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(33, getTopicsOfNamespaceResponse_); + } + if (((bitField1_ & 0x00000002) == 0x00000002)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(34, getSchema_); + } + if (((bitField1_ & 0x00000004) == 0x00000004)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(35, getSchemaResponse_); + } + if (((bitField1_ & 0x00000008) == 0x00000008)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(36, authChallenge_); + } + if (((bitField1_ & 0x00000010) == 0x00000010)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(37, authResponse_); + } + if (((bitField1_ & 0x00000020) == 0x00000020)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(38, ackResponse_); + } + if (((bitField1_ & 0x00000040) == 0x00000040)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(39, getOrCreateSchema_); + } + if (((bitField1_ & 0x00000080) == 0x00000080)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(40, getOrCreateSchemaResponse_); + } + if (((bitField1_ & 0x00000100) == 0x00000100)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(50, newTxn_); + } + if (((bitField1_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(51, newTxnResponse_); + } + if (((bitField1_ & 0x00000400) == 0x00000400)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(52, addPartitionToTxn_); + } + if (((bitField1_ & 0x00000800) == 0x00000800)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(53, addPartitionToTxnResponse_); + } + if (((bitField1_ & 0x00001000) == 0x00001000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(54, addSubscriptionToTxn_); + } + if (((bitField1_ & 0x00002000) == 0x00002000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(55, addSubscriptionToTxnResponse_); + } + if (((bitField1_ & 0x00004000) == 0x00004000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(56, endTxn_); + } + if (((bitField1_ & 0x00008000) == 0x00008000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(57, endTxnResponse_); + } + if (((bitField1_ & 0x00010000) == 0x00010000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(58, endTxnOnPartition_); + } + if (((bitField1_ & 0x00020000) == 0x00020000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(59, endTxnOnPartitionResponse_); + } + if (((bitField1_ & 0x00040000) == 0x00040000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(60, endTxnOnSubscription_); + } + if (((bitField1_ & 0x00080000) == 0x00080000)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(61, endTxnOnSubscriptionResponse_); + } + 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand 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.PulsarApi.BaseCommand, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.BaseCommandOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.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(); + type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; + bitField0_ = (bitField0_ & ~0x00000001); + connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000002); + connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000004); + subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000008); + producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000010); + send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000020); + sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000040); + sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000080); + message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000100); + ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000200); + flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000400); + unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000800); + success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00001000); + error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00002000); + closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00004000); + closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00008000); + producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00010000); + ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00020000); + pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00040000); + redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00080000); + partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00100000); + partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00200000); + lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00400000); + lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00800000); + consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x01000000); + consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x02000000); + reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x04000000); + seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x08000000); + getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x10000000); + getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x20000000); + activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x40000000); + getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x80000000); + getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000001); + getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000002); + getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000004); + authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000008); + authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000010); + ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000020); + getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000040); + getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000080); + newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000100); + newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000200); + addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000400); + addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00000800); + addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00001000); + addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00002000); + endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00004000); + endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00008000); + endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00010000); + endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00020000); + endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00040000); + endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); + bitField1_ = (bitField1_ & ~0x00080000); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand build() { + org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand result = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int from_bitField1_ = bitField1_; + int to_bitField0_ = 0; + int to_bitField1_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.type_ = type_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.connect_ = connect_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.connected_ = connected_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.subscribe_ = subscribe_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.producer_ = producer_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.send_ = send_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.sendReceipt_ = sendReceipt_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + result.sendError_ = sendError_; + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.ack_ = ack_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000400; + } + result.flow_ = flow_; + if (((from_bitField0_ & 0x00000800) == 0x00000800)) { + to_bitField0_ |= 0x00000800; + } + result.unsubscribe_ = unsubscribe_; + if (((from_bitField0_ & 0x00001000) == 0x00001000)) { + to_bitField0_ |= 0x00001000; + } + result.success_ = success_; + if (((from_bitField0_ & 0x00002000) == 0x00002000)) { + to_bitField0_ |= 0x00002000; + } + result.error_ = error_; + if (((from_bitField0_ & 0x00004000) == 0x00004000)) { + to_bitField0_ |= 0x00004000; + } + result.closeProducer_ = closeProducer_; + if (((from_bitField0_ & 0x00008000) == 0x00008000)) { + to_bitField0_ |= 0x00008000; + } + result.closeConsumer_ = closeConsumer_; + if (((from_bitField0_ & 0x00010000) == 0x00010000)) { + to_bitField0_ |= 0x00010000; + } + result.producerSuccess_ = producerSuccess_; + if (((from_bitField0_ & 0x00020000) == 0x00020000)) { + to_bitField0_ |= 0x00020000; + } + result.ping_ = ping_; + if (((from_bitField0_ & 0x00040000) == 0x00040000)) { + to_bitField0_ |= 0x00040000; + } + result.pong_ = pong_; + if (((from_bitField0_ & 0x00080000) == 0x00080000)) { + to_bitField0_ |= 0x00080000; + } + result.redeliverUnacknowledgedMessages_ = redeliverUnacknowledgedMessages_; + if (((from_bitField0_ & 0x00100000) == 0x00100000)) { + to_bitField0_ |= 0x00100000; + } + result.partitionMetadata_ = partitionMetadata_; + if (((from_bitField0_ & 0x00200000) == 0x00200000)) { + to_bitField0_ |= 0x00200000; + } + result.partitionMetadataResponse_ = partitionMetadataResponse_; + if (((from_bitField0_ & 0x00400000) == 0x00400000)) { + to_bitField0_ |= 0x00400000; + } + result.lookupTopic_ = lookupTopic_; + if (((from_bitField0_ & 0x00800000) == 0x00800000)) { + to_bitField0_ |= 0x00800000; + } + result.lookupTopicResponse_ = lookupTopicResponse_; + if (((from_bitField0_ & 0x01000000) == 0x01000000)) { + to_bitField0_ |= 0x01000000; + } + result.consumerStats_ = consumerStats_; + if (((from_bitField0_ & 0x02000000) == 0x02000000)) { + to_bitField0_ |= 0x02000000; + } + result.consumerStatsResponse_ = consumerStatsResponse_; + if (((from_bitField0_ & 0x04000000) == 0x04000000)) { + to_bitField0_ |= 0x04000000; + } + result.reachedEndOfTopic_ = reachedEndOfTopic_; + if (((from_bitField0_ & 0x08000000) == 0x08000000)) { + to_bitField0_ |= 0x08000000; + } + result.seek_ = seek_; + if (((from_bitField0_ & 0x10000000) == 0x10000000)) { + to_bitField0_ |= 0x10000000; + } + result.getLastMessageId_ = getLastMessageId_; + if (((from_bitField0_ & 0x20000000) == 0x20000000)) { + to_bitField0_ |= 0x20000000; + } + result.getLastMessageIdResponse_ = getLastMessageIdResponse_; + if (((from_bitField0_ & 0x40000000) == 0x40000000)) { + to_bitField0_ |= 0x40000000; + } + result.activeConsumerChange_ = activeConsumerChange_; + if (((from_bitField0_ & 0x80000000) == 0x80000000)) { + to_bitField0_ |= 0x80000000; + } + result.getTopicsOfNamespace_ = getTopicsOfNamespace_; + if (((from_bitField1_ & 0x00000001) == 0x00000001)) { + to_bitField1_ |= 0x00000001; + } + result.getTopicsOfNamespaceResponse_ = getTopicsOfNamespaceResponse_; + if (((from_bitField1_ & 0x00000002) == 0x00000002)) { + to_bitField1_ |= 0x00000002; + } + result.getSchema_ = getSchema_; + if (((from_bitField1_ & 0x00000004) == 0x00000004)) { + to_bitField1_ |= 0x00000004; + } + result.getSchemaResponse_ = getSchemaResponse_; + if (((from_bitField1_ & 0x00000008) == 0x00000008)) { + to_bitField1_ |= 0x00000008; + } + result.authChallenge_ = authChallenge_; + if (((from_bitField1_ & 0x00000010) == 0x00000010)) { + to_bitField1_ |= 0x00000010; + } + result.authResponse_ = authResponse_; + if (((from_bitField1_ & 0x00000020) == 0x00000020)) { + to_bitField1_ |= 0x00000020; + } + result.ackResponse_ = ackResponse_; + if (((from_bitField1_ & 0x00000040) == 0x00000040)) { + to_bitField1_ |= 0x00000040; + } + result.getOrCreateSchema_ = getOrCreateSchema_; + if (((from_bitField1_ & 0x00000080) == 0x00000080)) { + to_bitField1_ |= 0x00000080; + } + result.getOrCreateSchemaResponse_ = getOrCreateSchemaResponse_; + if (((from_bitField1_ & 0x00000100) == 0x00000100)) { + to_bitField1_ |= 0x00000100; + } + result.newTxn_ = newTxn_; + if (((from_bitField1_ & 0x00000200) == 0x00000200)) { + to_bitField1_ |= 0x00000200; + } + result.newTxnResponse_ = newTxnResponse_; + if (((from_bitField1_ & 0x00000400) == 0x00000400)) { + to_bitField1_ |= 0x00000400; + } + result.addPartitionToTxn_ = addPartitionToTxn_; + if (((from_bitField1_ & 0x00000800) == 0x00000800)) { + to_bitField1_ |= 0x00000800; + } + result.addPartitionToTxnResponse_ = addPartitionToTxnResponse_; + if (((from_bitField1_ & 0x00001000) == 0x00001000)) { + to_bitField1_ |= 0x00001000; + } + result.addSubscriptionToTxn_ = addSubscriptionToTxn_; + if (((from_bitField1_ & 0x00002000) == 0x00002000)) { + to_bitField1_ |= 0x00002000; + } + result.addSubscriptionToTxnResponse_ = addSubscriptionToTxnResponse_; + if (((from_bitField1_ & 0x00004000) == 0x00004000)) { + to_bitField1_ |= 0x00004000; + } + result.endTxn_ = endTxn_; + if (((from_bitField1_ & 0x00008000) == 0x00008000)) { + to_bitField1_ |= 0x00008000; + } + result.endTxnResponse_ = endTxnResponse_; + if (((from_bitField1_ & 0x00010000) == 0x00010000)) { + to_bitField1_ |= 0x00010000; + } + result.endTxnOnPartition_ = endTxnOnPartition_; + if (((from_bitField1_ & 0x00020000) == 0x00020000)) { + to_bitField1_ |= 0x00020000; + } + result.endTxnOnPartitionResponse_ = endTxnOnPartitionResponse_; + if (((from_bitField1_ & 0x00040000) == 0x00040000)) { + to_bitField1_ |= 0x00040000; + } + result.endTxnOnSubscription_ = endTxnOnSubscription_; + if (((from_bitField1_ & 0x00080000) == 0x00080000)) { + to_bitField1_ |= 0x00080000; + } + result.endTxnOnSubscriptionResponse_ = endTxnOnSubscriptionResponse_; + result.bitField0_ = to_bitField0_; + result.bitField1_ = to_bitField1_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.getDefaultInstance()) return this; + if (other.hasType()) { + setType(other.getType()); + } + if (other.hasConnect()) { + mergeConnect(other.getConnect()); + } + if (other.hasConnected()) { + mergeConnected(other.getConnected()); + } + if (other.hasSubscribe()) { + mergeSubscribe(other.getSubscribe()); + } + if (other.hasProducer()) { + mergeProducer(other.getProducer()); + } + if (other.hasSend()) { + mergeSend(other.getSend()); + } + if (other.hasSendReceipt()) { + mergeSendReceipt(other.getSendReceipt()); + } + if (other.hasSendError()) { + mergeSendError(other.getSendError()); + } + if (other.hasMessage()) { + mergeMessage(other.getMessage()); + } + if (other.hasAck()) { + mergeAck(other.getAck()); + } + if (other.hasFlow()) { + mergeFlow(other.getFlow()); + } + if (other.hasUnsubscribe()) { + mergeUnsubscribe(other.getUnsubscribe()); + } + if (other.hasSuccess()) { + mergeSuccess(other.getSuccess()); + } + if (other.hasError()) { + mergeError(other.getError()); + } + if (other.hasCloseProducer()) { + mergeCloseProducer(other.getCloseProducer()); + } + if (other.hasCloseConsumer()) { + mergeCloseConsumer(other.getCloseConsumer()); + } + if (other.hasProducerSuccess()) { + mergeProducerSuccess(other.getProducerSuccess()); + } + if (other.hasPing()) { + mergePing(other.getPing()); + } + if (other.hasPong()) { + mergePong(other.getPong()); + } + if (other.hasRedeliverUnacknowledgedMessages()) { + mergeRedeliverUnacknowledgedMessages(other.getRedeliverUnacknowledgedMessages()); + } + if (other.hasPartitionMetadata()) { + mergePartitionMetadata(other.getPartitionMetadata()); + } + if (other.hasPartitionMetadataResponse()) { + mergePartitionMetadataResponse(other.getPartitionMetadataResponse()); + } + if (other.hasLookupTopic()) { + mergeLookupTopic(other.getLookupTopic()); + } + if (other.hasLookupTopicResponse()) { + mergeLookupTopicResponse(other.getLookupTopicResponse()); + } + if (other.hasConsumerStats()) { + mergeConsumerStats(other.getConsumerStats()); + } + if (other.hasConsumerStatsResponse()) { + mergeConsumerStatsResponse(other.getConsumerStatsResponse()); + } + if (other.hasReachedEndOfTopic()) { + mergeReachedEndOfTopic(other.getReachedEndOfTopic()); + } + if (other.hasSeek()) { + mergeSeek(other.getSeek()); + } + if (other.hasGetLastMessageId()) { + mergeGetLastMessageId(other.getGetLastMessageId()); + } + if (other.hasGetLastMessageIdResponse()) { + mergeGetLastMessageIdResponse(other.getGetLastMessageIdResponse()); + } + if (other.hasActiveConsumerChange()) { + mergeActiveConsumerChange(other.getActiveConsumerChange()); + } + if (other.hasGetTopicsOfNamespace()) { + mergeGetTopicsOfNamespace(other.getGetTopicsOfNamespace()); + } + if (other.hasGetTopicsOfNamespaceResponse()) { + mergeGetTopicsOfNamespaceResponse(other.getGetTopicsOfNamespaceResponse()); + } + if (other.hasGetSchema()) { + mergeGetSchema(other.getGetSchema()); + } + if (other.hasGetSchemaResponse()) { + mergeGetSchemaResponse(other.getGetSchemaResponse()); + } + if (other.hasAuthChallenge()) { + mergeAuthChallenge(other.getAuthChallenge()); + } + if (other.hasAuthResponse()) { + mergeAuthResponse(other.getAuthResponse()); + } + if (other.hasAckResponse()) { + mergeAckResponse(other.getAckResponse()); + } + if (other.hasGetOrCreateSchema()) { + mergeGetOrCreateSchema(other.getGetOrCreateSchema()); + } + if (other.hasGetOrCreateSchemaResponse()) { + mergeGetOrCreateSchemaResponse(other.getGetOrCreateSchemaResponse()); + } + if (other.hasNewTxn()) { + mergeNewTxn(other.getNewTxn()); + } + if (other.hasNewTxnResponse()) { + mergeNewTxnResponse(other.getNewTxnResponse()); + } + if (other.hasAddPartitionToTxn()) { + mergeAddPartitionToTxn(other.getAddPartitionToTxn()); + } + if (other.hasAddPartitionToTxnResponse()) { + mergeAddPartitionToTxnResponse(other.getAddPartitionToTxnResponse()); + } + if (other.hasAddSubscriptionToTxn()) { + mergeAddSubscriptionToTxn(other.getAddSubscriptionToTxn()); + } + if (other.hasAddSubscriptionToTxnResponse()) { + mergeAddSubscriptionToTxnResponse(other.getAddSubscriptionToTxnResponse()); + } + if (other.hasEndTxn()) { + mergeEndTxn(other.getEndTxn()); + } + if (other.hasEndTxnResponse()) { + mergeEndTxnResponse(other.getEndTxnResponse()); + } + if (other.hasEndTxnOnPartition()) { + mergeEndTxnOnPartition(other.getEndTxnOnPartition()); + } + if (other.hasEndTxnOnPartitionResponse()) { + mergeEndTxnOnPartitionResponse(other.getEndTxnOnPartitionResponse()); + } + if (other.hasEndTxnOnSubscription()) { + mergeEndTxnOnSubscription(other.getEndTxnOnSubscription()); + } + if (other.hasEndTxnOnSubscriptionResponse()) { + mergeEndTxnOnSubscriptionResponse(other.getEndTxnOnSubscriptionResponse()); + } + return this; + } + + public final boolean isInitialized() { + if (!hasType()) { + + return false; + } + if (hasConnect()) { + if (!getConnect().isInitialized()) { + + return false; + } + } + if (hasConnected()) { + if (!getConnected().isInitialized()) { + + return false; + } + } + if (hasSubscribe()) { + if (!getSubscribe().isInitialized()) { + + return false; + } + } + if (hasProducer()) { + if (!getProducer().isInitialized()) { + + return false; + } + } + if (hasSend()) { + if (!getSend().isInitialized()) { + + return false; + } + } + if (hasSendReceipt()) { + if (!getSendReceipt().isInitialized()) { + + return false; + } + } + if (hasSendError()) { + if (!getSendError().isInitialized()) { + + return false; + } + } + if (hasMessage()) { + if (!getMessage().isInitialized()) { + + return false; + } + } + if (hasAck()) { + if (!getAck().isInitialized()) { + + return false; + } + } + if (hasFlow()) { + if (!getFlow().isInitialized()) { + + return false; + } + } + if (hasUnsubscribe()) { + if (!getUnsubscribe().isInitialized()) { + + return false; + } + } + if (hasSuccess()) { + if (!getSuccess().isInitialized()) { + + return false; + } + } + if (hasError()) { + if (!getError().isInitialized()) { + + return false; + } + } + if (hasCloseProducer()) { + if (!getCloseProducer().isInitialized()) { + + return false; + } + } + if (hasCloseConsumer()) { + if (!getCloseConsumer().isInitialized()) { + + return false; + } + } + if (hasProducerSuccess()) { + if (!getProducerSuccess().isInitialized()) { + + return false; + } + } + if (hasRedeliverUnacknowledgedMessages()) { + if (!getRedeliverUnacknowledgedMessages().isInitialized()) { + + return false; + } + } + if (hasPartitionMetadata()) { + if (!getPartitionMetadata().isInitialized()) { + + return false; + } + } + if (hasPartitionMetadataResponse()) { + if (!getPartitionMetadataResponse().isInitialized()) { + + return false; + } + } + if (hasLookupTopic()) { + if (!getLookupTopic().isInitialized()) { + + return false; + } + } + if (hasLookupTopicResponse()) { + if (!getLookupTopicResponse().isInitialized()) { + + return false; + } + } + if (hasConsumerStats()) { + if (!getConsumerStats().isInitialized()) { + + return false; + } + } + if (hasConsumerStatsResponse()) { + if (!getConsumerStatsResponse().isInitialized()) { + + return false; + } + } + if (hasReachedEndOfTopic()) { + if (!getReachedEndOfTopic().isInitialized()) { + + return false; + } + } + if (hasSeek()) { + if (!getSeek().isInitialized()) { + + return false; + } + } + if (hasGetLastMessageId()) { + if (!getGetLastMessageId().isInitialized()) { + + return false; + } + } + if (hasGetLastMessageIdResponse()) { + if (!getGetLastMessageIdResponse().isInitialized()) { + + return false; + } + } + if (hasActiveConsumerChange()) { + if (!getActiveConsumerChange().isInitialized()) { + + return false; + } + } + if (hasGetTopicsOfNamespace()) { + if (!getGetTopicsOfNamespace().isInitialized()) { + + return false; + } + } + if (hasGetTopicsOfNamespaceResponse()) { + if (!getGetTopicsOfNamespaceResponse().isInitialized()) { + + return false; + } + } + if (hasGetSchema()) { + if (!getGetSchema().isInitialized()) { + + return false; + } + } + if (hasGetSchemaResponse()) { + if (!getGetSchemaResponse().isInitialized()) { + + return false; + } + } + if (hasAckResponse()) { + if (!getAckResponse().isInitialized()) { + + return false; + } + } + if (hasGetOrCreateSchema()) { + if (!getGetOrCreateSchema().isInitialized()) { + + return false; + } + } + if (hasGetOrCreateSchemaResponse()) { + if (!getGetOrCreateSchemaResponse().isInitialized()) { + + return false; + } + } + if (hasNewTxn()) { + if (!getNewTxn().isInitialized()) { + + return false; + } + } + if (hasNewTxnResponse()) { + if (!getNewTxnResponse().isInitialized()) { + + return false; + } + } + if (hasAddPartitionToTxn()) { + if (!getAddPartitionToTxn().isInitialized()) { + + return false; + } + } + if (hasAddPartitionToTxnResponse()) { + if (!getAddPartitionToTxnResponse().isInitialized()) { + + return false; + } + } + if (hasAddSubscriptionToTxn()) { + if (!getAddSubscriptionToTxn().isInitialized()) { + + return false; + } + } + if (hasAddSubscriptionToTxnResponse()) { + if (!getAddSubscriptionToTxnResponse().isInitialized()) { + + return false; + } + } + if (hasEndTxn()) { + if (!getEndTxn().isInitialized()) { + + return false; + } + } + if (hasEndTxnResponse()) { + if (!getEndTxnResponse().isInitialized()) { + + return false; + } + } + if (hasEndTxnOnPartition()) { + if (!getEndTxnOnPartition().isInitialized()) { + + return false; + } + } + if (hasEndTxnOnPartitionResponse()) { + if (!getEndTxnOnPartitionResponse().isInitialized()) { + + return false; + } + } + if (hasEndTxnOnSubscription()) { + if (!getEndTxnOnSubscription().isInitialized()) { + + return false; + } + } + if (hasEndTxnOnSubscriptionResponse()) { + if (!getEndTxnOnSubscriptionResponse().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 8: { + int rawValue = input.readEnum(); + org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type value = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.valueOf(rawValue); + if (value != null) { + bitField0_ |= 0x00000001; + type_ = value; + } + break; + } + case 18: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.newBuilder(); + if (hasConnect()) { + subBuilder.mergeFrom(getConnect()); + } + input.readMessage(subBuilder, extensionRegistry); + setConnect(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 26: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.newBuilder(); + if (hasConnected()) { + subBuilder.mergeFrom(getConnected()); + } + input.readMessage(subBuilder, extensionRegistry); + setConnected(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 34: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.newBuilder(); + if (hasSubscribe()) { + subBuilder.mergeFrom(getSubscribe()); + } + input.readMessage(subBuilder, extensionRegistry); + setSubscribe(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 42: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.newBuilder(); + if (hasProducer()) { + subBuilder.mergeFrom(getProducer()); + } + input.readMessage(subBuilder, extensionRegistry); + setProducer(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 50: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.newBuilder(); + if (hasSend()) { + subBuilder.mergeFrom(getSend()); + } + input.readMessage(subBuilder, extensionRegistry); + setSend(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 58: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.newBuilder(); + if (hasSendReceipt()) { + subBuilder.mergeFrom(getSendReceipt()); + } + input.readMessage(subBuilder, extensionRegistry); + setSendReceipt(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 66: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.newBuilder(); + if (hasSendError()) { + subBuilder.mergeFrom(getSendError()); + } + input.readMessage(subBuilder, extensionRegistry); + setSendError(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 74: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.newBuilder(); + if (hasMessage()) { + subBuilder.mergeFrom(getMessage()); + } + input.readMessage(subBuilder, extensionRegistry); + setMessage(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 82: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.newBuilder(); + if (hasAck()) { + subBuilder.mergeFrom(getAck()); + } + input.readMessage(subBuilder, extensionRegistry); + setAck(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 90: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.newBuilder(); + if (hasFlow()) { + subBuilder.mergeFrom(getFlow()); + } + input.readMessage(subBuilder, extensionRegistry); + setFlow(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 98: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.newBuilder(); + if (hasUnsubscribe()) { + subBuilder.mergeFrom(getUnsubscribe()); + } + input.readMessage(subBuilder, extensionRegistry); + setUnsubscribe(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 106: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.newBuilder(); + if (hasSuccess()) { + subBuilder.mergeFrom(getSuccess()); + } + input.readMessage(subBuilder, extensionRegistry); + setSuccess(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 114: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandError.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.newBuilder(); + if (hasError()) { + subBuilder.mergeFrom(getError()); + } + input.readMessage(subBuilder, extensionRegistry); + setError(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 122: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.newBuilder(); + if (hasCloseProducer()) { + subBuilder.mergeFrom(getCloseProducer()); + } + input.readMessage(subBuilder, extensionRegistry); + setCloseProducer(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 130: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.newBuilder(); + if (hasCloseConsumer()) { + subBuilder.mergeFrom(getCloseConsumer()); + } + input.readMessage(subBuilder, extensionRegistry); + setCloseConsumer(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 138: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.newBuilder(); + if (hasProducerSuccess()) { + subBuilder.mergeFrom(getProducerSuccess()); + } + input.readMessage(subBuilder, extensionRegistry); + setProducerSuccess(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 146: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.newBuilder(); + if (hasPing()) { + subBuilder.mergeFrom(getPing()); + } + input.readMessage(subBuilder, extensionRegistry); + setPing(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 154: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.newBuilder(); + if (hasPong()) { + subBuilder.mergeFrom(getPong()); + } + input.readMessage(subBuilder, extensionRegistry); + setPong(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 162: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.newBuilder(); + if (hasRedeliverUnacknowledgedMessages()) { + subBuilder.mergeFrom(getRedeliverUnacknowledgedMessages()); + } + input.readMessage(subBuilder, extensionRegistry); + setRedeliverUnacknowledgedMessages(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 170: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.newBuilder(); + if (hasPartitionMetadata()) { + subBuilder.mergeFrom(getPartitionMetadata()); + } + input.readMessage(subBuilder, extensionRegistry); + setPartitionMetadata(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 178: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.newBuilder(); + if (hasPartitionMetadataResponse()) { + subBuilder.mergeFrom(getPartitionMetadataResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setPartitionMetadataResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 186: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.newBuilder(); + if (hasLookupTopic()) { + subBuilder.mergeFrom(getLookupTopic()); + } + input.readMessage(subBuilder, extensionRegistry); + setLookupTopic(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 194: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.newBuilder(); + if (hasLookupTopicResponse()) { + subBuilder.mergeFrom(getLookupTopicResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setLookupTopicResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 202: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder(); + if (hasConsumerStats()) { + subBuilder.mergeFrom(getConsumerStats()); + } + input.readMessage(subBuilder, extensionRegistry); + setConsumerStats(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 210: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder(); + if (hasConsumerStatsResponse()) { + subBuilder.mergeFrom(getConsumerStatsResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setConsumerStatsResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 218: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.newBuilder(); + if (hasReachedEndOfTopic()) { + subBuilder.mergeFrom(getReachedEndOfTopic()); + } + input.readMessage(subBuilder, extensionRegistry); + setReachedEndOfTopic(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 226: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.newBuilder(); + if (hasSeek()) { + subBuilder.mergeFrom(getSeek()); + } + input.readMessage(subBuilder, extensionRegistry); + setSeek(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 234: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.newBuilder(); + if (hasGetLastMessageId()) { + subBuilder.mergeFrom(getGetLastMessageId()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetLastMessageId(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 242: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.newBuilder(); + if (hasGetLastMessageIdResponse()) { + subBuilder.mergeFrom(getGetLastMessageIdResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetLastMessageIdResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 250: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.newBuilder(); + if (hasActiveConsumerChange()) { + subBuilder.mergeFrom(getActiveConsumerChange()); + } + input.readMessage(subBuilder, extensionRegistry); + setActiveConsumerChange(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 258: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.newBuilder(); + if (hasGetTopicsOfNamespace()) { + subBuilder.mergeFrom(getGetTopicsOfNamespace()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetTopicsOfNamespace(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 266: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.newBuilder(); + if (hasGetTopicsOfNamespaceResponse()) { + subBuilder.mergeFrom(getGetTopicsOfNamespaceResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetTopicsOfNamespaceResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 274: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.newBuilder(); + if (hasGetSchema()) { + subBuilder.mergeFrom(getGetSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 282: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.newBuilder(); + if (hasGetSchemaResponse()) { + subBuilder.mergeFrom(getGetSchemaResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetSchemaResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 290: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.newBuilder(); + if (hasAuthChallenge()) { + subBuilder.mergeFrom(getAuthChallenge()); + } + input.readMessage(subBuilder, extensionRegistry); + setAuthChallenge(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 298: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.newBuilder(); + if (hasAuthResponse()) { + subBuilder.mergeFrom(getAuthResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setAuthResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 306: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.newBuilder(); + if (hasAckResponse()) { + subBuilder.mergeFrom(getAckResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setAckResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 314: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.newBuilder(); + if (hasGetOrCreateSchema()) { + subBuilder.mergeFrom(getGetOrCreateSchema()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetOrCreateSchema(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 322: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.newBuilder(); + if (hasGetOrCreateSchemaResponse()) { + subBuilder.mergeFrom(getGetOrCreateSchemaResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setGetOrCreateSchemaResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 402: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.newBuilder(); + if (hasNewTxn()) { + subBuilder.mergeFrom(getNewTxn()); + } + input.readMessage(subBuilder, extensionRegistry); + setNewTxn(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 410: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.newBuilder(); + if (hasNewTxnResponse()) { + subBuilder.mergeFrom(getNewTxnResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setNewTxnResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 418: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.newBuilder(); + if (hasAddPartitionToTxn()) { + subBuilder.mergeFrom(getAddPartitionToTxn()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddPartitionToTxn(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 426: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.newBuilder(); + if (hasAddPartitionToTxnResponse()) { + subBuilder.mergeFrom(getAddPartitionToTxnResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddPartitionToTxnResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 434: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.newBuilder(); + if (hasAddSubscriptionToTxn()) { + subBuilder.mergeFrom(getAddSubscriptionToTxn()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddSubscriptionToTxn(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 442: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.newBuilder(); + if (hasAddSubscriptionToTxnResponse()) { + subBuilder.mergeFrom(getAddSubscriptionToTxnResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setAddSubscriptionToTxnResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 450: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.newBuilder(); + if (hasEndTxn()) { + subBuilder.mergeFrom(getEndTxn()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxn(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 458: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.newBuilder(); + if (hasEndTxnResponse()) { + subBuilder.mergeFrom(getEndTxnResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxnResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 466: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.newBuilder(); + if (hasEndTxnOnPartition()) { + subBuilder.mergeFrom(getEndTxnOnPartition()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxnOnPartition(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 474: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.newBuilder(); + if (hasEndTxnOnPartitionResponse()) { + subBuilder.mergeFrom(getEndTxnOnPartitionResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxnOnPartitionResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 482: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.newBuilder(); + if (hasEndTxnOnSubscription()) { + subBuilder.mergeFrom(getEndTxnOnSubscription()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxnOnSubscription(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + case 490: { + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.newBuilder(); + if (hasEndTxnOnSubscriptionResponse()) { + subBuilder.mergeFrom(getEndTxnOnSubscriptionResponse()); + } + input.readMessage(subBuilder, extensionRegistry); + setEndTxnOnSubscriptionResponse(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } + } + } + } + + private int bitField0_; + private int bitField1_; + + // required .pulsar.proto.BaseCommand.Type type = 1; + private org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; + public boolean hasType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type getType() { + return type_; + } + public Builder setType(org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + type_ = value; + + return this; + } + public Builder clearType() { + bitField0_ = (bitField0_ & ~0x00000001); + type_ = org.apache.pulsar.common.api.proto.PulsarApi.BaseCommand.Type.CONNECT; + + return this; + } + + // optional .pulsar.proto.CommandConnect connect = 2; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); + public boolean hasConnect() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect getConnect() { + return connect_; + } + public Builder setConnect(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect value) { + if (value == null) { + throw new NullPointerException(); + } + connect_ = value; + + bitField0_ |= 0x00000002; + return this; + } + public Builder setConnect( + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.Builder builderForValue) { + connect_ = builderForValue.build(); + + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeConnect(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect value) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + connect_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance()) { + connect_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.newBuilder(connect_).mergeFrom(value).buildPartial(); + } else { + connect_ = value; + } + + bitField0_ |= 0x00000002; + return this; + } + public Builder clearConnect() { + connect_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + // optional .pulsar.proto.CommandConnected connected = 3; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); + public boolean hasConnected() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected getConnected() { + return connected_; + } + public Builder setConnected(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected value) { + if (value == null) { + throw new NullPointerException(); + } + connected_ = value; + + bitField0_ |= 0x00000004; + return this; + } + public Builder setConnected( + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.Builder builderForValue) { + connected_ = builderForValue.build(); + + bitField0_ |= 0x00000004; + return this; + } + public Builder mergeConnected(org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected value) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + connected_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance()) { + connected_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.newBuilder(connected_).mergeFrom(value).buildPartial(); + } else { + connected_ = value; + } + + bitField0_ |= 0x00000004; + return this; + } + public Builder clearConnected() { + connected_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConnected.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000004); + return this; + } + + // optional .pulsar.proto.CommandSubscribe subscribe = 4; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); + public boolean hasSubscribe() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe getSubscribe() { + return subscribe_; + } + public Builder setSubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe value) { + if (value == null) { + throw new NullPointerException(); + } + subscribe_ = value; + + bitField0_ |= 0x00000008; + return this; + } + public Builder setSubscribe( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.Builder builderForValue) { + subscribe_ = builderForValue.build(); + + bitField0_ |= 0x00000008; + return this; + } + public Builder mergeSubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe value) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + subscribe_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance()) { + subscribe_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.newBuilder(subscribe_).mergeFrom(value).buildPartial(); + } else { + subscribe_ = value; + } + + bitField0_ |= 0x00000008; + return this; + } + public Builder clearSubscribe() { + subscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSubscribe.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + // optional .pulsar.proto.CommandProducer producer = 5; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); + public boolean hasProducer() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer getProducer() { + return producer_; + } + public Builder setProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer value) { + if (value == null) { + throw new NullPointerException(); + } + producer_ = value; + + bitField0_ |= 0x00000010; + return this; + } + public Builder setProducer( + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.Builder builderForValue) { + producer_ = builderForValue.build(); + + bitField0_ |= 0x00000010; + return this; + } + public Builder mergeProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer value) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + producer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance()) { + producer_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.newBuilder(producer_).mergeFrom(value).buildPartial(); + } else { + producer_ = value; + } + + bitField0_ |= 0x00000010; + return this; + } + public Builder clearProducer() { + producer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducer.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000010); + return this; + } + + // optional .pulsar.proto.CommandSend send = 6; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSend send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); + public boolean hasSend() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSend getSend() { + return send_; + } + public Builder setSend(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend value) { + if (value == null) { + throw new NullPointerException(); + } + send_ = value; + + bitField0_ |= 0x00000020; + return this; + } + public Builder setSend( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.Builder builderForValue) { + send_ = builderForValue.build(); + + bitField0_ |= 0x00000020; + return this; + } + public Builder mergeSend(org.apache.pulsar.common.api.proto.PulsarApi.CommandSend value) { + if (((bitField0_ & 0x00000020) == 0x00000020) && + send_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance()) { + send_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.newBuilder(send_).mergeFrom(value).buildPartial(); + } else { + send_ = value; + } + + bitField0_ |= 0x00000020; + return this; + } + public Builder clearSend() { + send_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSend.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000020); + return this; + } + + // optional .pulsar.proto.CommandSendReceipt send_receipt = 7; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); + public boolean hasSendReceipt() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt getSendReceipt() { + return sendReceipt_; + } + public Builder setSendReceipt(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt value) { + if (value == null) { + throw new NullPointerException(); + } + sendReceipt_ = value; + + bitField0_ |= 0x00000040; + return this; + } + public Builder setSendReceipt( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.Builder builderForValue) { + sendReceipt_ = builderForValue.build(); + + bitField0_ |= 0x00000040; + return this; + } + public Builder mergeSendReceipt(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt value) { + if (((bitField0_ & 0x00000040) == 0x00000040) && + sendReceipt_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance()) { + sendReceipt_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.newBuilder(sendReceipt_).mergeFrom(value).buildPartial(); + } else { + sendReceipt_ = value; + } + + bitField0_ |= 0x00000040; + return this; + } + public Builder clearSendReceipt() { + sendReceipt_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendReceipt.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000040); + return this; + } + + // optional .pulsar.proto.CommandSendError send_error = 8; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); + public boolean hasSendError() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError getSendError() { + return sendError_; + } + public Builder setSendError(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError value) { + if (value == null) { + throw new NullPointerException(); + } + sendError_ = value; + + bitField0_ |= 0x00000080; + return this; + } + public Builder setSendError( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.Builder builderForValue) { + sendError_ = builderForValue.build(); + + bitField0_ |= 0x00000080; + return this; + } + public Builder mergeSendError(org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError value) { + if (((bitField0_ & 0x00000080) == 0x00000080) && + sendError_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance()) { + sendError_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.newBuilder(sendError_).mergeFrom(value).buildPartial(); + } else { + sendError_ = value; + } + + bitField0_ |= 0x00000080; + return this; + } + public Builder clearSendError() { + sendError_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSendError.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000080); + return this; + } + + // optional .pulsar.proto.CommandMessage message = 9; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); + public boolean hasMessage() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage getMessage() { + return message_; + } + public Builder setMessage(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage value) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + + bitField0_ |= 0x00000100; + return this; + } + public Builder setMessage( + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.Builder builderForValue) { + message_ = builderForValue.build(); + + bitField0_ |= 0x00000100; + return this; + } + public Builder mergeMessage(org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage value) { + if (((bitField0_ & 0x00000100) == 0x00000100) && + message_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance()) { + message_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.newBuilder(message_).mergeFrom(value).buildPartial(); + } else { + message_ = value; + } + + bitField0_ |= 0x00000100; + return this; + } + public Builder clearMessage() { + message_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandMessage.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000100); + return this; + } + + // optional .pulsar.proto.CommandAck ack = 10; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAck ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); + public boolean hasAck() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAck getAck() { + return ack_; + } + public Builder setAck(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck value) { + if (value == null) { + throw new NullPointerException(); + } + ack_ = value; + + bitField0_ |= 0x00000200; + return this; + } + public Builder setAck( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.Builder builderForValue) { + ack_ = builderForValue.build(); + + bitField0_ |= 0x00000200; + return this; + } + public Builder mergeAck(org.apache.pulsar.common.api.proto.PulsarApi.CommandAck value) { + if (((bitField0_ & 0x00000200) == 0x00000200) && + ack_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance()) { + ack_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.newBuilder(ack_).mergeFrom(value).buildPartial(); + } else { + ack_ = value; + } + + bitField0_ |= 0x00000200; + return this; + } + public Builder clearAck() { + ack_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000200); + return this; + } + + // optional .pulsar.proto.CommandFlow flow = 11; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); + public boolean hasFlow() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow getFlow() { + return flow_; + } + public Builder setFlow(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow value) { + if (value == null) { + throw new NullPointerException(); + } + flow_ = value; + + bitField0_ |= 0x00000400; + return this; + } + public Builder setFlow( + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.Builder builderForValue) { + flow_ = builderForValue.build(); + + bitField0_ |= 0x00000400; + return this; + } + public Builder mergeFlow(org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow value) { + if (((bitField0_ & 0x00000400) == 0x00000400) && + flow_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance()) { + flow_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.newBuilder(flow_).mergeFrom(value).buildPartial(); + } else { + flow_ = value; + } + + bitField0_ |= 0x00000400; + return this; + } + public Builder clearFlow() { + flow_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandFlow.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000400); + return this; + } + + // optional .pulsar.proto.CommandUnsubscribe unsubscribe = 12; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); + public boolean hasUnsubscribe() { + return ((bitField0_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe getUnsubscribe() { + return unsubscribe_; + } + public Builder setUnsubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe value) { + if (value == null) { + throw new NullPointerException(); + } + unsubscribe_ = value; + + bitField0_ |= 0x00000800; + return this; + } + public Builder setUnsubscribe( + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.Builder builderForValue) { + unsubscribe_ = builderForValue.build(); + + bitField0_ |= 0x00000800; + return this; + } + public Builder mergeUnsubscribe(org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe value) { + if (((bitField0_ & 0x00000800) == 0x00000800) && + unsubscribe_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance()) { + unsubscribe_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.newBuilder(unsubscribe_).mergeFrom(value).buildPartial(); + } else { + unsubscribe_ = value; + } + + bitField0_ |= 0x00000800; + return this; + } + public Builder clearUnsubscribe() { + unsubscribe_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000800); + return this; + } + + // optional .pulsar.proto.CommandSuccess success = 13; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); + public boolean hasSuccess() { + return ((bitField0_ & 0x00001000) == 0x00001000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess getSuccess() { + return success_; + } + public Builder setSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess value) { + if (value == null) { + throw new NullPointerException(); + } + success_ = value; + + bitField0_ |= 0x00001000; + return this; + } + public Builder setSuccess( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.Builder builderForValue) { + success_ = builderForValue.build(); + + bitField0_ |= 0x00001000; + return this; + } + public Builder mergeSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess value) { + if (((bitField0_ & 0x00001000) == 0x00001000) && + success_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance()) { + success_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.newBuilder(success_).mergeFrom(value).buildPartial(); + } else { + success_ = value; + } + + bitField0_ |= 0x00001000; + return this; + } + public Builder clearSuccess() { + success_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00001000); + return this; + } + + // optional .pulsar.proto.CommandError error = 14; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandError error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); + public boolean hasError() { + return ((bitField0_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandError getError() { + return error_; + } + public Builder setError(org.apache.pulsar.common.api.proto.PulsarApi.CommandError value) { + if (value == null) { + throw new NullPointerException(); + } + error_ = value; + + bitField0_ |= 0x00002000; + return this; + } + public Builder setError( + org.apache.pulsar.common.api.proto.PulsarApi.CommandError.Builder builderForValue) { + error_ = builderForValue.build(); + + bitField0_ |= 0x00002000; + return this; + } + public Builder mergeError(org.apache.pulsar.common.api.proto.PulsarApi.CommandError value) { + if (((bitField0_ & 0x00002000) == 0x00002000) && + error_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance()) { + error_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandError.newBuilder(error_).mergeFrom(value).buildPartial(); + } else { + error_ = value; + } + + bitField0_ |= 0x00002000; + return this; + } + public Builder clearError() { + error_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandError.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00002000); + return this; + } + + // optional .pulsar.proto.CommandCloseProducer close_producer = 15; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); + public boolean hasCloseProducer() { + return ((bitField0_ & 0x00004000) == 0x00004000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer getCloseProducer() { + return closeProducer_; + } + public Builder setCloseProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer value) { + if (value == null) { + throw new NullPointerException(); + } + closeProducer_ = value; + + bitField0_ |= 0x00004000; + return this; + } + public Builder setCloseProducer( + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.Builder builderForValue) { + closeProducer_ = builderForValue.build(); + + bitField0_ |= 0x00004000; + return this; + } + public Builder mergeCloseProducer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer value) { + if (((bitField0_ & 0x00004000) == 0x00004000) && + closeProducer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance()) { + closeProducer_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.newBuilder(closeProducer_).mergeFrom(value).buildPartial(); + } else { + closeProducer_ = value; + } + + bitField0_ |= 0x00004000; + return this; + } + public Builder clearCloseProducer() { + closeProducer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseProducer.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00004000); + return this; + } + + // optional .pulsar.proto.CommandCloseConsumer close_consumer = 16; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); + public boolean hasCloseConsumer() { + return ((bitField0_ & 0x00008000) == 0x00008000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer getCloseConsumer() { + return closeConsumer_; + } + public Builder setCloseConsumer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer value) { + if (value == null) { + throw new NullPointerException(); + } + closeConsumer_ = value; + + bitField0_ |= 0x00008000; + return this; + } + public Builder setCloseConsumer( + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.Builder builderForValue) { + closeConsumer_ = builderForValue.build(); + + bitField0_ |= 0x00008000; + return this; + } + public Builder mergeCloseConsumer(org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer value) { + if (((bitField0_ & 0x00008000) == 0x00008000) && + closeConsumer_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance()) { + closeConsumer_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.newBuilder(closeConsumer_).mergeFrom(value).buildPartial(); + } else { + closeConsumer_ = value; + } + + bitField0_ |= 0x00008000; + return this; + } + public Builder clearCloseConsumer() { + closeConsumer_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandCloseConsumer.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00008000); + return this; + } + + // optional .pulsar.proto.CommandProducerSuccess producer_success = 17; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); + public boolean hasProducerSuccess() { + return ((bitField0_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess getProducerSuccess() { + return producerSuccess_; + } + public Builder setProducerSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess value) { + if (value == null) { + throw new NullPointerException(); + } + producerSuccess_ = value; + + bitField0_ |= 0x00010000; + return this; + } + public Builder setProducerSuccess( + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.Builder builderForValue) { + producerSuccess_ = builderForValue.build(); + + bitField0_ |= 0x00010000; + return this; + } + public Builder mergeProducerSuccess(org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess value) { + if (((bitField0_ & 0x00010000) == 0x00010000) && + producerSuccess_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance()) { + producerSuccess_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.newBuilder(producerSuccess_).mergeFrom(value).buildPartial(); + } else { + producerSuccess_ = value; + } + + bitField0_ |= 0x00010000; + return this; + } + public Builder clearProducerSuccess() { + producerSuccess_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandProducerSuccess.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00010000); + return this; + } + + // optional .pulsar.proto.CommandPing ping = 18; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPing ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); + public boolean hasPing() { + return ((bitField0_ & 0x00020000) == 0x00020000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPing getPing() { + return ping_; + } + public Builder setPing(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing value) { + if (value == null) { + throw new NullPointerException(); + } + ping_ = value; + + bitField0_ |= 0x00020000; + return this; + } + public Builder setPing( + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.Builder builderForValue) { + ping_ = builderForValue.build(); + + bitField0_ |= 0x00020000; + return this; + } + public Builder mergePing(org.apache.pulsar.common.api.proto.PulsarApi.CommandPing value) { + if (((bitField0_ & 0x00020000) == 0x00020000) && + ping_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance()) { + ping_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.newBuilder(ping_).mergeFrom(value).buildPartial(); + } else { + ping_ = value; + } + + bitField0_ |= 0x00020000; + return this; + } + public Builder clearPing() { + ping_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPing.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00020000); + return this; + } + + // optional .pulsar.proto.CommandPong pong = 19; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPong pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); + public boolean hasPong() { + return ((bitField0_ & 0x00040000) == 0x00040000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPong getPong() { + return pong_; + } + public Builder setPong(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong value) { + if (value == null) { + throw new NullPointerException(); + } + pong_ = value; + + bitField0_ |= 0x00040000; + return this; + } + public Builder setPong( + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.Builder builderForValue) { + pong_ = builderForValue.build(); + + bitField0_ |= 0x00040000; + return this; + } + public Builder mergePong(org.apache.pulsar.common.api.proto.PulsarApi.CommandPong value) { + if (((bitField0_ & 0x00040000) == 0x00040000) && + pong_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance()) { + pong_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.newBuilder(pong_).mergeFrom(value).buildPartial(); + } else { + pong_ = value; + } + + bitField0_ |= 0x00040000; + return this; + } + public Builder clearPong() { + pong_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPong.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00040000); + return this; + } + + // optional .pulsar.proto.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages = 20; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); + public boolean hasRedeliverUnacknowledgedMessages() { + return ((bitField0_ & 0x00080000) == 0x00080000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages getRedeliverUnacknowledgedMessages() { + return redeliverUnacknowledgedMessages_; + } + public Builder setRedeliverUnacknowledgedMessages(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages value) { + if (value == null) { + throw new NullPointerException(); + } + redeliverUnacknowledgedMessages_ = value; + + bitField0_ |= 0x00080000; + return this; + } + public Builder setRedeliverUnacknowledgedMessages( + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.Builder builderForValue) { + redeliverUnacknowledgedMessages_ = builderForValue.build(); + + bitField0_ |= 0x00080000; + return this; + } + public Builder mergeRedeliverUnacknowledgedMessages(org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages value) { + if (((bitField0_ & 0x00080000) == 0x00080000) && + redeliverUnacknowledgedMessages_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance()) { + redeliverUnacknowledgedMessages_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.newBuilder(redeliverUnacknowledgedMessages_).mergeFrom(value).buildPartial(); + } else { + redeliverUnacknowledgedMessages_ = value; + } + + bitField0_ |= 0x00080000; + return this; + } + public Builder clearRedeliverUnacknowledgedMessages() { + redeliverUnacknowledgedMessages_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandRedeliverUnacknowledgedMessages.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00080000); + return this; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadata partitionMetadata = 21; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); + public boolean hasPartitionMetadata() { + return ((bitField0_ & 0x00100000) == 0x00100000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata getPartitionMetadata() { + return partitionMetadata_; + } + public Builder setPartitionMetadata(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata value) { + if (value == null) { + throw new NullPointerException(); + } + partitionMetadata_ = value; + + bitField0_ |= 0x00100000; + return this; + } + public Builder setPartitionMetadata( + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.Builder builderForValue) { + partitionMetadata_ = builderForValue.build(); + + bitField0_ |= 0x00100000; + return this; + } + public Builder mergePartitionMetadata(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata value) { + if (((bitField0_ & 0x00100000) == 0x00100000) && + partitionMetadata_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance()) { + partitionMetadata_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.newBuilder(partitionMetadata_).mergeFrom(value).buildPartial(); + } else { + partitionMetadata_ = value; + } + + bitField0_ |= 0x00100000; + return this; + } + public Builder clearPartitionMetadata() { + partitionMetadata_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadata.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00100000); + return this; + } + + // optional .pulsar.proto.CommandPartitionedTopicMetadataResponse partitionMetadataResponse = 22; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); + public boolean hasPartitionMetadataResponse() { + return ((bitField0_ & 0x00200000) == 0x00200000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse getPartitionMetadataResponse() { + return partitionMetadataResponse_; + } + public Builder setPartitionMetadataResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse value) { + if (value == null) { + throw new NullPointerException(); + } + partitionMetadataResponse_ = value; + + bitField0_ |= 0x00200000; + return this; + } + public Builder setPartitionMetadataResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.Builder builderForValue) { + partitionMetadataResponse_ = builderForValue.build(); + + bitField0_ |= 0x00200000; + return this; + } + public Builder mergePartitionMetadataResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse value) { + if (((bitField0_ & 0x00200000) == 0x00200000) && + partitionMetadataResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance()) { + partitionMetadataResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.newBuilder(partitionMetadataResponse_).mergeFrom(value).buildPartial(); + } else { + partitionMetadataResponse_ = value; + } + + bitField0_ |= 0x00200000; + return this; + } + public Builder clearPartitionMetadataResponse() { + partitionMetadataResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandPartitionedTopicMetadataResponse.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00200000); + return this; + } + + // optional .pulsar.proto.CommandLookupTopic lookupTopic = 23; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); + public boolean hasLookupTopic() { + return ((bitField0_ & 0x00400000) == 0x00400000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic getLookupTopic() { + return lookupTopic_; + } + public Builder setLookupTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic value) { + if (value == null) { + throw new NullPointerException(); + } + lookupTopic_ = value; + + bitField0_ |= 0x00400000; + return this; + } + public Builder setLookupTopic( + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.Builder builderForValue) { + lookupTopic_ = builderForValue.build(); + + bitField0_ |= 0x00400000; + return this; + } + public Builder mergeLookupTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic value) { + if (((bitField0_ & 0x00400000) == 0x00400000) && + lookupTopic_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance()) { + lookupTopic_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.newBuilder(lookupTopic_).mergeFrom(value).buildPartial(); + } else { + lookupTopic_ = value; + } + + bitField0_ |= 0x00400000; + return this; + } + public Builder clearLookupTopic() { + lookupTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopic.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00400000); + return this; + } + + // optional .pulsar.proto.CommandLookupTopicResponse lookupTopicResponse = 24; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); + public boolean hasLookupTopicResponse() { + return ((bitField0_ & 0x00800000) == 0x00800000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse getLookupTopicResponse() { + return lookupTopicResponse_; + } + public Builder setLookupTopicResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse value) { + if (value == null) { + throw new NullPointerException(); + } + lookupTopicResponse_ = value; + + bitField0_ |= 0x00800000; + return this; + } + public Builder setLookupTopicResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.Builder builderForValue) { + lookupTopicResponse_ = builderForValue.build(); + + bitField0_ |= 0x00800000; + return this; + } + public Builder mergeLookupTopicResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse value) { + if (((bitField0_ & 0x00800000) == 0x00800000) && + lookupTopicResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance()) { + lookupTopicResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.newBuilder(lookupTopicResponse_).mergeFrom(value).buildPartial(); + } else { + lookupTopicResponse_ = value; + } + + bitField0_ |= 0x00800000; + return this; + } + public Builder clearLookupTopicResponse() { + lookupTopicResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandLookupTopicResponse.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00800000); + return this; + } + + // optional .pulsar.proto.CommandConsumerStats consumerStats = 25; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); + public boolean hasConsumerStats() { + return ((bitField0_ & 0x01000000) == 0x01000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats getConsumerStats() { + return consumerStats_; + } + public Builder setConsumerStats(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) { + if (value == null) { + throw new NullPointerException(); + } + consumerStats_ = value; + + bitField0_ |= 0x01000000; + return this; + } + public Builder setConsumerStats( + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.Builder builderForValue) { + consumerStats_ = builderForValue.build(); + + bitField0_ |= 0x01000000; + return this; + } + public Builder mergeConsumerStats(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats value) { + if (((bitField0_ & 0x01000000) == 0x01000000) && + consumerStats_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance()) { + consumerStats_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.newBuilder(consumerStats_).mergeFrom(value).buildPartial(); + } else { + consumerStats_ = value; + } + + bitField0_ |= 0x01000000; + return this; + } + public Builder clearConsumerStats() { + consumerStats_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStats.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x01000000); + return this; + } + + // optional .pulsar.proto.CommandConsumerStatsResponse consumerStatsResponse = 26; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); + public boolean hasConsumerStatsResponse() { + return ((bitField0_ & 0x02000000) == 0x02000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse getConsumerStatsResponse() { + return consumerStatsResponse_; + } + public Builder setConsumerStatsResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) { + if (value == null) { + throw new NullPointerException(); + } + consumerStatsResponse_ = value; + + bitField0_ |= 0x02000000; + return this; + } + public Builder setConsumerStatsResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.Builder builderForValue) { + consumerStatsResponse_ = builderForValue.build(); + + bitField0_ |= 0x02000000; + return this; + } + public Builder mergeConsumerStatsResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse value) { + if (((bitField0_ & 0x02000000) == 0x02000000) && + consumerStatsResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance()) { + consumerStatsResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.newBuilder(consumerStatsResponse_).mergeFrom(value).buildPartial(); + } else { + consumerStatsResponse_ = value; + } + + bitField0_ |= 0x02000000; + return this; + } + public Builder clearConsumerStatsResponse() { + consumerStatsResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandConsumerStatsResponse.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x02000000); + return this; + } + + // optional .pulsar.proto.CommandReachedEndOfTopic reachedEndOfTopic = 27; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); + public boolean hasReachedEndOfTopic() { + return ((bitField0_ & 0x04000000) == 0x04000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic getReachedEndOfTopic() { + return reachedEndOfTopic_; + } + public Builder setReachedEndOfTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic value) { + if (value == null) { + throw new NullPointerException(); + } + reachedEndOfTopic_ = value; + + bitField0_ |= 0x04000000; + return this; + } + public Builder setReachedEndOfTopic( + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.Builder builderForValue) { + reachedEndOfTopic_ = builderForValue.build(); + + bitField0_ |= 0x04000000; + return this; + } + public Builder mergeReachedEndOfTopic(org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic value) { + if (((bitField0_ & 0x04000000) == 0x04000000) && + reachedEndOfTopic_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance()) { + reachedEndOfTopic_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.newBuilder(reachedEndOfTopic_).mergeFrom(value).buildPartial(); + } else { + reachedEndOfTopic_ = value; + } + + bitField0_ |= 0x04000000; + return this; + } + public Builder clearReachedEndOfTopic() { + reachedEndOfTopic_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandReachedEndOfTopic.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x04000000); + return this; + } + + // optional .pulsar.proto.CommandSeek seek = 28; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); + public boolean hasSeek() { + return ((bitField0_ & 0x08000000) == 0x08000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek getSeek() { + return seek_; + } + public Builder setSeek(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek value) { + if (value == null) { + throw new NullPointerException(); + } + seek_ = value; + + bitField0_ |= 0x08000000; + return this; + } + public Builder setSeek( + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.Builder builderForValue) { + seek_ = builderForValue.build(); + + bitField0_ |= 0x08000000; + return this; + } + public Builder mergeSeek(org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek value) { + if (((bitField0_ & 0x08000000) == 0x08000000) && + seek_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance()) { + seek_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.newBuilder(seek_).mergeFrom(value).buildPartial(); + } else { + seek_ = value; + } + + bitField0_ |= 0x08000000; + return this; + } + public Builder clearSeek() { + seek_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandSeek.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x08000000); + return this; + } + + // optional .pulsar.proto.CommandGetLastMessageId getLastMessageId = 29; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); + public boolean hasGetLastMessageId() { + return ((bitField0_ & 0x10000000) == 0x10000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId getGetLastMessageId() { + return getLastMessageId_; + } + public Builder setGetLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId value) { + if (value == null) { + throw new NullPointerException(); + } + getLastMessageId_ = value; + + bitField0_ |= 0x10000000; + return this; + } + public Builder setGetLastMessageId( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.Builder builderForValue) { + getLastMessageId_ = builderForValue.build(); + + bitField0_ |= 0x10000000; + return this; + } + public Builder mergeGetLastMessageId(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId value) { + if (((bitField0_ & 0x10000000) == 0x10000000) && + getLastMessageId_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance()) { + getLastMessageId_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.newBuilder(getLastMessageId_).mergeFrom(value).buildPartial(); + } else { + getLastMessageId_ = value; + } + + bitField0_ |= 0x10000000; + return this; + } + public Builder clearGetLastMessageId() { + getLastMessageId_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageId.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x10000000); + return this; + } + + // optional .pulsar.proto.CommandGetLastMessageIdResponse getLastMessageIdResponse = 30; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); + public boolean hasGetLastMessageIdResponse() { + return ((bitField0_ & 0x20000000) == 0x20000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse getGetLastMessageIdResponse() { + return getLastMessageIdResponse_; + } + public Builder setGetLastMessageIdResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse value) { + if (value == null) { + throw new NullPointerException(); + } + getLastMessageIdResponse_ = value; + + bitField0_ |= 0x20000000; + return this; + } + public Builder setGetLastMessageIdResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.Builder builderForValue) { + getLastMessageIdResponse_ = builderForValue.build(); + + bitField0_ |= 0x20000000; + return this; + } + public Builder mergeGetLastMessageIdResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse value) { + if (((bitField0_ & 0x20000000) == 0x20000000) && + getLastMessageIdResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance()) { + getLastMessageIdResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.newBuilder(getLastMessageIdResponse_).mergeFrom(value).buildPartial(); + } else { + getLastMessageIdResponse_ = value; + } + + bitField0_ |= 0x20000000; + return this; + } + public Builder clearGetLastMessageIdResponse() { + getLastMessageIdResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetLastMessageIdResponse.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x20000000); + return this; + } + + // optional .pulsar.proto.CommandActiveConsumerChange active_consumer_change = 31; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); + public boolean hasActiveConsumerChange() { + return ((bitField0_ & 0x40000000) == 0x40000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange getActiveConsumerChange() { + return activeConsumerChange_; + } + public Builder setActiveConsumerChange(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange value) { + if (value == null) { + throw new NullPointerException(); + } + activeConsumerChange_ = value; + + bitField0_ |= 0x40000000; + return this; + } + public Builder setActiveConsumerChange( + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.Builder builderForValue) { + activeConsumerChange_ = builderForValue.build(); + + bitField0_ |= 0x40000000; + return this; + } + public Builder mergeActiveConsumerChange(org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange value) { + if (((bitField0_ & 0x40000000) == 0x40000000) && + activeConsumerChange_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance()) { + activeConsumerChange_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.newBuilder(activeConsumerChange_).mergeFrom(value).buildPartial(); + } else { + activeConsumerChange_ = value; + } + + bitField0_ |= 0x40000000; + return this; + } + public Builder clearActiveConsumerChange() { + activeConsumerChange_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandActiveConsumerChange.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x40000000); + return this; + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespace getTopicsOfNamespace = 32; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); + public boolean hasGetTopicsOfNamespace() { + return ((bitField0_ & 0x80000000) == 0x80000000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace getGetTopicsOfNamespace() { + return getTopicsOfNamespace_; + } + public Builder setGetTopicsOfNamespace(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace value) { + if (value == null) { + throw new NullPointerException(); + } + getTopicsOfNamespace_ = value; + + bitField0_ |= 0x80000000; + return this; + } + public Builder setGetTopicsOfNamespace( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.Builder builderForValue) { + getTopicsOfNamespace_ = builderForValue.build(); + + bitField0_ |= 0x80000000; + return this; + } + public Builder mergeGetTopicsOfNamespace(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace value) { + if (((bitField0_ & 0x80000000) == 0x80000000) && + getTopicsOfNamespace_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance()) { + getTopicsOfNamespace_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.newBuilder(getTopicsOfNamespace_).mergeFrom(value).buildPartial(); + } else { + getTopicsOfNamespace_ = value; + } + + bitField0_ |= 0x80000000; + return this; + } + public Builder clearGetTopicsOfNamespace() { + getTopicsOfNamespace_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespace.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x80000000); + return this; + } + + // optional .pulsar.proto.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse = 33; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); + public boolean hasGetTopicsOfNamespaceResponse() { + return ((bitField1_ & 0x00000001) == 0x00000001); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse getGetTopicsOfNamespaceResponse() { + return getTopicsOfNamespaceResponse_; + } + public Builder setGetTopicsOfNamespaceResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse value) { + if (value == null) { + throw new NullPointerException(); + } + getTopicsOfNamespaceResponse_ = value; + + bitField1_ |= 0x00000001; + return this; + } + public Builder setGetTopicsOfNamespaceResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.Builder builderForValue) { + getTopicsOfNamespaceResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000001; + return this; + } + public Builder mergeGetTopicsOfNamespaceResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse value) { + if (((bitField1_ & 0x00000001) == 0x00000001) && + getTopicsOfNamespaceResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance()) { + getTopicsOfNamespaceResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.newBuilder(getTopicsOfNamespaceResponse_).mergeFrom(value).buildPartial(); + } else { + getTopicsOfNamespaceResponse_ = value; + } + + bitField1_ |= 0x00000001; + return this; + } + public Builder clearGetTopicsOfNamespaceResponse() { + getTopicsOfNamespaceResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetTopicsOfNamespaceResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000001); + return this; + } + + // optional .pulsar.proto.CommandGetSchema getSchema = 34; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); + public boolean hasGetSchema() { + return ((bitField1_ & 0x00000002) == 0x00000002); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema getGetSchema() { + return getSchema_; + } + public Builder setGetSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema value) { + if (value == null) { + throw new NullPointerException(); + } + getSchema_ = value; + + bitField1_ |= 0x00000002; + return this; + } + public Builder setGetSchema( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.Builder builderForValue) { + getSchema_ = builderForValue.build(); + + bitField1_ |= 0x00000002; + return this; + } + public Builder mergeGetSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema value) { + if (((bitField1_ & 0x00000002) == 0x00000002) && + getSchema_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance()) { + getSchema_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.newBuilder(getSchema_).mergeFrom(value).buildPartial(); + } else { + getSchema_ = value; + } + + bitField1_ |= 0x00000002; + return this; + } + public Builder clearGetSchema() { + getSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchema.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000002); + return this; + } + + // optional .pulsar.proto.CommandGetSchemaResponse getSchemaResponse = 35; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); + public boolean hasGetSchemaResponse() { + return ((bitField1_ & 0x00000004) == 0x00000004); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse getGetSchemaResponse() { + return getSchemaResponse_; + } + public Builder setGetSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse value) { + if (value == null) { + throw new NullPointerException(); + } + getSchemaResponse_ = value; + + bitField1_ |= 0x00000004; + return this; + } + public Builder setGetSchemaResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.Builder builderForValue) { + getSchemaResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000004; + return this; + } + public Builder mergeGetSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse value) { + if (((bitField1_ & 0x00000004) == 0x00000004) && + getSchemaResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance()) { + getSchemaResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.newBuilder(getSchemaResponse_).mergeFrom(value).buildPartial(); + } else { + getSchemaResponse_ = value; + } + + bitField1_ |= 0x00000004; + return this; + } + public Builder clearGetSchemaResponse() { + getSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetSchemaResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000004); + return this; + } + + // optional .pulsar.proto.CommandAuthChallenge authChallenge = 36; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); + public boolean hasAuthChallenge() { + return ((bitField1_ & 0x00000008) == 0x00000008); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge getAuthChallenge() { + return authChallenge_; + } + public Builder setAuthChallenge(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge value) { + if (value == null) { + throw new NullPointerException(); + } + authChallenge_ = value; + + bitField1_ |= 0x00000008; + return this; + } + public Builder setAuthChallenge( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.Builder builderForValue) { + authChallenge_ = builderForValue.build(); + + bitField1_ |= 0x00000008; + return this; + } + public Builder mergeAuthChallenge(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge value) { + if (((bitField1_ & 0x00000008) == 0x00000008) && + authChallenge_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance()) { + authChallenge_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.newBuilder(authChallenge_).mergeFrom(value).buildPartial(); + } else { + authChallenge_ = value; + } + + bitField1_ |= 0x00000008; + return this; + } + public Builder clearAuthChallenge() { + authChallenge_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthChallenge.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000008); + return this; + } + + // optional .pulsar.proto.CommandAuthResponse authResponse = 37; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); + public boolean hasAuthResponse() { + return ((bitField1_ & 0x00000010) == 0x00000010); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse getAuthResponse() { + return authResponse_; + } + public Builder setAuthResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse value) { + if (value == null) { + throw new NullPointerException(); + } + authResponse_ = value; + + bitField1_ |= 0x00000010; + return this; + } + public Builder setAuthResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.Builder builderForValue) { + authResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000010; + return this; + } + public Builder mergeAuthResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse value) { + if (((bitField1_ & 0x00000010) == 0x00000010) && + authResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance()) { + authResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.newBuilder(authResponse_).mergeFrom(value).buildPartial(); + } else { + authResponse_ = value; + } + + bitField1_ |= 0x00000010; + return this; + } + public Builder clearAuthResponse() { + authResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAuthResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000010); + return this; + } + + // optional .pulsar.proto.CommandAckResponse ackResponse = 38; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); + public boolean hasAckResponse() { + return ((bitField1_ & 0x00000020) == 0x00000020); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse getAckResponse() { + return ackResponse_; + } + public Builder setAckResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse value) { + if (value == null) { + throw new NullPointerException(); + } + ackResponse_ = value; + + bitField1_ |= 0x00000020; + return this; + } + public Builder setAckResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.Builder builderForValue) { + ackResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000020; + return this; + } + public Builder mergeAckResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse value) { + if (((bitField1_ & 0x00000020) == 0x00000020) && + ackResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance()) { + ackResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.newBuilder(ackResponse_).mergeFrom(value).buildPartial(); + } else { + ackResponse_ = value; + } + + bitField1_ |= 0x00000020; + return this; + } + public Builder clearAckResponse() { + ackResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAckResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000020); + return this; + } + + // optional .pulsar.proto.CommandGetOrCreateSchema getOrCreateSchema = 39; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); + public boolean hasGetOrCreateSchema() { + return ((bitField1_ & 0x00000040) == 0x00000040); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema getGetOrCreateSchema() { + return getOrCreateSchema_; + } + public Builder setGetOrCreateSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema value) { + if (value == null) { + throw new NullPointerException(); + } + getOrCreateSchema_ = value; + + bitField1_ |= 0x00000040; + return this; + } + public Builder setGetOrCreateSchema( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.Builder builderForValue) { + getOrCreateSchema_ = builderForValue.build(); + + bitField1_ |= 0x00000040; + return this; + } + public Builder mergeGetOrCreateSchema(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema value) { + if (((bitField1_ & 0x00000040) == 0x00000040) && + getOrCreateSchema_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance()) { + getOrCreateSchema_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.newBuilder(getOrCreateSchema_).mergeFrom(value).buildPartial(); + } else { + getOrCreateSchema_ = value; + } + + bitField1_ |= 0x00000040; + return this; + } + public Builder clearGetOrCreateSchema() { + getOrCreateSchema_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchema.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000040); + return this; + } + + // optional .pulsar.proto.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse = 40; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); + public boolean hasGetOrCreateSchemaResponse() { + return ((bitField1_ & 0x00000080) == 0x00000080); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse getGetOrCreateSchemaResponse() { + return getOrCreateSchemaResponse_; + } + public Builder setGetOrCreateSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse value) { + if (value == null) { + throw new NullPointerException(); + } + getOrCreateSchemaResponse_ = value; + + bitField1_ |= 0x00000080; + return this; + } + public Builder setGetOrCreateSchemaResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.Builder builderForValue) { + getOrCreateSchemaResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000080; + return this; + } + public Builder mergeGetOrCreateSchemaResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse value) { + if (((bitField1_ & 0x00000080) == 0x00000080) && + getOrCreateSchemaResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance()) { + getOrCreateSchemaResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.newBuilder(getOrCreateSchemaResponse_).mergeFrom(value).buildPartial(); + } else { + getOrCreateSchemaResponse_ = value; + } + + bitField1_ |= 0x00000080; + return this; + } + public Builder clearGetOrCreateSchemaResponse() { + getOrCreateSchemaResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandGetOrCreateSchemaResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000080); + return this; + } + + // optional .pulsar.proto.CommandNewTxn newTxn = 50; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); + public boolean hasNewTxn() { + return ((bitField1_ & 0x00000100) == 0x00000100); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn getNewTxn() { + return newTxn_; + } + public Builder setNewTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn value) { + if (value == null) { + throw new NullPointerException(); + } + newTxn_ = value; + + bitField1_ |= 0x00000100; + return this; + } + public Builder setNewTxn( + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.Builder builderForValue) { + newTxn_ = builderForValue.build(); + + bitField1_ |= 0x00000100; + return this; + } + public Builder mergeNewTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn value) { + if (((bitField1_ & 0x00000100) == 0x00000100) && + newTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance()) { + newTxn_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.newBuilder(newTxn_).mergeFrom(value).buildPartial(); + } else { + newTxn_ = value; + } + + bitField1_ |= 0x00000100; + return this; + } + public Builder clearNewTxn() { + newTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxn.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000100); + return this; + } + + // optional .pulsar.proto.CommandNewTxnResponse newTxnResponse = 51; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); + public boolean hasNewTxnResponse() { + return ((bitField1_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse getNewTxnResponse() { + return newTxnResponse_; + } + public Builder setNewTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse value) { + if (value == null) { + throw new NullPointerException(); + } + newTxnResponse_ = value; + + bitField1_ |= 0x00000200; + return this; + } + public Builder setNewTxnResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.Builder builderForValue) { + newTxnResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000200; + return this; + } + public Builder mergeNewTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse value) { + if (((bitField1_ & 0x00000200) == 0x00000200) && + newTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance()) { + newTxnResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.newBuilder(newTxnResponse_).mergeFrom(value).buildPartial(); + } else { + newTxnResponse_ = value; + } + + bitField1_ |= 0x00000200; + return this; + } + public Builder clearNewTxnResponse() { + newTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandNewTxnResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000200); + return this; + } + + // optional .pulsar.proto.CommandAddPartitionToTxn addPartitionToTxn = 52; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); + public boolean hasAddPartitionToTxn() { + return ((bitField1_ & 0x00000400) == 0x00000400); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn getAddPartitionToTxn() { + return addPartitionToTxn_; + } + public Builder setAddPartitionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn value) { + if (value == null) { + throw new NullPointerException(); + } + addPartitionToTxn_ = value; + + bitField1_ |= 0x00000400; + return this; + } + public Builder setAddPartitionToTxn( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.Builder builderForValue) { + addPartitionToTxn_ = builderForValue.build(); + + bitField1_ |= 0x00000400; + return this; + } + public Builder mergeAddPartitionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn value) { + if (((bitField1_ & 0x00000400) == 0x00000400) && + addPartitionToTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance()) { + addPartitionToTxn_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.newBuilder(addPartitionToTxn_).mergeFrom(value).buildPartial(); + } else { + addPartitionToTxn_ = value; + } + + bitField1_ |= 0x00000400; + return this; + } + public Builder clearAddPartitionToTxn() { + addPartitionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxn.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000400); + return this; + } + + // optional .pulsar.proto.CommandAddPartitionToTxnResponse addPartitionToTxnResponse = 53; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); + public boolean hasAddPartitionToTxnResponse() { + return ((bitField1_ & 0x00000800) == 0x00000800); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse getAddPartitionToTxnResponse() { + return addPartitionToTxnResponse_; + } + public Builder setAddPartitionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse value) { + if (value == null) { + throw new NullPointerException(); + } + addPartitionToTxnResponse_ = value; + + bitField1_ |= 0x00000800; + return this; + } + public Builder setAddPartitionToTxnResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.Builder builderForValue) { + addPartitionToTxnResponse_ = builderForValue.build(); + + bitField1_ |= 0x00000800; + return this; + } + public Builder mergeAddPartitionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse value) { + if (((bitField1_ & 0x00000800) == 0x00000800) && + addPartitionToTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance()) { + addPartitionToTxnResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.newBuilder(addPartitionToTxnResponse_).mergeFrom(value).buildPartial(); + } else { + addPartitionToTxnResponse_ = value; + } + + bitField1_ |= 0x00000800; + return this; + } + public Builder clearAddPartitionToTxnResponse() { + addPartitionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddPartitionToTxnResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00000800); + return this; + } + + // optional .pulsar.proto.CommandAddSubscriptionToTxn addSubscriptionToTxn = 54; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); + public boolean hasAddSubscriptionToTxn() { + return ((bitField1_ & 0x00001000) == 0x00001000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn getAddSubscriptionToTxn() { + return addSubscriptionToTxn_; + } + public Builder setAddSubscriptionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn value) { + if (value == null) { + throw new NullPointerException(); + } + addSubscriptionToTxn_ = value; + + bitField1_ |= 0x00001000; + return this; + } + public Builder setAddSubscriptionToTxn( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.Builder builderForValue) { + addSubscriptionToTxn_ = builderForValue.build(); + + bitField1_ |= 0x00001000; + return this; + } + public Builder mergeAddSubscriptionToTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn value) { + if (((bitField1_ & 0x00001000) == 0x00001000) && + addSubscriptionToTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance()) { + addSubscriptionToTxn_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.newBuilder(addSubscriptionToTxn_).mergeFrom(value).buildPartial(); + } else { + addSubscriptionToTxn_ = value; + } + + bitField1_ |= 0x00001000; + return this; + } + public Builder clearAddSubscriptionToTxn() { + addSubscriptionToTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxn.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00001000); + return this; + } + + // optional .pulsar.proto.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse = 55; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); + public boolean hasAddSubscriptionToTxnResponse() { + return ((bitField1_ & 0x00002000) == 0x00002000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse getAddSubscriptionToTxnResponse() { + return addSubscriptionToTxnResponse_; + } + public Builder setAddSubscriptionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse value) { + if (value == null) { + throw new NullPointerException(); + } + addSubscriptionToTxnResponse_ = value; + + bitField1_ |= 0x00002000; + return this; + } + public Builder setAddSubscriptionToTxnResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.Builder builderForValue) { + addSubscriptionToTxnResponse_ = builderForValue.build(); + + bitField1_ |= 0x00002000; + return this; + } + public Builder mergeAddSubscriptionToTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse value) { + if (((bitField1_ & 0x00002000) == 0x00002000) && + addSubscriptionToTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance()) { + addSubscriptionToTxnResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.newBuilder(addSubscriptionToTxnResponse_).mergeFrom(value).buildPartial(); + } else { + addSubscriptionToTxnResponse_ = value; + } + + bitField1_ |= 0x00002000; + return this; + } + public Builder clearAddSubscriptionToTxnResponse() { + addSubscriptionToTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandAddSubscriptionToTxnResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00002000); + return this; + } + + // optional .pulsar.proto.CommandEndTxn endTxn = 56; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); + public boolean hasEndTxn() { + return ((bitField1_ & 0x00004000) == 0x00004000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn getEndTxn() { + return endTxn_; + } + public Builder setEndTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn value) { + if (value == null) { + throw new NullPointerException(); + } + endTxn_ = value; + + bitField1_ |= 0x00004000; + return this; + } + public Builder setEndTxn( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.Builder builderForValue) { + endTxn_ = builderForValue.build(); + + bitField1_ |= 0x00004000; + return this; + } + public Builder mergeEndTxn(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn value) { + if (((bitField1_ & 0x00004000) == 0x00004000) && + endTxn_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance()) { + endTxn_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.newBuilder(endTxn_).mergeFrom(value).buildPartial(); + } else { + endTxn_ = value; + } + + bitField1_ |= 0x00004000; + return this; + } + public Builder clearEndTxn() { + endTxn_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxn.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00004000); + return this; + } + + // optional .pulsar.proto.CommandEndTxnResponse endTxnResponse = 57; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); + public boolean hasEndTxnResponse() { + return ((bitField1_ & 0x00008000) == 0x00008000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse getEndTxnResponse() { + return endTxnResponse_; + } + public Builder setEndTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse value) { + if (value == null) { + throw new NullPointerException(); + } + endTxnResponse_ = value; + + bitField1_ |= 0x00008000; + return this; + } + public Builder setEndTxnResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.Builder builderForValue) { + endTxnResponse_ = builderForValue.build(); + + bitField1_ |= 0x00008000; + return this; + } + public Builder mergeEndTxnResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse value) { + if (((bitField1_ & 0x00008000) == 0x00008000) && + endTxnResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance()) { + endTxnResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.newBuilder(endTxnResponse_).mergeFrom(value).buildPartial(); + } else { + endTxnResponse_ = value; + } + + bitField1_ |= 0x00008000; + return this; + } + public Builder clearEndTxnResponse() { + endTxnResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00008000); + return this; + } + + // optional .pulsar.proto.CommandEndTxnOnPartition endTxnOnPartition = 58; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); + public boolean hasEndTxnOnPartition() { + return ((bitField1_ & 0x00010000) == 0x00010000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition getEndTxnOnPartition() { + return endTxnOnPartition_; + } + public Builder setEndTxnOnPartition(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition value) { + if (value == null) { + throw new NullPointerException(); + } + endTxnOnPartition_ = value; + + bitField1_ |= 0x00010000; + return this; + } + public Builder setEndTxnOnPartition( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.Builder builderForValue) { + endTxnOnPartition_ = builderForValue.build(); + + bitField1_ |= 0x00010000; + return this; + } + public Builder mergeEndTxnOnPartition(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition value) { + if (((bitField1_ & 0x00010000) == 0x00010000) && + endTxnOnPartition_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance()) { + endTxnOnPartition_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.newBuilder(endTxnOnPartition_).mergeFrom(value).buildPartial(); + } else { + endTxnOnPartition_ = value; + } + + bitField1_ |= 0x00010000; + return this; + } + public Builder clearEndTxnOnPartition() { + endTxnOnPartition_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartition.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00010000); + return this; + } + + // optional .pulsar.proto.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse = 59; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); + public boolean hasEndTxnOnPartitionResponse() { + return ((bitField1_ & 0x00020000) == 0x00020000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse getEndTxnOnPartitionResponse() { + return endTxnOnPartitionResponse_; + } + public Builder setEndTxnOnPartitionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse value) { + if (value == null) { + throw new NullPointerException(); + } + endTxnOnPartitionResponse_ = value; + + bitField1_ |= 0x00020000; + return this; + } + public Builder setEndTxnOnPartitionResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.Builder builderForValue) { + endTxnOnPartitionResponse_ = builderForValue.build(); + + bitField1_ |= 0x00020000; + return this; + } + public Builder mergeEndTxnOnPartitionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse value) { + if (((bitField1_ & 0x00020000) == 0x00020000) && + endTxnOnPartitionResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance()) { + endTxnOnPartitionResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.newBuilder(endTxnOnPartitionResponse_).mergeFrom(value).buildPartial(); + } else { + endTxnOnPartitionResponse_ = value; + } + + bitField1_ |= 0x00020000; + return this; + } + public Builder clearEndTxnOnPartitionResponse() { + endTxnOnPartitionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnPartitionResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00020000); + return this; + } + + // optional .pulsar.proto.CommandEndTxnOnSubscription endTxnOnSubscription = 60; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); + public boolean hasEndTxnOnSubscription() { + return ((bitField1_ & 0x00040000) == 0x00040000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription getEndTxnOnSubscription() { + return endTxnOnSubscription_; + } + public Builder setEndTxnOnSubscription(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription value) { + if (value == null) { + throw new NullPointerException(); + } + endTxnOnSubscription_ = value; + + bitField1_ |= 0x00040000; + return this; + } + public Builder setEndTxnOnSubscription( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.Builder builderForValue) { + endTxnOnSubscription_ = builderForValue.build(); + + bitField1_ |= 0x00040000; + return this; + } + public Builder mergeEndTxnOnSubscription(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription value) { + if (((bitField1_ & 0x00040000) == 0x00040000) && + endTxnOnSubscription_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance()) { + endTxnOnSubscription_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.newBuilder(endTxnOnSubscription_).mergeFrom(value).buildPartial(); + } else { + endTxnOnSubscription_ = value; + } + + bitField1_ |= 0x00040000; + return this; + } + public Builder clearEndTxnOnSubscription() { + endTxnOnSubscription_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscription.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00040000); + return this; + } + + // optional .pulsar.proto.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse = 61; + private org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); + public boolean hasEndTxnOnSubscriptionResponse() { + return ((bitField1_ & 0x00080000) == 0x00080000); + } + public org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse getEndTxnOnSubscriptionResponse() { + return endTxnOnSubscriptionResponse_; + } + public Builder setEndTxnOnSubscriptionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse value) { + if (value == null) { + throw new NullPointerException(); + } + endTxnOnSubscriptionResponse_ = value; + + bitField1_ |= 0x00080000; + return this; + } + public Builder setEndTxnOnSubscriptionResponse( + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.Builder builderForValue) { + endTxnOnSubscriptionResponse_ = builderForValue.build(); + + bitField1_ |= 0x00080000; + return this; + } + public Builder mergeEndTxnOnSubscriptionResponse(org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse value) { + if (((bitField1_ & 0x00080000) == 0x00080000) && + endTxnOnSubscriptionResponse_ != org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance()) { + endTxnOnSubscriptionResponse_ = + org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.newBuilder(endTxnOnSubscriptionResponse_).mergeFrom(value).buildPartial(); + } else { + endTxnOnSubscriptionResponse_ = value; + } + + bitField1_ |= 0x00080000; + return this; + } + public Builder clearEndTxnOnSubscriptionResponse() { + endTxnOnSubscriptionResponse_ = org.apache.pulsar.common.api.proto.PulsarApi.CommandEndTxnOnSubscriptionResponse.getDefaultInstance(); + + bitField1_ = (bitField1_ & ~0x00080000); + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.BaseCommand) + } + + static { + defaultInstance = new BaseCommand(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.BaseCommand) + } + + + static { + } + + // @@protoc_insertion_point(outer_class_scope) +} From 0799dfd8720d420ae34d95b23b826d3a5929e147 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 30 Jan 2020 22:34:17 +0800 Subject: [PATCH 04/21] Fix checkstyle --- .../main/java/org/apache/pulsar/common/protocol/Commands.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 4b6ce54b6062e..c44f51c8cb636 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -40,7 +40,6 @@ import lombok.experimental.UtilityClass; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Range; From cab09482c608d202922983321a78ef63465a2853 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Fri, 31 Jan 2020 10:28:12 +0800 Subject: [PATCH 05/21] Fix unit test --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 2e318f2e91225..3db9c7549cb17 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1532,9 +1532,9 @@ public void asyncMarkDelete(final Position position, Map propertie if (config.isBatchIndexDeleteEnabled()) { if (newPosition.ackSet != null) { batchDeletedIndexes.put(newPosition, newPosition.ackSet); + newPosition = ledger.getPreviousPosition(newPosition); } batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition).clear(); - newPosition = ledger.getPreviousPosition(newPosition); } else if (newPosition.ackSet != null) { callback.markDeleteFailed(new ManagedLedgerException("Batch ack set not support"), ctx); return; From e320c6f3393daf0780cc8b1e17f1c8bae6222b63 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 13 Feb 2020 20:17:54 +0800 Subject: [PATCH 06/21] Generate PulsarApi.java --- .../pulsar/common/api/proto/PulsarApi.java | 404 ++++++++++++++++++ 1 file changed, 404 insertions(+) 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 a00005f43def6..0869499ce082f 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 @@ -6473,6 +6473,10 @@ public interface CommandConnectOrBuilder // optional string original_auth_method = 9; boolean hasOriginalAuthMethod(); String getOriginalAuthMethod(); + + // optional .pulsar.proto.FeatureFlags feature_flags = 10; + boolean hasFeatureFlags(); + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags(); } public static final class CommandConnect extends org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite @@ -6731,6 +6735,16 @@ private org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString getOriginal } } + // optional .pulsar.proto.FeatureFlags feature_flags = 10; + public static final int FEATURE_FLAGS_FIELD_NUMBER = 10; + private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags featureFlags_; + public boolean hasFeatureFlags() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags() { + return featureFlags_; + } + private void initFields() { clientVersion_ = ""; authMethod_ = org.apache.pulsar.common.api.proto.PulsarApi.AuthMethod.AuthMethodNone; @@ -6741,6 +6755,7 @@ private void initFields() { originalPrincipal_ = ""; originalAuthData_ = ""; originalAuthMethod_ = ""; + featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -6790,6 +6805,9 @@ public void writeTo(org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStr if (((bitField0_ & 0x00000100) == 0x00000100)) { output.writeBytes(9, getOriginalAuthMethodBytes()); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeMessage(10, featureFlags_); + } } private int memoizedSerializedSize = -1; @@ -6834,6 +6852,10 @@ public int getSerializedSize() { size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream .computeBytesSize(9, getOriginalAuthMethodBytes()); } + if (((bitField0_ & 0x00000200) == 0x00000200)) { + size += org.apache.pulsar.shaded.com.google.protobuf.v241.CodedOutputStream + .computeMessageSize(10, featureFlags_); + } memoizedSerializedSize = size; return size; } @@ -6965,6 +6987,8 @@ public Builder clear() { bitField0_ = (bitField0_ & ~0x00000080); originalAuthMethod_ = ""; bitField0_ = (bitField0_ & ~0x00000100); + featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); + bitField0_ = (bitField0_ & ~0x00000200); return this; } @@ -7034,6 +7058,10 @@ public org.apache.pulsar.common.api.proto.PulsarApi.CommandConnect buildPartial( to_bitField0_ |= 0x00000100; } result.originalAuthMethod_ = originalAuthMethod_; + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.featureFlags_ = featureFlags_; result.bitField0_ = to_bitField0_; return result; } @@ -7067,6 +7095,9 @@ public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.CommandCon if (other.hasOriginalAuthMethod()) { setOriginalAuthMethod(other.getOriginalAuthMethod()); } + if (other.hasFeatureFlags()) { + mergeFeatureFlags(other.getFeatureFlags()); + } return this; } @@ -7149,6 +7180,16 @@ public Builder mergeFrom( originalAuthMethod_ = input.readBytes(); break; } + case 82: { + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.Builder subBuilder = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.newBuilder(); + if (hasFeatureFlags()) { + subBuilder.mergeFrom(getFeatureFlags()); + } + input.readMessage(subBuilder, extensionRegistry); + setFeatureFlags(subBuilder.buildPartial()); + subBuilder.recycle(); + break; + } } } } @@ -7440,6 +7481,49 @@ void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.Byt } + // optional .pulsar.proto.FeatureFlags feature_flags = 10; + private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); + public boolean hasFeatureFlags() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getFeatureFlags() { + return featureFlags_; + } + public Builder setFeatureFlags(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags value) { + if (value == null) { + throw new NullPointerException(); + } + featureFlags_ = value; + + bitField0_ |= 0x00000200; + return this; + } + public Builder setFeatureFlags( + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.Builder builderForValue) { + featureFlags_ = builderForValue.build(); + + bitField0_ |= 0x00000200; + return this; + } + public Builder mergeFeatureFlags(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags value) { + if (((bitField0_ & 0x00000200) == 0x00000200) && + featureFlags_ != org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance()) { + featureFlags_ = + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.newBuilder(featureFlags_).mergeFrom(value).buildPartial(); + } else { + featureFlags_ = value; + } + + bitField0_ |= 0x00000200; + return this; + } + public Builder clearFeatureFlags() { + featureFlags_ = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); + + bitField0_ = (bitField0_ & ~0x00000200); + return this; + } + // @@protoc_insertion_point(builder_scope:pulsar.proto.CommandConnect) } @@ -7451,6 +7535,326 @@ void setOriginalAuthMethod(org.apache.pulsar.shaded.com.google.protobuf.v241.Byt // @@protoc_insertion_point(class_scope:pulsar.proto.CommandConnect) } + public interface FeatureFlagsOrBuilder + extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { + + // optional bool supports_auth_refresh = 1 [default = false]; + boolean hasSupportsAuthRefresh(); + boolean getSupportsAuthRefresh(); + } + public static final class FeatureFlags extends + org.apache.pulsar.shaded.com.google.protobuf.v241.GeneratedMessageLite + implements FeatureFlagsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream.ByteBufGeneratedMessage { + // Use FeatureFlags.newBuilder() to construct. + private io.netty.util.Recycler.Handle handle; + private FeatureFlags(io.netty.util.Recycler.Handle handle) { + this.handle = handle; + } + + private static final io.netty.util.Recycler RECYCLER = new io.netty.util.Recycler() { + protected FeatureFlags newObject(Handle handle) { + return new FeatureFlags(handle); + } + }; + + public void recycle() { + this.initFields(); + this.memoizedIsInitialized = -1; + this.bitField0_ = 0; + this.memoizedSerializedSize = -1; + if (handle != null) { RECYCLER.recycle(this, handle); } + } + + private FeatureFlags(boolean noInit) {} + + private static final FeatureFlags defaultInstance; + public static FeatureFlags getDefaultInstance() { + return defaultInstance; + } + + public FeatureFlags getDefaultInstanceForType() { + return defaultInstance; + } + + private int bitField0_; + // optional bool supports_auth_refresh = 1 [default = false]; + public static final int SUPPORTS_AUTH_REFRESH_FIELD_NUMBER = 1; + private boolean supportsAuthRefresh_; + public boolean hasSupportsAuthRefresh() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public boolean getSupportsAuthRefresh() { + return supportsAuthRefresh_; + } + + private void initFields() { + supportsAuthRefresh_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + 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.writeBool(1, supportsAuthRefresh_); + } + } + + 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 + .computeBoolSize(1, supportsAuthRefresh_); + } + 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags 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.PulsarApi.FeatureFlags, Builder> + implements org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlagsOrBuilder, org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream.ByteBufMessageBuilder { + // Construct using org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.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(); + supportsAuthRefresh_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags getDefaultInstanceForType() { + return org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance(); + } + + public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags build() { + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags buildParsed() + throws org.apache.pulsar.shaded.com.google.protobuf.v241.InvalidProtocolBufferException { + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags buildPartial() { + org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags result = org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.RECYCLER.get(); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.supportsAuthRefresh_ = supportsAuthRefresh_; + result.bitField0_ = to_bitField0_; + return result; + } + + public Builder mergeFrom(org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags other) { + if (other == org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags.getDefaultInstance()) return this; + if (other.hasSupportsAuthRefresh()) { + setSupportsAuthRefresh(other.getSupportsAuthRefresh()); + } + return this; + } + + public final boolean isInitialized() { + 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; + supportsAuthRefresh_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // optional bool supports_auth_refresh = 1 [default = false]; + private boolean supportsAuthRefresh_ ; + public boolean hasSupportsAuthRefresh() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public boolean getSupportsAuthRefresh() { + return supportsAuthRefresh_; + } + public Builder setSupportsAuthRefresh(boolean value) { + bitField0_ |= 0x00000001; + supportsAuthRefresh_ = value; + + return this; + } + public Builder clearSupportsAuthRefresh() { + bitField0_ = (bitField0_ & ~0x00000001); + supportsAuthRefresh_ = false; + + return this; + } + + // @@protoc_insertion_point(builder_scope:pulsar.proto.FeatureFlags) + } + + static { + defaultInstance = new FeatureFlags(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:pulsar.proto.FeatureFlags) + } + public interface CommandConnectedOrBuilder extends org.apache.pulsar.shaded.com.google.protobuf.v241.MessageLiteOrBuilder { From 76217ac6147160ef79a7dc5b604ada6604c08f79 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 13 Feb 2020 20:43:45 +0800 Subject: [PATCH 07/21] Fix checkstyle --- .../main/java/org/apache/pulsar/common/protocol/Commands.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index c44f51c8cb636..06de1dc7e5819 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -105,7 +105,6 @@ import org.apache.pulsar.common.api.proto.PulsarApi.CommandSuccess; import org.apache.pulsar.common.api.proto.PulsarApi.CommandUnsubscribe; import org.apache.pulsar.common.api.proto.PulsarApi.FeatureFlags; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.api.proto.PulsarApi.KeyLongValue; import org.apache.pulsar.common.api.proto.PulsarApi.KeyValue; import org.apache.pulsar.common.api.proto.PulsarApi.MessageIdData; From dd28770994dc3ba6911838eabb8d633327bf7598 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Tue, 18 Feb 2020 17:49:58 +0800 Subject: [PATCH 08/21] Apply comments --- conf/broker.conf | 2 +- .../bookkeeper/mledger/ManagedCursor.java | 2 +- .../mledger/ManagedLedgerConfig.java | 10 ++-- .../mledger/impl/ManagedCursorImpl.java | 54 +++++++++---------- .../src/main/proto/MLDataFormats.proto | 8 +-- .../impl/ManagedCursorContainerTest.java | 2 +- .../mledger/impl/ManagedCursorTest.java | 18 +++---- .../pulsar/broker/ServiceConfiguration.java | 2 +- .../service/AbstractBaseDispatcher.java | 3 +- .../pulsar/broker/service/BrokerService.java | 3 +- .../impl/BatchMessageIndexAckDisableTest.java | 2 +- ...sistentAcknowledgmentsGroupingTracker.java | 12 ++--- .../pulsar/common/api/proto/PulsarApi.java | 8 +-- pulsar-common/src/main/proto/PulsarApi.proto | 2 +- 14 files changed, 64 insertions(+), 64 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index dddd8dded0939..94f9572a4cef8 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -326,7 +326,7 @@ delayedDeliveryEnabled=true delayedDeliveryTickTimeMillis=1000 # Whether to enable acknowledge of batch local index. -batchIndexAcknowledgeEnable = true +acknowledgmentAtBatchIndexLevelEnabled=true # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index 3cb55e3d0774e..9540f927d4fe2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -613,5 +613,5 @@ Set asyncReplayEntries( /** * Get deleted batch indexes list for a batch message. */ - long[] getDeletedBatchIndexesLongArray(PositionImpl position); + long[] getDeletedBatchIndexesAsLongArray(PositionImpl position); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 57ebbe8c14a45..dde67c2632f4c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -43,7 +43,7 @@ public class ManagedLedgerConfig { private boolean createIfMissing = true; private int maxUnackedRangesToPersist = 10000; private int maxBatchDeletedIndexToPersist = 10000; - private boolean batchIndexDeleteEnabled = true; + private boolean deletionAtBatchIndexLevelEnabled = true; private int maxUnackedRangesToPersistInZk = 1000; private int maxEntriesPerLedger = 50000; private int maxSizePerLedgerMb = 100; @@ -595,11 +595,11 @@ public void setBookKeeperEnsemblePlacementPolicyProperties( this.bookKeeperEnsemblePlacementPolicyProperties = bookKeeperEnsemblePlacementPolicyProperties; } - public boolean isBatchIndexDeleteEnabled() { - return batchIndexDeleteEnabled; + public boolean isDeletionAtBatchIndexLevelEnabled() { + return deletionAtBatchIndexLevelEnabled; } - public void setBatchIndexDeleteEnabled(boolean batchIndexDeleteEnabled) { - this.batchIndexDeleteEnabled = batchIndexDeleteEnabled; + public void setDeletionAtBatchIndexLevelEnabled(boolean deletionAtBatchIndexLevelEnabled) { + this.deletionAtBatchIndexLevelEnabled = deletionAtBatchIndexLevelEnabled; } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 3db9c7549cb17..a9bf544cb6d10 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -160,8 +160,8 @@ public class ManagedCursorImpl implements ManagedCursor { }; private final LongPairRangeSet individualDeletedMessages; - // Maintain the indexes deleted status of batch messages that not deleted completely - // (ledgerId, entryId) -> deleted indexes + // Maintain the deletion status for batch messages + // (ledgerId, entryId) -> deletion indexes private final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); @@ -234,7 +234,7 @@ public interface VoidCallback { this.individualDeletedMessages = config.isUnackedRangesOpenCacheSetEnabled() ? new ConcurrentOpenLongPairRangeSet<>(4096, positionRangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(positionRangeConverter); - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { this.batchDeletedIndexes = new ConcurrentSkipListMap<>(); } else { this.batchDeletedIndexes = null; @@ -386,8 +386,8 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } - if (config.isBatchIndexDeleteEnabled() && positionInfo.getBatchDeletedIndexesCount() > 0) { - recoverBatchDeletedIndexes(positionInfo.getBatchDeletedIndexesList()); + if (config.isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { + recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } recoveredCursor(position, recoveredProperties, lh); callback.operationComplete(); @@ -408,15 +408,15 @@ private void recoverIndividualDeletedMessages(List i } } - private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList) { + private void recoverBatchDeletedIndexes (List batchDeletedIndexInfoList) { lock.writeLock().lock(); try { this.batchDeletedIndexes.clear(); batchDeletedIndexInfoList.forEach(batchDeletedIndexInfo -> { - if (batchDeletedIndexInfo.getAckBitSetCount() > 0) { - long[] array = new long[batchDeletedIndexInfo.getAckBitSetCount()]; - for (int i = 0; i < batchDeletedIndexInfo.getAckBitSetList().size(); i++) { - array[i] = batchDeletedIndexInfo.getAckBitSetList().get(i); + if (batchDeletedIndexInfo.getDeleteSetCount() > 0) { + long[] array = new long[batchDeletedIndexInfo.getDeleteSetCount()]; + for (int i = 0; i < batchDeletedIndexInfo.getDeleteSetList().size(); i++) { + array[i] = batchDeletedIndexInfo.getDeleteSetList().get(i); } this.batchDeletedIndexes.put(PositionImpl.get(batchDeletedIndexInfo.getPosition().getLedgerId(), batchDeletedIndexInfo.getPosition().getEntryId()), BitSet.valueOf(array)); @@ -936,7 +936,7 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.clear(); } @@ -1529,7 +1529,7 @@ public void asyncMarkDelete(final Position position, Map propertie PositionImpl newPosition = (PositionImpl) position; - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { if (newPosition.ackSet != null) { batchDeletedIndexes.put(newPosition, newPosition.ackSet); newPosition = ledger.getPreviousPosition(newPosition); @@ -1631,7 +1631,7 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true).clear(); } } finally { @@ -1768,7 +1768,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) || position.compareTo(markDeletePosition) <= 0) { - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.remove(position); } if (log.isDebugEnabled()) { @@ -1777,7 +1777,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb continue; } if (position.ackSet == null) { - if (config.isBatchIndexDeleteEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.remove(position); } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make @@ -1791,7 +1791,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); } - } else if (config.isBatchIndexDeleteEnabled()) { + } else if (config.isDeletionAtBatchIndexLevelEnabled()) { BitSet bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSet.valueOf(position.ackSet.toLongArray())); bitSet.and(position.ackSet); if (bitSet.isEmpty()) { @@ -2112,8 +2112,8 @@ private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl positio info.addAllProperties(buildPropertiesMap(properties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); - if (config.isBatchIndexDeleteEnabled()) { - info.addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()); + if (config.isDeletionAtBatchIndexLevelEnabled()) { + info.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()); } } @@ -2359,27 +2359,27 @@ private List buildIndividualDeletedMessageRanges() { } } - private List buildBatchDeletedIndexInfoList() { + private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { - if (!config.isBatchIndexDeleteEnabled() || batchDeletedIndexes.isEmpty()) { + if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo .newBuilder(); - MLDataFormats.BatchDeletedIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchDeletedIndexInfo + MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo .newBuilder(); - List result = Lists.newArrayList(); + List result = Lists.newArrayList(); for (Map.Entry entry : batchDeletedIndexes.entrySet()) { nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); long[] array = entry.getValue().toLongArray(); - List ackBitSet = new ArrayList<>(array.length); + List deleteSet = new ArrayList<>(array.length); for (long l : array) { - ackBitSet.add(l); + deleteSet.add(l); } - batchDeletedIndexInfoBuilder.addAllAckBitSet(ackBitSet); + batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); result.add(batchDeletedIndexInfoBuilder.build()); if (result.size() >= config.getMaxBatchDeletedIndexToPersist()) { break; @@ -2396,7 +2396,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) - .addAllBatchDeletedIndexes(buildBatchDeletedIndexInfoList()) + .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()) .addAllProperties(buildPropertiesMap(mdEntry.properties)).build(); @@ -2780,7 +2780,7 @@ private ManagedCursorImpl cursorImpl() { } @Override - public long[] getDeletedBatchIndexesLongArray(PositionImpl position) { + public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { lock.readLock().lock(); try { BitSet bitSet = batchDeletedIndexes.get(position); diff --git a/managed-ledger/src/main/proto/MLDataFormats.proto b/managed-ledger/src/main/proto/MLDataFormats.proto index 9110663069c56..adef6dcdfb40b 100644 --- a/managed-ledger/src/main/proto/MLDataFormats.proto +++ b/managed-ledger/src/main/proto/MLDataFormats.proto @@ -68,7 +68,7 @@ message PositionInfo { repeated LongProperty properties = 4; // Store which index in the batch message has been deleted - repeated BatchDeletedIndexInfo batchDeletedIndexes = 5; + repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 5; } message NestedPositionInfo { @@ -81,9 +81,9 @@ message MessageRange { required NestedPositionInfo upperEndpoint = 2; } -message BatchDeletedIndexInfo { +message BatchedEntryDeletionIndexInfo { required NestedPositionInfo position = 1; - repeated int64 ackBitSet = 2; + repeated int64 deleteSet = 2; } // Generic string and long tuple @@ -109,5 +109,5 @@ message ManagedCursorInfo { optional int64 lastActive = 6; // Store which index in the batch message has been deleted - repeated BatchDeletedIndexInfo batchDeletedIndexes = 7; + repeated BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = 7; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java index 0c796b4604889..b599800da4639 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorContainerTest.java @@ -328,7 +328,7 @@ public void trimDeletedEntries(List entries) { } @Override - public long[] getDeletedBatchIndexesLongArray(PositionImpl position) { + public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { return new long[0]; } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 8c1d02534bafb..4b8c42d7a9c39 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3066,19 +3066,19 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc } assertEquals(cursor.getNumberOfEntries(), totalEntries); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(2).setEnd(4).build())); - List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(4, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(3).setEnd(8).build())); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(1, deletedIndexes.size()); Assert.assertEquals(2, deletedIndexes.get(0).getStart()); Assert.assertEquals(8, deletedIndexes.get(0).getEnd()); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(0).build())); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); Assert.assertEquals(2, deletedIndexes.size()); Assert.assertEquals(0, deletedIndexes.get(0).getStart()); Assert.assertEquals(0, deletedIndexes.get(0).getEnd()); @@ -3087,24 +3087,24 @@ public void testBatchIndexDelete() throws ManagedLedgerException, InterruptedExc deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(1).setEnd(1).build())); deleteBatchIndex(cursor, positions[0], 10, Lists.newArrayList(IntRange.newBuilder().setStart(9).setEnd(9).build())); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[0]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[0]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(positions[0], cursor.getMarkDeletedPosition()); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.delete(positions[1]); deleteBatchIndex(cursor, positions[1], 10, Lists.newArrayList(IntRange.newBuilder().setStart(6).setEnd(8).build())); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[1]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[1]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[2], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.markDelete(positions[3]); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[2]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[2]), 10); Assert.assertNull(deletedIndexes); deleteBatchIndex(cursor, positions[3], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(5).build())); cursor.resetCursor(positions[0]); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[3]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[3]), 10); Assert.assertNull(deletedIndexes); } @@ -3129,13 +3129,13 @@ public void testBatchIndexesDeletionPersistAndRecover() throws ManagedLedgerExce ledger = factory.open("test_batch_indexes_deletion_persistent"); cursor = ledger.openCursor("c1"); - List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[5]), 10); + List deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[5]), 10); Assert.assertEquals(deletedIndexes.size(), 1); Assert.assertEquals(deletedIndexes.get(0).getStart(), 3); Assert.assertEquals(deletedIndexes.get(0).getEnd(), 6); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[4]); deleteBatchIndex(cursor, positions[5], 10, Lists.newArrayList(IntRange.newBuilder().setStart(0).setEnd(9).build())); - deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesLongArray((PositionImpl) positions[5]), 10); + deletedIndexes = getAckedIndexRange(cursor.getDeletedBatchIndexesAsLongArray((PositionImpl) positions[5]), 10); Assert.assertNull(deletedIndexes); Assert.assertEquals(cursor.getMarkDeletedPosition(), positions[5]); } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 124a8f368e4c7..00c320f8f4bd1 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -174,7 +174,7 @@ public class ServiceConfiguration implements PulsarConfiguration { private long delayedDeliveryTickTimeMillis = 1000; @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the acknowledge of batch local index") - private boolean batchIndexAcknowledgeEnable = true; + private boolean acknowledgmentAtBatchIndexLevelEnabled = true; @FieldContext( category = CATEGORY_WEBSOCKET, 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 d2b863a17a163..f5b5da2ecbbb9 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 @@ -32,7 +32,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.PulsarApi; -import org.apache.pulsar.common.api.proto.PulsarApi.IntRange; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; @@ -106,7 +105,7 @@ && trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { totalBytes += metadataAndPayload.readableBytes(); batchSizes.setBatchSize(i, batchSize); if (indexesAcks != null && cursor != null) { - long[] ackSet = cursor.getDeletedBatchIndexesLongArray(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); + long[] ackSet = cursor.getDeletedBatchIndexesAsLongArray(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); if (ackSet != null) { indexesAcks.setIndexesAcks(Pair.of(batchSize, ackSet)); } else { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 7eccd141ebcd1..d21ab8467a9c3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1056,7 +1056,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t ); } managedLedgerConfig.setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); - managedLedgerConfig.setBatchIndexDeleteEnabled(serviceConfig.isBatchIndexAcknowledgeEnable()); + + managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled(serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); future.complete(managedLedgerConfig); }, (exception) -> future.completeExceptionally(exception))); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java index 29fb24e5c59c3..71305b300cc1e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckDisableTest.java @@ -46,7 +46,7 @@ public class BatchMessageIndexAckDisableTest extends ProducerConsumerBase { @BeforeMethod @Override protected void setup() throws Exception { - conf.setBatchIndexAcknowledgeEnable(false); + conf.setAcknowledgmentAtBatchIndexLevelEnabled(false); super.internalSetup(); super.producerBaseSetup(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 64acdbb92ce56..f955859a1b5dd 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -62,13 +62,13 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * Latest cumulative ack sent to broker */ private volatile MessageIdImpl lastCumulativeAck = (MessageIdImpl) MessageId.earliest; - private volatile ConcurrentBitSet lastCumulativeAckSet = null; + private volatile BitSet lastCumulativeAckSet = null; private volatile boolean cumulativeAckFlushRequired = false; private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_UPDATER = AtomicReferenceFieldUpdater .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, MessageIdImpl.class, "lastCumulativeAck"); - private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_SET_UPDATER = AtomicReferenceFieldUpdater - .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, ConcurrentBitSet.class, "lastCumulativeAckSet"); + private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_SET_UPDATER = AtomicReferenceFieldUpdater + .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, BitSet.class, "lastCumulativeAckSet"); /** @@ -135,7 +135,7 @@ public void addBatchIndexAcknowledgment(BatchMessageIdImpl msgId, int batchIndex if (acknowledgementGroupTimeMicros == 0 || !properties.isEmpty()) { doImmediateBatchIndexAck(msgId, batchIndex, batchSize, ackType, properties); } else if (ackType == AckType.Cumulative) { - ConcurrentBitSet bitSet = new ConcurrentBitSet(batchSize); + BitSet bitSet = new BitSet(batchSize); bitSet.set(0, batchSize); bitSet.clear(0, batchIndex + 1); doCumulativeAck(msgId, bitSet); @@ -154,11 +154,11 @@ public void addBatchIndexAcknowledgment(BatchMessageIdImpl msgId, int batchIndex } } - private void doCumulativeAck(MessageIdImpl msgId, ConcurrentBitSet bitSet) { + private void doCumulativeAck(MessageIdImpl msgId, BitSet bitSet) { // Handle concurrent updates from different threads while (true) { MessageIdImpl lastCumlativeAck = this.lastCumulativeAck; - ConcurrentBitSet lastBitSet = this.lastCumulativeAckSet; + BitSet lastBitSet = this.lastCumulativeAckSet; if (msgId.compareTo(lastCumlativeAck) > 0) { if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId) && LAST_CUMULATIVE_ACK_SET_UPDATER.compareAndSet(this, lastBitSet, bitSet)) { // Successfully updated the last cumulative ack. Next flush iteration will send this to broker. 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 0869499ce082f..0ebea6af41f94 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 @@ -1134,7 +1134,7 @@ public interface MessageIdDataOrBuilder boolean hasBatchIndex(); int getBatchIndex(); - // repeated int64 ackSet = 5; + // repeated int64 ack_set = 5; java.util.List getAckSetList(); int getAckSetCount(); long getAckSet(int index); @@ -1214,8 +1214,8 @@ public int getBatchIndex() { return batchIndex_; } - // repeated int64 ackSet = 5; - public static final int ACKSET_FIELD_NUMBER = 5; + // repeated int64 ack_set = 5; + public static final int ACK_SET_FIELD_NUMBER = 5; private java.util.List ackSet_; public java.util.List getAckSetList() { @@ -1665,7 +1665,7 @@ public Builder clearBatchIndex() { return this; } - // repeated int64 ackSet = 5; + // repeated int64 ack_set = 5; private java.util.List ackSet_ = java.util.Collections.emptyList();; private void ensureAckSetIsMutable() { if (!((bitField0_ & 0x00000010) == 0x00000010)) { diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index c05d31bc94268..972b5479b7df7 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -54,7 +54,7 @@ message MessageIdData { required uint64 entryId = 2; optional int32 partition = 3 [default = -1]; optional int32 batch_index = 4 [default = -1]; - repeated int64 ackSet = 5; + repeated int64 ack_set = 5; } message KeyValue { From fa2b9f707bb1e4e49e8e01ba7f9c5651ee807f43 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Tue, 18 Feb 2020 21:28:24 +0800 Subject: [PATCH 09/21] Apply comments --- conf/broker.conf | 2 +- .../mledger/impl/ManagedCursorImpl.java | 59 ++++++++----------- .../pulsar/broker/ServiceConfiguration.java | 2 +- ...sistentAcknowledgmentsGroupingTracker.java | 29 ++++----- .../pulsar/common/protocol/Commands.java | 6 +- .../common/util/SafeCollectionUtils.java | 23 ++------ .../util/collections/BitSetRecyclable.java | 52 ++++++++++++++++ .../util/collections/ConcurrentBitSet.java | 4 ++ .../ConcurrentBitSetRecyclable.java | 50 ++++++++++++++++ .../collections/BitSetRecyclableTest.java | 37 ++++++++++++ .../ConcurrentBitSetRecyclableTest.java | 37 ++++++++++++ 11 files changed, 230 insertions(+), 71 deletions(-) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java create mode 100644 pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclableTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 94f9572a4cef8..bd3990f5c9e2f 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -326,7 +326,7 @@ delayedDeliveryEnabled=true delayedDeliveryTickTimeMillis=1000 # Whether to enable acknowledge of batch local index. -acknowledgmentAtBatchIndexLevelEnabled=true +acknowledgmentAtBatchIndexLevelEnabled=false # Enable tracking of replicated subscriptions state across clusters. enableReplicatedSubscriptions=true diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index a9bf544cb6d10..75325ddad8183 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -46,6 +46,7 @@ import java.util.ArrayList; import java.util.BitSet; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -93,6 +94,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; @@ -2360,35 +2362,29 @@ private List buildIndividualDeletedMessageRanges() { } private List buildBatchEntryDeletionIndexInfoList() { - lock.readLock().lock(); - try { - if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { - return Collections.emptyList(); - } - MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo - .newBuilder(); - MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo - .newBuilder(); - List result = Lists.newArrayList(); - for (Map.Entry entry : batchDeletedIndexes.entrySet()) { - nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); - nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); - batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); - long[] array = entry.getValue().toLongArray(); - List deleteSet = new ArrayList<>(array.length); - for (long l : array) { - deleteSet.add(l); - } - batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); - result.add(batchDeletedIndexInfoBuilder.build()); - if (result.size() >= config.getMaxBatchDeletedIndexToPersist()) { - break; - } + if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + return Collections.emptyList(); + } + MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo + .newBuilder(); + MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo + .newBuilder(); + List result = Lists.newArrayList(); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + while (iterator.hasNext() && result.size() < config.getMaxBatchDeletedIndexToPersist()) { + Map.Entry entry = iterator.next(); + nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); + nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); + batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); + long[] array = entry.getValue().toLongArray(); + List deleteSet = new ArrayList<>(array.length); + for (long l : array) { + deleteSet.add(l); } - return result; - } finally { - lock.readLock().unlock(); + batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); + result.add(batchDeletedIndexInfoBuilder.build()); } + return result; } void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { @@ -2781,13 +2777,8 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - lock.readLock().lock(); - try { - BitSet bitSet = batchDeletedIndexes.get(position); - return bitSet == null ? null : bitSet.toLongArray(); - } finally { - lock.readLock().unlock(); - } + BitSet bitSet = batchDeletedIndexes.get(position); + return bitSet == null ? null : bitSet.toLongArray(); } private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 00c320f8f4bd1..17dcd54cc5937 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -174,7 +174,7 @@ public class ServiceConfiguration implements PulsarConfiguration { private long delayedDeliveryTickTimeMillis = 1000; @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the acknowledge of batch local index") - private boolean acknowledgmentAtBatchIndexLevelEnabled = true; + private boolean acknowledgmentAtBatchIndexLevelEnabled = false; @FieldContext( category = CATEGORY_WEBSOCKET, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index f955859a1b5dd..68fe718bf25b4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -22,7 +22,6 @@ import io.netty.channel.EventLoopGroup; import java.util.ArrayList; -import java.util.BitSet; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -35,13 +34,14 @@ import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentBitSet; +import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; /** * Group the acknowledgements for a certain time and then sends them out in a single protobuf command. @@ -62,13 +62,13 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * Latest cumulative ack sent to broker */ private volatile MessageIdImpl lastCumulativeAck = (MessageIdImpl) MessageId.earliest; - private volatile BitSet lastCumulativeAckSet = null; + private volatile BitSetRecyclable lastCumulativeAckSet = null; private volatile boolean cumulativeAckFlushRequired = false; private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_UPDATER = AtomicReferenceFieldUpdater .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, MessageIdImpl.class, "lastCumulativeAck"); - private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_SET_UPDATER = AtomicReferenceFieldUpdater - .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, BitSet.class, "lastCumulativeAckSet"); + private static final AtomicReferenceFieldUpdater LAST_CUMULATIVE_ACK_SET_UPDATER = AtomicReferenceFieldUpdater + .newUpdater(PersistentAcknowledgmentsGroupingTracker.class, BitSetRecyclable.class, "lastCumulativeAckSet"); /** @@ -76,7 +76,7 @@ public class PersistentAcknowledgmentsGroupingTracker implements Acknowledgments * broker. */ private final ConcurrentSkipListSet pendingIndividualAcks; - private final ConcurrentHashMap pendingIndividualBatchIndexAcks; + private final ConcurrentHashMap pendingIndividualBatchIndexAcks; private final ScheduledFuture scheduledTask; @@ -135,14 +135,14 @@ public void addBatchIndexAcknowledgment(BatchMessageIdImpl msgId, int batchIndex if (acknowledgementGroupTimeMicros == 0 || !properties.isEmpty()) { doImmediateBatchIndexAck(msgId, batchIndex, batchSize, ackType, properties); } else if (ackType == AckType.Cumulative) { - BitSet bitSet = new BitSet(batchSize); + BitSetRecyclable bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); bitSet.clear(0, batchIndex + 1); doCumulativeAck(msgId, bitSet); } else if (ackType == AckType.Individual) { - ConcurrentBitSet bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( + ConcurrentBitSetRecyclable bitSet = pendingIndividualBatchIndexAcks.computeIfAbsent( new MessageIdImpl(msgId.getLedgerId(), msgId.getEntryId(), msgId.getPartitionIndex()), (v) -> { - ConcurrentBitSet value = new ConcurrentBitSet(batchSize); + ConcurrentBitSetRecyclable value = ConcurrentBitSetRecyclable.create(); value.set(0, batchSize + 1); value.clear(batchIndex); return value; @@ -154,13 +154,14 @@ public void addBatchIndexAcknowledgment(BatchMessageIdImpl msgId, int batchIndex } } - private void doCumulativeAck(MessageIdImpl msgId, BitSet bitSet) { + private void doCumulativeAck(MessageIdImpl msgId, BitSetRecyclable bitSet) { // Handle concurrent updates from different threads while (true) { MessageIdImpl lastCumlativeAck = this.lastCumulativeAck; - BitSet lastBitSet = this.lastCumulativeAckSet; + BitSetRecyclable lastBitSet = this.lastCumulativeAckSet; if (msgId.compareTo(lastCumlativeAck) > 0) { if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId) && LAST_CUMULATIVE_ACK_SET_UPDATER.compareAndSet(this, lastBitSet, bitSet)) { + lastBitSet.recycle(); // Successfully updated the last cumulative ack. Next flush iteration will send this to broker. cumulativeAckFlushRequired = true; return; @@ -228,7 +229,7 @@ public void flush() { } // Flush all individual acks - List> entriesToAck = new ArrayList<>(pendingIndividualAcks.size() + pendingIndividualBatchIndexAcks.size()); + List> entriesToAck = new ArrayList<>(pendingIndividualAcks.size() + pendingIndividualBatchIndexAcks.size()); if (!pendingIndividualAcks.isEmpty()) { if (Commands.peerSupportsMultiMessageAcknowledgment(cnx.getRemoteEndpointProtocolVersion())) { // We can send 1 single protobuf command with all individual acks @@ -256,10 +257,10 @@ public void flush() { } if (!pendingIndividualBatchIndexAcks.isEmpty()) { - Iterator> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); + Iterator> iterator = pendingIndividualBatchIndexAcks.entrySet().iterator(); while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); + Map.Entry entry = iterator.next(); entriesToAck.add(Triple.of(entry.getKey().ledgerId, entry.getKey().entryId, entry.getValue())); iterator.remove(); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 06de1dc7e5819..58a2a51d105c0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -119,6 +119,7 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.SafeCollectionUtils; +import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream; import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream; import org.apache.pulsar.shaded.com.google.protobuf.v241.ByteString; @@ -864,7 +865,7 @@ public static ByteBuf newLookupErrorResponse(ServerError error, String errorMsg, return res; } - public static ByteBuf newMultiMessageAck(long consumerId, List> entries) { + public static ByteBuf newMultiMessageAck(long consumerId, List> entries) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); ackBuilder.setConsumerId(consumerId); ackBuilder.setAckType(AckType.Individual); @@ -873,7 +874,7 @@ public static ByteBuf newMultiMessageAck(long consumerId, List longArrayToList(long[] array) { - if (array == null || array.length == 0) { - return Collections.emptyList(); - } else { - List result = new ArrayList<>(array.length); - for (long l : array) { - result.add(l); - } - return result; - } + return array == null || array.length == 0 ? Collections.emptyList() : Arrays.stream(array).boxed().collect(Collectors.toList()); } public static long[] longListToArray(List list) { - if (list == null || list.size() == 0) { - return new long[0]; - } else { - long[] array = new long[list.size()]; - for (int i = 0; i < list.size(); i++) { - array[i] = list.get(i); - } - return array; - } + return list == null || list.size() == 0 ? new long[0] : list.stream().mapToLong(l->l).toArray(); } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java new file mode 100644 index 0000000000000..649f5cdb80575 --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java @@ -0,0 +1,52 @@ +/** + * 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.util.collections; + +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; + +import java.util.BitSet; + +/** + * BitSet leverage netty recycler. + */ +public class BitSetRecyclable extends BitSet { + + private final Handle recyclerHandle; + + private static final Recycler RECYCLER = new Recycler() { + protected BitSetRecyclable newObject(Recycler.Handle recyclerHandle) { + return new BitSetRecyclable(recyclerHandle); + } + }; + + private BitSetRecyclable(Handle recyclerHandle) { + super(); + this.recyclerHandle = recyclerHandle; + } + + public static BitSetRecyclable create() { + return RECYCLER.get(); + } + + public void recycle() { + this.clear(); + recyclerHandle.recycle(this); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java index 8f80d03a60eb4..24decbce6e309 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSet.java @@ -29,6 +29,10 @@ public class ConcurrentBitSet extends BitSet { private static final long serialVersionUID = 1L; private final StampedLock rwLock = new StampedLock(); + public ConcurrentBitSet() { + super(); + } + /** * Creates a bit set whose initial size is large enough to explicitly represent bits with indices in the range * {@code 0} through {@code nbits-1}. All bits are initially {@code false}. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java new file mode 100644 index 0000000000000..8e787c1ea6ebb --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclable.java @@ -0,0 +1,50 @@ +/** + * 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.util.collections; + +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; + +/** + * Safe multithreaded version of {@code BitSet} and leverage netty recycler. + */ +public class ConcurrentBitSetRecyclable extends ConcurrentBitSet { + + private final Handle recyclerHandle; + + private static final Recycler RECYCLER = new Recycler() { + protected ConcurrentBitSetRecyclable newObject(Handle recyclerHandle) { + return new ConcurrentBitSetRecyclable(recyclerHandle); + } + }; + + private ConcurrentBitSetRecyclable(Handle recyclerHandle) { + super(); + this.recyclerHandle = recyclerHandle; + } + + public static ConcurrentBitSetRecyclable create() { + return RECYCLER.get(); + } + + public void recycle() { + this.clear(); + recyclerHandle.recycle(this); + } +} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java new file mode 100644 index 0000000000000..07155ad8a5f45 --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java @@ -0,0 +1,37 @@ +/** + * 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.util.collections; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class BitSetRecyclableTest { + + @Test + public void testRecycle() { + BitSetRecyclable bitset1 = BitSetRecyclable.create(); + bitset1.set(3); + bitset1.recycle(); + BitSetRecyclable bitset2 = BitSetRecyclable.create(); + BitSetRecyclable bitset3 = BitSetRecyclable.create(); + Assert.assertSame(bitset2, bitset1); + Assert.assertFalse(bitset2.get(3)); + Assert.assertNotSame(bitset3, bitset1); + } +} diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclableTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclableTest.java new file mode 100644 index 0000000000000..b037c705fae1e --- /dev/null +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentBitSetRecyclableTest.java @@ -0,0 +1,37 @@ +/** + * 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.util.collections; + +import org.testng.Assert; +import org.testng.annotations.Test; + +public class ConcurrentBitSetRecyclableTest { + + @Test + public void testRecycle() { + ConcurrentBitSetRecyclable bitset1 = ConcurrentBitSetRecyclable.create(); + bitset1.set(3); + bitset1.recycle(); + ConcurrentBitSetRecyclable bitset2 = ConcurrentBitSetRecyclable.create(); + ConcurrentBitSetRecyclable bitset3 = ConcurrentBitSetRecyclable.create(); + Assert.assertSame(bitset2, bitset1); + Assert.assertFalse(bitset2.get(3)); + Assert.assertNotSame(bitset3, bitset1); + } +} From a75e9f653532f724422f3c327ca9d48b5719b1e2 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Tue, 18 Feb 2020 21:34:54 +0800 Subject: [PATCH 10/21] Fix checkstyle --- .../main/java/org/apache/pulsar/common/protocol/Commands.java | 3 ++- .../org/apache/pulsar/common/util/SafeCollectionUtils.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 58a2a51d105c0..88fe3a41bc3a1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -865,7 +865,8 @@ public static ByteBuf newLookupErrorResponse(ServerError error, String errorMsg, return res; } - public static ByteBuf newMultiMessageAck(long consumerId, List> entries) { + public static ByteBuf newMultiMessageAck(long consumerId, + List> entries) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); ackBuilder.setConsumerId(consumerId); ackBuilder.setAckType(AckType.Individual); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java index a576041f3d533..21488fb3e5ac8 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SafeCollectionUtils.java @@ -29,7 +29,8 @@ public class SafeCollectionUtils { public static List longArrayToList(long[] array) { - return array == null || array.length == 0 ? Collections.emptyList() : Arrays.stream(array).boxed().collect(Collectors.toList()); + return array == null || array.length == 0 ? Collections.emptyList() + : Arrays.stream(array).boxed().collect(Collectors.toList()); } public static long[] longListToArray(List list) { From aa388f3dc85e9f849cba63d7df05c74382660ef8 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Tue, 18 Feb 2020 22:09:48 +0800 Subject: [PATCH 11/21] Fix tests. --- .../apache/pulsar/client/impl/BatchMessageIndexAckTest.java | 1 + .../client/impl/PersistentAcknowledgmentsGroupingTracker.java | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java index a6053a8e49f48..114b9ae3fdc1e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BatchMessageIndexAckTest.java @@ -46,6 +46,7 @@ public class BatchMessageIndexAckTest extends ProducerConsumerBase { @BeforeMethod @Override protected void setup() throws Exception { + conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); super.internalSetup(); super.producerBaseSetup(); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 68fe718bf25b4..2675a61074e03 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -161,7 +161,9 @@ private void doCumulativeAck(MessageIdImpl msgId, BitSetRecyclable bitSet) { BitSetRecyclable lastBitSet = this.lastCumulativeAckSet; if (msgId.compareTo(lastCumlativeAck) > 0) { if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId) && LAST_CUMULATIVE_ACK_SET_UPDATER.compareAndSet(this, lastBitSet, bitSet)) { - lastBitSet.recycle(); + if (lastBitSet != null) { + lastBitSet.recycle(); + } // Successfully updated the last cumulative ack. Next flush iteration will send this to broker. cumulativeAckFlushRequired = true; return; From af497bd7077ebeac08b9b6f44face015c94e85ba Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Wed, 19 Feb 2020 20:09:07 +0800 Subject: [PATCH 12/21] Introduce BitSetRecyclable that leverage with netty recycler --- .../main/resources/pulsar/suppressions.xml | 1 + .../mledger/impl/ManagedCursorImpl.java | 34 +- .../bookkeeper/mledger/impl/PositionImpl.java | 8 +- .../mledger/impl/ManagedCursorTest.java | 2 +- .../pulsar/broker/service/Consumer.java | 4 +- ...sistentAcknowledgmentsGroupingTracker.java | 3 +- .../pulsar/common/protocol/Commands.java | 7 +- .../util/collections/BitSetRecyclable.java | 1169 ++++++++++++++++- ...va => BitSetRecyclableRecyclableTest.java} | 13 +- 9 files changed, 1207 insertions(+), 34 deletions(-) rename pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/{BitSetRecyclableTest.java => BitSetRecyclableRecyclableTest.java} (76%) diff --git a/buildtools/src/main/resources/pulsar/suppressions.xml b/buildtools/src/main/resources/pulsar/suppressions.xml index 7fd94d913f377..d6d38a2dfa0a6 100644 --- a/buildtools/src/main/resources/pulsar/suppressions.xml +++ b/buildtools/src/main/resources/pulsar/suppressions.xml @@ -40,4 +40,5 @@ + diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 75325ddad8183..a546a88530af8 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -44,7 +44,6 @@ import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.BitSet; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -164,7 +163,7 @@ public class ManagedCursorImpl implements ManagedCursor { // Maintain the deletion status for batch messages // (ledgerId, entryId) -> deletion indexes - private final ConcurrentSkipListMap batchDeletedIndexes; + private final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -421,7 +420,7 @@ private void recoverBatchDeletedIndexes (List propertie if (config.isDeletionAtBatchIndexLevelEnabled()) { if (newPosition.ackSet != null) { - batchDeletedIndexes.put(newPosition, newPosition.ackSet); + batchDeletedIndexes.put(newPosition, BitSetRecyclable.create().resetWords(newPosition.ackSet)); newPosition = ledger.getPreviousPosition(newPosition); } - batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition).clear(); + Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, newPosition); + subMap.values().forEach(BitSetRecyclable::recycle); + subMap.clear(); } else if (newPosition.ackSet != null) { callback.markDeleteFailed(new ManagedLedgerException("Batch ack set not support"), ctx); return; @@ -1634,7 +1636,9 @@ public void operationComplete() { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true).clear(); + Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); + subMap.values().forEach(BitSetRecyclable::recycle); + subMap.clear(); } } finally { lock.writeLock().unlock(); @@ -1771,7 +1775,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) || position.compareTo(markDeletePosition) <= 0) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } if (log.isDebugEnabled()) { log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); @@ -1780,7 +1784,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (position.ackSet == null) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make // the RangeSet recognize the "continuity" between adjacent Positions @@ -1794,14 +1798,16 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb individualDeletedMessages); } } else if (config.isDeletionAtBatchIndexLevelEnabled()) { - BitSet bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSet.valueOf(position.ackSet.toLongArray())); - bitSet.and(position.ackSet); + BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSetRecyclable.create().resetWords(position.ackSet)); + BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); + bitSet.and(givenBitSet); + givenBitSet.recycle(); if (bitSet.isEmpty()) { PositionImpl previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); ++messagesConsumedCounter; - batchDeletedIndexes.remove(position); + batchDeletedIndexes.remove(position).recycle(); } } } @@ -2370,9 +2376,9 @@ private List buildBatchEntryDeletio MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats.BatchedEntryDeletionIndexInfo .newBuilder(); List result = Lists.newArrayList(); - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < config.getMaxBatchDeletedIndexToPersist()) { - Map.Entry entry = iterator.next(); + Map.Entry entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); @@ -2777,7 +2783,7 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - BitSet bitSet = batchDeletedIndexes.get(position); + BitSetRecyclable bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java index 7995840d5cf9c..1c35ef1f6325a 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java @@ -26,13 +26,11 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; -import java.util.BitSet; - public class PositionImpl implements Position, Comparable { protected long ledgerId; protected long entryId; - protected BitSet ackSet; + protected long[] ackSet; public static final PositionImpl earliest = new PositionImpl(-1, -1); public static final PositionImpl latest = new PositionImpl(Long.MAX_VALUE, Long.MAX_VALUE); @@ -52,7 +50,7 @@ public PositionImpl(long ledgerId, long entryId) { this.entryId = entryId; } - public PositionImpl(long ledgerId, long entryId, BitSet ackSet) { + public PositionImpl(long ledgerId, long entryId, long[] ackSet) { this.ledgerId = ledgerId; this.entryId = entryId; this.ackSet = ackSet; @@ -67,7 +65,7 @@ public static PositionImpl get(long ledgerId, long entryId) { return new PositionImpl(ledgerId, entryId); } - public static PositionImpl get(long ledgerId, long entryId, BitSet ackSet) { + public static PositionImpl get(long ledgerId, long entryId, long[] ackSet) { return new PositionImpl(ledgerId, entryId, ackSet); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java index 4b8c42d7a9c39..1ed50f4419e07 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorTest.java @@ -3149,7 +3149,7 @@ private void deleteBatchIndex(ManagedCursor cursor, Position position, int batch deleteIndexes.forEach(intRange -> { bitSet.clear(intRange.getStart(), intRange.getEnd() + 1); }); - pos.ackSet = bitSet; + pos.ackSet = bitSet.toLongArray(); cursor.asyncDelete(pos, new DeleteCallback() { 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 733e38b095ddd..2be73cd384145 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 @@ -375,7 +375,7 @@ void messageAcked(CommandAck ack) { if (ack.getMessageIdCount() == 1) { MessageIdData msgId = ack.getMessageId(0); if (msgId.getAckSetCount() > 0) { - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), SafeCollectionUtils.longListToArray(msgId.getAckSetList())); } else { position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); } @@ -388,7 +388,7 @@ void messageAcked(CommandAck ack) { MessageIdData msgId = ack.getMessageId(i); PositionImpl position; if (msgId.getAckSetCount() > 0) { - position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), BitSet.valueOf(SafeCollectionUtils.longListToArray(msgId.getAckSetList()))); + position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId(), SafeCollectionUtils.longListToArray(msgId.getAckSetList())); } else { position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index 2675a61074e03..d163d1a8dedaa 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -40,7 +40,6 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.api.proto.PulsarApi.CommandAck.AckType; import org.apache.pulsar.common.util.collections.BitSetRecyclable; -import org.apache.pulsar.common.util.collections.ConcurrentBitSet; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; /** @@ -195,7 +194,7 @@ private boolean doImmediateBatchIndexAck(BatchMessageIdImpl msgId, int batchInde if (cnx == null) { return false; } - ConcurrentBitSet bitSet = new ConcurrentBitSet(batchSize); + BitSetRecyclable bitSet = BitSetRecyclable.create(); bitSet.set(0, batchSize); if (ackType == AckType.Cumulative) { bitSet.clear(0, batchIndex + 1); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index 88fe3a41bc3a1..cede8e8c2e591 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -30,7 +30,6 @@ import io.netty.buffer.Unpooled; import java.io.IOException; -import java.util.BitSet; import java.util.Collections; import java.util.List; import java.util.Map; @@ -119,6 +118,7 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.SafeCollectionUtils; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.ConcurrentBitSetRecyclable; import org.apache.pulsar.common.util.protobuf.ByteBufCodedInputStream; import org.apache.pulsar.common.util.protobuf.ByteBufCodedOutputStream; @@ -901,12 +901,12 @@ public static ByteBuf newMultiMessageAck(long consumerId, return res; } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType, ValidationError validationError, Map properties) { return newAck(consumerId, ledgerId, entryId, ackSet, ackType, validationError, properties, 0, 0); } - public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSet ackSet, AckType ackType, + public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSetRecyclable ackSet, AckType ackType, ValidationError validationError, Map properties, long txnIdLeastBits, long txnIdMostBits) { CommandAck.Builder ackBuilder = CommandAck.newBuilder(); @@ -937,6 +937,7 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSe ByteBuf res = serializeWithSize(BaseCommand.newBuilder().setType(Type.ACK).setAck(ack)); ack.recycle(); + ackSet.recycle(); ackBuilder.recycle(); messageIdDataBuilder.recycle(); messageIdData.recycle(); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java index 649f5cdb80575..285416ac606b2 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java @@ -21,14 +21,1169 @@ import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; +import java.io.*; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.LongBuffer; +import java.util.Arrays; import java.util.BitSet; /** - * BitSet leverage netty recycler. + * This this copy of {@link BitSet}. + * Provides {@link BitSetRecyclable#resetWords(long[])} method and leverage with netty recycler. */ -public class BitSetRecyclable extends BitSet { +public class BitSetRecyclable implements Cloneable, java.io.Serializable { + /* + * BitSets are packed into arrays of "words." Currently a word is + * a long, which consists of 64 bits, requiring 6 address bits. + * The choice of word size is determined purely by performance concerns. + */ + private final static int ADDRESS_BITS_PER_WORD = 6; + private final static int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD; + private final static int BIT_INDEX_MASK = BITS_PER_WORD - 1; - private final Handle recyclerHandle; + /* Used to shift left or right for a partial word mask */ + private static final long WORD_MASK = 0xffffffffffffffffL; + + /** + * @serialField bits long[] + * + * The bits in this BitSet. The ith bit is stored in bits[i/64] at + * bit position i % 64 (where bit position 0 refers to the least + * significant bit and 63 refers to the most significant bit). + */ + private static final ObjectStreamField[] serialPersistentFields = { + new ObjectStreamField("bits", long[].class), + }; + + /** + * The internal field corresponding to the serialField "bits". + */ + private long[] words; + + /** + * The number of words in the logical size of this BitSet. + */ + private transient int wordsInUse = 0; + + /** + * Whether the size of "words" is user-specified. If so, we assume + * the user knows what he's doing and try harder to preserve it. + */ + private transient boolean sizeIsSticky = false; + + /* use serialVersionUID from JDK 1.0.2 for interoperability */ + private static final long serialVersionUID = 7997698588986878753L; + + /** + * Given a bit index, return word index containing it. + */ + private static int wordIndex(int bitIndex) { + return bitIndex >> ADDRESS_BITS_PER_WORD; + } + + /** + * Every public method must preserve these invariants. + */ + private void checkInvariants() { + assert(wordsInUse == 0 || words[wordsInUse - 1] != 0); + assert(wordsInUse >= 0 && wordsInUse <= words.length); + assert(wordsInUse == words.length || words[wordsInUse] == 0); + } + + /** + * Sets the field wordsInUse to the logical size in words of the bit set. + * WARNING:This method assumes that the number of words actually in use is + * less than or equal to the current value of wordsInUse! + */ + private void recalculateWordsInUse() { + // Traverse the bitset until a used word is found + int i; + for (i = wordsInUse-1; i >= 0; i--) + if (words[i] != 0) + break; + + wordsInUse = i+1; // The new logical size + } + + /** + * Creates a new bit set. All bits are initially {@code false}. + */ + public BitSetRecyclable() { + initWords(BITS_PER_WORD); + sizeIsSticky = false; + } + + /** + * Creates a bit set whose initial size is large enough to explicitly + * represent bits with indices in the range {@code 0} through + * {@code nbits-1}. All bits are initially {@code false}. + * + * @param nbits the initial size of the bit set + * @throws NegativeArraySizeException if the specified initial size + * is negative + */ + public BitSetRecyclable(int nbits) { + // nbits can't be negative; size 0 is OK + if (nbits < 0) + throw new NegativeArraySizeException("nbits < 0: " + nbits); + + initWords(nbits); + sizeIsSticky = true; + } + + private void initWords(int nbits) { + words = new long[wordIndex(nbits-1) + 1]; + } + + /** + * Creates a bit set using words as the internal representation. + * The last word (if there is one) must be non-zero. + */ + private BitSetRecyclable(long[] words) { + this.words = words; + this.wordsInUse = words.length; + checkInvariants(); + } + + /** + * Returns a new bit set containing all the bits in the given long array. + * + *

More precisely, + *
{@code BitSet.valueOf(longs).get(n) == ((longs[n/64] & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * longs.length}. + * + *

This method is equivalent to + * {@code BitSet.valueOf(LongBuffer.wrap(longs))}. + * + * @param longs a long array containing a little-endian representation + * of a sequence of bits to be used as the initial bits of the + * new bit set + * @return a {@code BitSet} containing all the bits in the long array + * @since 1.7 + */ + public static BitSetRecyclable valueOf(long[] longs) { + int n; + for (n = longs.length; n > 0 && longs[n - 1] == 0; n--) + ; + return new BitSetRecyclable(Arrays.copyOf(longs, n)); + } + + /** + * Returns a new bit set containing all the bits in the given long + * buffer between its position and limit. + * + *

More precisely, + *
{@code BitSet.valueOf(lb).get(n) == ((lb.get(lb.position()+n/64) & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * lb.remaining()}. + * + *

The long buffer is not modified by this method, and no + * reference to the buffer is retained by the bit set. + * + * @param lb a long buffer containing a little-endian representation + * of a sequence of bits between its position and limit, to be + * used as the initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the buffer in the + * specified range + * @since 1.7 + */ + public static BitSetRecyclable valueOf(LongBuffer lb) { + lb = lb.slice(); + int n; + for (n = lb.remaining(); n > 0 && lb.get(n - 1) == 0; n--) + ; + long[] words = new long[n]; + lb.get(words); + return new BitSetRecyclable(words); + } + + /** + * Returns a new bit set containing all the bits in the given byte array. + * + *

More precisely, + *
{@code BitSet.valueOf(bytes).get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bytes.length}. + * + *

This method is equivalent to + * {@code BitSet.valueOf(ByteBuffer.wrap(bytes))}. + * + * @param bytes a byte array containing a little-endian + * representation of a sequence of bits to be used as the + * initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the byte array + * @since 1.7 + */ + public static BitSetRecyclable valueOf(byte[] bytes) { + return BitSetRecyclable.valueOf(ByteBuffer.wrap(bytes)); + } + + /** + * Returns a new bit set containing all the bits in the given byte + * buffer between its position and limit. + * + *

More precisely, + *
{@code BitSet.valueOf(bb).get(n) == ((bb.get(bb.position()+n/8) & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bb.remaining()}. + * + *

The byte buffer is not modified by this method, and no + * reference to the buffer is retained by the bit set. + * + * @param bb a byte buffer containing a little-endian representation + * of a sequence of bits between its position and limit, to be + * used as the initial bits of the new bit set + * @return a {@code BitSet} containing all the bits in the buffer in the + * specified range + * @since 1.7 + */ + public static BitSetRecyclable valueOf(ByteBuffer bb) { + bb = bb.slice().order(ByteOrder.LITTLE_ENDIAN); + int n; + for (n = bb.remaining(); n > 0 && bb.get(n - 1) == 0; n--) + ; + long[] words = new long[(n + 7) / 8]; + bb.limit(n); + int i = 0; + while (bb.remaining() >= 8) + words[i++] = bb.getLong(); + for (int remaining = bb.remaining(), j = 0; j < remaining; j++) + words[i] |= (bb.get() & 0xffL) << (8 * j); + return new BitSetRecyclable(words); + } + + /** + * Returns a new byte array containing all the bits in this bit set. + * + *

More precisely, if + *
{@code byte[] bytes = s.toByteArray();} + *
then {@code bytes.length == (s.length()+7)/8} and + *
{@code s.get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)} + *
for all {@code n < 8 * bytes.length}. + * + * @return a byte array containing a little-endian representation + * of all the bits in this bit set + * @since 1.7 + */ + public byte[] toByteArray() { + int n = wordsInUse; + if (n == 0) + return new byte[0]; + int len = 8 * (n-1); + for (long x = words[n - 1]; x != 0; x >>>= 8) + len++; + byte[] bytes = new byte[len]; + ByteBuffer bb = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + for (int i = 0; i < n - 1; i++) + bb.putLong(words[i]); + for (long x = words[n - 1]; x != 0; x >>>= 8) + bb.put((byte) (x & 0xff)); + return bytes; + } + + /** + * Returns a new long array containing all the bits in this bit set. + * + *

More precisely, if + *
{@code long[] longs = s.toLongArray();} + *
then {@code longs.length == (s.length()+63)/64} and + *
{@code s.get(n) == ((longs[n/64] & (1L<<(n%64))) != 0)} + *
for all {@code n < 64 * longs.length}. + * + * @return a long array containing a little-endian representation + * of all the bits in this bit set + * @since 1.7 + */ + public long[] toLongArray() { + return Arrays.copyOf(words, wordsInUse); + } + + /** + * Ensures that the BitSet can hold enough words. + * @param wordsRequired the minimum acceptable number of words. + */ + private void ensureCapacity(int wordsRequired) { + if (words.length < wordsRequired) { + // Allocate larger of doubled size or required size + int request = Math.max(2 * words.length, wordsRequired); + words = Arrays.copyOf(words, request); + sizeIsSticky = false; + } + } + + /** + * Ensures that the BitSet can accommodate a given wordIndex, + * temporarily violating the invariants. The caller must + * restore the invariants before returning to the user, + * possibly using recalculateWordsInUse(). + * @param wordIndex the index to be accommodated. + */ + private void expandTo(int wordIndex) { + int wordsRequired = wordIndex+1; + if (wordsInUse < wordsRequired) { + ensureCapacity(wordsRequired); + wordsInUse = wordsRequired; + } + } + + /** + * Checks that fromIndex ... toIndex is a valid range of bit indices. + */ + private static void checkRange(int fromIndex, int toIndex) { + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + if (toIndex < 0) + throw new IndexOutOfBoundsException("toIndex < 0: " + toIndex); + if (fromIndex > toIndex) + throw new IndexOutOfBoundsException("fromIndex: " + fromIndex + + " > toIndex: " + toIndex); + } + + /** + * Sets the bit at the specified index to the complement of its + * current value. + * + * @param bitIndex the index of the bit to flip + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public void flip(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + expandTo(wordIndex); + + words[wordIndex] ^= (1L << bitIndex); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets each bit from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to the complement of its current + * value. + * + * @param fromIndex index of the first bit to flip + * @param toIndex index after the last bit to flip + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void flip(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + int startWordIndex = wordIndex(fromIndex); + int endWordIndex = wordIndex(toIndex - 1); + expandTo(endWordIndex); + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] ^= (firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] ^= firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] ^= WORD_MASK; + + // Handle last word + words[endWordIndex] ^= lastWordMask; + } + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets the bit at the specified index to {@code true}. + * + * @param bitIndex a bit index + * @throws IndexOutOfBoundsException if the specified index is negative + * @since JDK1.0 + */ + public void set(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + expandTo(wordIndex); + + words[wordIndex] |= (1L << bitIndex); // Restores invariants + + checkInvariants(); + } + + /** + * Sets the bit at the specified index to the specified value. + * + * @param bitIndex a bit index + * @param value a boolean value to set + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public void set(int bitIndex, boolean value) { + if (value) + set(bitIndex); + else + clear(bitIndex); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to {@code true}. + * + * @param fromIndex index of the first bit to be set + * @param toIndex index after the last bit to be set + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void set(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + // Increase capacity if necessary + int startWordIndex = wordIndex(fromIndex); + int endWordIndex = wordIndex(toIndex - 1); + expandTo(endWordIndex); + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] |= (firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] |= firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] = WORD_MASK; + + // Handle last word (restores invariants) + words[endWordIndex] |= lastWordMask; + } + + checkInvariants(); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to the specified value. + * + * @param fromIndex index of the first bit to be set + * @param toIndex index after the last bit to be set + * @param value value to set the selected bits to + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void set(int fromIndex, int toIndex, boolean value) { + if (value) + set(fromIndex, toIndex); + else + clear(fromIndex, toIndex); + } + + /** + * Sets the bit specified by the index to {@code false}. + * + * @param bitIndex the index of the bit to be cleared + * @throws IndexOutOfBoundsException if the specified index is negative + * @since JDK1.0 + */ + public void clear(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + int wordIndex = wordIndex(bitIndex); + if (wordIndex >= wordsInUse) + return; + + words[wordIndex] &= ~(1L << bitIndex); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets the bits from the specified {@code fromIndex} (inclusive) to the + * specified {@code toIndex} (exclusive) to {@code false}. + * + * @param fromIndex index of the first bit to be cleared + * @param toIndex index after the last bit to be cleared + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public void clear(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + if (fromIndex == toIndex) + return; + + int startWordIndex = wordIndex(fromIndex); + if (startWordIndex >= wordsInUse) + return; + + int endWordIndex = wordIndex(toIndex - 1); + if (endWordIndex >= wordsInUse) { + toIndex = length(); + endWordIndex = wordsInUse - 1; + } + + long firstWordMask = WORD_MASK << fromIndex; + long lastWordMask = WORD_MASK >>> -toIndex; + if (startWordIndex == endWordIndex) { + // Case 1: One word + words[startWordIndex] &= ~(firstWordMask & lastWordMask); + } else { + // Case 2: Multiple words + // Handle first word + words[startWordIndex] &= ~firstWordMask; + + // Handle intermediate words, if any + for (int i = startWordIndex+1; i < endWordIndex; i++) + words[i] = 0; + + // Handle last word + words[endWordIndex] &= ~lastWordMask; + } + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Sets all of the bits in this BitSet to {@code false}. + * + * @since 1.4 + */ + public void clear() { + while (wordsInUse > 0) + words[--wordsInUse] = 0; + } + + /** + * Returns the value of the bit with the specified index. The value + * is {@code true} if the bit with the index {@code bitIndex} + * is currently set in this {@code BitSet}; otherwise, the result + * is {@code false}. + * + * @param bitIndex the bit index + * @return the value of the bit with the specified index + * @throws IndexOutOfBoundsException if the specified index is negative + */ + public boolean get(int bitIndex) { + if (bitIndex < 0) + throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex); + + checkInvariants(); + + int wordIndex = wordIndex(bitIndex); + return (wordIndex < wordsInUse) + && ((words[wordIndex] & (1L << bitIndex)) != 0); + } + + /** + * Returns a new {@code BitSet} composed of bits from this {@code BitSet} + * from {@code fromIndex} (inclusive) to {@code toIndex} (exclusive). + * + * @param fromIndex index of the first bit to include + * @param toIndex index after the last bit to include + * @return a new {@code BitSet} from a range of this {@code BitSet} + * @throws IndexOutOfBoundsException if {@code fromIndex} is negative, + * or {@code toIndex} is negative, or {@code fromIndex} is + * larger than {@code toIndex} + * @since 1.4 + */ + public BitSetRecyclable get(int fromIndex, int toIndex) { + checkRange(fromIndex, toIndex); + + checkInvariants(); + + int len = length(); + + // If no set bits in range return empty bitset + if (len <= fromIndex || fromIndex == toIndex) + return new BitSetRecyclable(0); + + // An optimization + if (toIndex > len) + toIndex = len; + + BitSetRecyclable result = new BitSetRecyclable(toIndex - fromIndex); + int targetWords = wordIndex(toIndex - fromIndex - 1) + 1; + int sourceIndex = wordIndex(fromIndex); + boolean wordAligned = ((fromIndex & BIT_INDEX_MASK) == 0); + + // Process all words but the last word + for (int i = 0; i < targetWords - 1; i++, sourceIndex++) + result.words[i] = wordAligned ? words[sourceIndex] : + (words[sourceIndex] >>> fromIndex) | + (words[sourceIndex+1] << -fromIndex); + + // Process the last word + long lastWordMask = WORD_MASK >>> -toIndex; + result.words[targetWords - 1] = + ((toIndex-1) & BIT_INDEX_MASK) < (fromIndex & BIT_INDEX_MASK) + ? /* straddles source words */ + ((words[sourceIndex] >>> fromIndex) | + (words[sourceIndex+1] & lastWordMask) << -fromIndex) + : + ((words[sourceIndex] & lastWordMask) >>> fromIndex); + + // Set wordsInUse correctly + result.wordsInUse = targetWords; + result.recalculateWordsInUse(); + result.checkInvariants(); + + return result; + } + + /** + * Returns the index of the first bit that is set to {@code true} + * that occurs on or after the specified starting index. If no such + * bit exists then {@code -1} is returned. + * + *

To iterate over the {@code true} bits in a {@code BitSet}, + * use the following loop: + * + *

 {@code
+     * for (int i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i+1)) {
+     *     // operate on index i here
+     *     if (i == Integer.MAX_VALUE) {
+     *         break; // or (i+1) would overflow
+     *     }
+     * }}
+ * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next set bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public int nextSetBit(int fromIndex) { + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return -1; + + long word = words[u] & (WORD_MASK << fromIndex); + + while (true) { + if (word != 0) + return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word); + if (++u == wordsInUse) + return -1; + word = words[u]; + } + } + + /** + * Returns the index of the first bit that is set to {@code false} + * that occurs on or after the specified starting index. + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next clear bit + * @throws IndexOutOfBoundsException if the specified index is negative + * @since 1.4 + */ + public int nextClearBit(int fromIndex) { + // Neither spec nor implementation handle bitsets of maximal length. + // See 4816253. + if (fromIndex < 0) + throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex); + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return fromIndex; + + long word = ~words[u] & (WORD_MASK << fromIndex); + + while (true) { + if (word != 0) + return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word); + if (++u == wordsInUse) + return wordsInUse * BITS_PER_WORD; + word = ~words[u]; + } + } + + /** + * Returns the index of the nearest bit that is set to {@code true} + * that occurs on or before the specified starting index. + * If no such bit exists, or if {@code -1} is given as the + * starting index, then {@code -1} is returned. + * + *

To iterate over the {@code true} bits in a {@code BitSet}, + * use the following loop: + * + *

 {@code
+     * for (int i = bs.length(); (i = bs.previousSetBit(i-1)) >= 0; ) {
+     *     // operate on index i here
+     * }}
+ * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the previous set bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is less + * than {@code -1} + * @since 1.7 + */ + public int previousSetBit(int fromIndex) { + if (fromIndex < 0) { + if (fromIndex == -1) + return -1; + throw new IndexOutOfBoundsException( + "fromIndex < -1: " + fromIndex); + } + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return length() - 1; + + long word = words[u] & (WORD_MASK >>> -(fromIndex+1)); + + while (true) { + if (word != 0) + return (u+1) * BITS_PER_WORD - 1 - Long.numberOfLeadingZeros(word); + if (u-- == 0) + return -1; + word = words[u]; + } + } + + /** + * Returns the index of the nearest bit that is set to {@code false} + * that occurs on or before the specified starting index. + * If no such bit exists, or if {@code -1} is given as the + * starting index, then {@code -1} is returned. + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the previous clear bit, or {@code -1} if there + * is no such bit + * @throws IndexOutOfBoundsException if the specified index is less + * than {@code -1} + * @since 1.7 + */ + public int previousClearBit(int fromIndex) { + if (fromIndex < 0) { + if (fromIndex == -1) + return -1; + throw new IndexOutOfBoundsException( + "fromIndex < -1: " + fromIndex); + } + + checkInvariants(); + + int u = wordIndex(fromIndex); + if (u >= wordsInUse) + return fromIndex; + + long word = ~words[u] & (WORD_MASK >>> -(fromIndex+1)); + + while (true) { + if (word != 0) + return (u+1) * BITS_PER_WORD -1 - Long.numberOfLeadingZeros(word); + if (u-- == 0) + return -1; + word = ~words[u]; + } + } + + /** + * Returns the "logical size" of this {@code BitSet}: the index of + * the highest set bit in the {@code BitSet} plus one. Returns zero + * if the {@code BitSet} contains no set bits. + * + * @return the logical size of this {@code BitSet} + * @since 1.2 + */ + public int length() { + if (wordsInUse == 0) + return 0; + + return BITS_PER_WORD * (wordsInUse - 1) + + (BITS_PER_WORD - Long.numberOfLeadingZeros(words[wordsInUse - 1])); + } + + /** + * Returns true if this {@code BitSet} contains no bits that are set + * to {@code true}. + * + * @return boolean indicating whether this {@code BitSet} is empty + * @since 1.4 + */ + public boolean isEmpty() { + return wordsInUse == 0; + } + + /** + * Returns true if the specified {@code BitSet} has any bits set to + * {@code true} that are also set to {@code true} in this {@code BitSet}. + * + * @param set {@code BitSet} to intersect with + * @return boolean indicating whether this {@code BitSet} intersects + * the specified {@code BitSet} + * @since 1.4 + */ + public boolean intersects(BitSetRecyclable set) { + for (int i = Math.min(wordsInUse, set.wordsInUse) - 1; i >= 0; i--) + if ((words[i] & set.words[i]) != 0) + return true; + return false; + } + + /** + * Returns the number of bits set to {@code true} in this {@code BitSet}. + * + * @return the number of bits set to {@code true} in this {@code BitSet} + * @since 1.4 + */ + public int cardinality() { + int sum = 0; + for (int i = 0; i < wordsInUse; i++) + sum += Long.bitCount(words[i]); + return sum; + } + + /** + * Performs a logical AND of this target bit set with the + * argument bit set. This bit set is modified so that each bit in it + * has the value {@code true} if and only if it both initially + * had the value {@code true} and the corresponding bit in the + * bit set argument also had the value {@code true}. + * + * @param set a bit set + */ + public void and(BitSetRecyclable set) { + if (this == set) + return; + + while (wordsInUse > set.wordsInUse) + words[--wordsInUse] = 0; + + // Perform logical AND on words in common + for (int i = 0; i < wordsInUse; i++) + words[i] &= set.words[i]; + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Performs a logical OR of this bit set with the bit set + * argument. This bit set is modified so that a bit in it has the + * value {@code true} if and only if it either already had the + * value {@code true} or the corresponding bit in the bit set + * argument has the value {@code true}. + * + * @param set a bit set + */ + public void or(BitSetRecyclable set) { + if (this == set) + return; + + int wordsInCommon = Math.min(wordsInUse, set.wordsInUse); + + if (wordsInUse < set.wordsInUse) { + ensureCapacity(set.wordsInUse); + wordsInUse = set.wordsInUse; + } + + // Perform logical OR on words in common + for (int i = 0; i < wordsInCommon; i++) + words[i] |= set.words[i]; + + // Copy any remaining words + if (wordsInCommon < set.wordsInUse) + System.arraycopy(set.words, wordsInCommon, + words, wordsInCommon, + wordsInUse - wordsInCommon); + + // recalculateWordsInUse() is unnecessary + checkInvariants(); + } + + /** + * Performs a logical XOR of this bit set with the bit set + * argument. This bit set is modified so that a bit in it has the + * value {@code true} if and only if one of the following + * statements holds: + *
    + *
  • The bit initially has the value {@code true}, and the + * corresponding bit in the argument has the value {@code false}. + *
  • The bit initially has the value {@code false}, and the + * corresponding bit in the argument has the value {@code true}. + *
+ * + * @param set a bit set + */ + public void xor(BitSetRecyclable set) { + int wordsInCommon = Math.min(wordsInUse, set.wordsInUse); + + if (wordsInUse < set.wordsInUse) { + ensureCapacity(set.wordsInUse); + wordsInUse = set.wordsInUse; + } + + // Perform logical XOR on words in common + for (int i = 0; i < wordsInCommon; i++) + words[i] ^= set.words[i]; + + // Copy any remaining words + if (wordsInCommon < set.wordsInUse) + System.arraycopy(set.words, wordsInCommon, + words, wordsInCommon, + set.wordsInUse - wordsInCommon); + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Clears all of the bits in this {@code BitSet} whose corresponding + * bit is set in the specified {@code BitSet}. + * + * @param set the {@code BitSet} with which to mask this + * {@code BitSet} + * @since 1.2 + */ + public void andNot(BitSetRecyclable set) { + // Perform logical (a & !b) on words in common + for (int i = Math.min(wordsInUse, set.wordsInUse) - 1; i >= 0; i--) + words[i] &= ~set.words[i]; + + recalculateWordsInUse(); + checkInvariants(); + } + + /** + * Returns the hash code value for this bit set. The hash code depends + * only on which bits are set within this {@code BitSet}. + * + *

The hash code is defined to be the result of the following + * calculation: + *

 {@code
+     * public int hashCode() {
+     *     long h = 1234;
+     *     long[] words = toLongArray();
+     *     for (int i = words.length; --i >= 0; )
+     *         h ^= words[i] * (i + 1);
+     *     return (int)((h >> 32) ^ h);
+     * }}
+ * Note that the hash code changes if the set of bits is altered. + * + * @return the hash code value for this bit set + */ + public int hashCode() { + long h = 1234; + for (int i = wordsInUse; --i >= 0; ) + h ^= words[i] * (i + 1); + + return (int)((h >> 32) ^ h); + } + + /** + * Returns the number of bits of space actually in use by this + * {@code BitSet} to represent bit values. + * The maximum element in the set is the size - 1st element. + * + * @return the number of bits currently in this bit set + */ + public int size() { + return words.length * BITS_PER_WORD; + } + + /** + * Compares this object against the specified object. + * The result is {@code true} if and only if the argument is + * not {@code null} and is a {@code Bitset} object that has + * exactly the same set of bits set to {@code true} as this bit + * set. That is, for every nonnegative {@code int} index {@code k}, + *
((BitSet)obj).get(k) == this.get(k)
+ * must be true. The current sizes of the two bit sets are not compared. + * + * @param obj the object to compare with + * @return {@code true} if the objects are the same; + * {@code false} otherwise + * @see #size() + */ + public boolean equals(Object obj) { + if (!(obj instanceof BitSetRecyclable)) + return false; + if (this == obj) + return true; + + BitSetRecyclable set = (BitSetRecyclable) obj; + + checkInvariants(); + set.checkInvariants(); + + if (wordsInUse != set.wordsInUse) + return false; + + // Check words in use by both BitSets + for (int i = 0; i < wordsInUse; i++) + if (words[i] != set.words[i]) + return false; + + return true; + } + + /** + * Cloning this {@code BitSet} produces a new {@code BitSet} + * that is equal to it. + * The clone of the bit set is another bit set that has exactly the + * same bits set to {@code true} as this bit set. + * + * @return a clone of this bit set + * @see #size() + */ + public Object clone() { + if (! sizeIsSticky) + trimToSize(); + + try { + BitSetRecyclable result = (BitSetRecyclable) super.clone(); + result.words = words.clone(); + result.checkInvariants(); + return result; + } catch (CloneNotSupportedException e) { + throw new InternalError(e); + } + } + + /** + * Attempts to reduce internal storage used for the bits in this bit set. + * Calling this method may, but is not required to, affect the value + * returned by a subsequent call to the {@link #size()} method. + */ + private void trimToSize() { + if (wordsInUse != words.length) { + words = Arrays.copyOf(words, wordsInUse); + checkInvariants(); + } + } + + /** + * Save the state of the {@code BitSet} instance to a stream (i.e., + * serialize it). + */ + private void writeObject(ObjectOutputStream s) + throws IOException { + + checkInvariants(); + + if (! sizeIsSticky) + trimToSize(); + + ObjectOutputStream.PutField fields = s.putFields(); + fields.put("bits", words); + s.writeFields(); + } + + /** + * Reconstitute the {@code BitSet} instance from a stream (i.e., + * deserialize it). + */ + private void readObject(ObjectInputStream s) + throws IOException, ClassNotFoundException { + + ObjectInputStream.GetField fields = s.readFields(); + words = (long[]) fields.get("bits", null); + + // Assume maximum length then find real length + // because recalculateWordsInUse assumes maintenance + // or reduction in logical size + wordsInUse = words.length; + recalculateWordsInUse(); + sizeIsSticky = (words.length > 0 && words[words.length-1] == 0L); // heuristic + checkInvariants(); + } + + /** + * Returns a string representation of this bit set. For every index + * for which this {@code BitSet} contains a bit in the set + * state, the decimal representation of that index is included in + * the result. Such indices are listed in order from lowest to + * highest, separated by ", " (a comma and a space) and + * surrounded by braces, resulting in the usual mathematical + * notation for a set of integers. + * + *

Example: + *

+     * BitSet drPepper = new BitSet();
+ * Now {@code drPepper.toString()} returns "{@code {}}". + *
+     * drPepper.set(2);
+ * Now {@code drPepper.toString()} returns "{@code {2}}". + *
+     * drPepper.set(4);
+     * drPepper.set(10);
+ * Now {@code drPepper.toString()} returns "{@code {2, 4, 10}}". + * + * @return a string representation of this bit set + */ + public String toString() { + checkInvariants(); + + int numBits = (wordsInUse > 128) ? + cardinality() : wordsInUse * BITS_PER_WORD; + StringBuilder b = new StringBuilder(6*numBits + 2); + b.append('{'); + + int i = nextSetBit(0); + if (i != -1) { + b.append(i); + while (true) { + if (++i < 0) break; + if ((i = nextSetBit(i)) < 0) break; + int endOfRun = nextClearBit(i); + do { b.append(", ").append(i); } + while (++i != endOfRun); + } + } + + b.append('}'); + return b.toString(); + } + + public BitSetRecyclable resetWords(long[] words) { + int n; + for (n = words.length; n > 0 && words[n - 1] == 0; n--) + ; + long[] longs = Arrays.copyOf(words, n); + this.words = longs; + this.wordsInUse = longs.length; + checkInvariants(); + return this; + } + + private Handle recyclerHandle = null; private static final Recycler RECYCLER = new Recycler() { protected BitSetRecyclable newObject(Recycler.Handle recyclerHandle) { @@ -37,7 +1192,7 @@ protected BitSetRecyclable newObject(Recycler.Handle recyclerH }; private BitSetRecyclable(Handle recyclerHandle) { - super(); + this(); this.recyclerHandle = recyclerHandle; } @@ -46,7 +1201,9 @@ public static BitSetRecyclable create() { } public void recycle() { - this.clear(); - recyclerHandle.recycle(this); + if (recyclerHandle != null) { + this.clear(); + recyclerHandle.recycle(this); + } } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java similarity index 76% rename from pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java rename to pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java index 07155ad8a5f45..0f42f35608a76 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/BitSetRecyclableRecyclableTest.java @@ -21,7 +21,7 @@ import org.testng.Assert; import org.testng.annotations.Test; -public class BitSetRecyclableTest { +public class BitSetRecyclableRecyclableTest { @Test public void testRecycle() { @@ -34,4 +34,15 @@ public void testRecycle() { Assert.assertFalse(bitset2.get(3)); Assert.assertNotSame(bitset3, bitset1); } + + @Test + public void testResetWords() { + BitSetRecyclable bitset1 = BitSetRecyclable.create(); + BitSetRecyclable bitset2 = BitSetRecyclable.create(); + bitset1.set(256); + bitset2.set(128); + bitset1.resetWords(bitset2.toLongArray()); + Assert.assertTrue(bitset1.get(128)); + Assert.assertFalse(bitset1.get(256)); + } } From 26816c02195fc844f0caaa84d1c3038004a01c37 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Wed, 19 Feb 2020 22:40:47 +0800 Subject: [PATCH 13/21] Check not null for recycle. --- .../mledger/impl/ManagedCursorImpl.java | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index a546a88530af8..093ecfc961dee 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1775,7 +1775,10 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) || position.compareTo(markDeletePosition) <= 0) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position).recycle(); + BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); + if (bitSetRecyclable != null) { + bitSetRecyclable.recycle(); + } } if (log.isDebugEnabled()) { log.debug("[{}] [{}] Position was already deleted {}", ledger.getName(), name, position); @@ -1784,7 +1787,10 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (position.ackSet == null) { if (config.isDeletionAtBatchIndexLevelEnabled()) { - batchDeletedIndexes.remove(position).recycle(); + BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); + if (bitSetRecyclable != null) { + bitSetRecyclable.recycle(); + } } // Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will make // the RangeSet recognize the "continuity" between adjacent Positions @@ -1807,7 +1813,10 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); ++messagesConsumedCounter; - batchDeletedIndexes.remove(position).recycle(); + BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); + if (bitSetRecyclable != null) { + bitSetRecyclable.recycle(); + } } } } From ae832ffe0951dfbd559e8401820eeb3ed5816a55 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Wed, 19 Feb 2020 23:58:32 +0800 Subject: [PATCH 14/21] Fix tests. --- .../java/org/apache/pulsar/common/protocol/Commands.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java index cede8e8c2e591..2318a52997117 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/Commands.java @@ -884,8 +884,9 @@ public static ByteBuf newMultiMessageAck(long consumerId, } MessageIdData messageIdData = messageIdDataBuilder.build(); ackBuilder.addMessageId(messageIdData); - - bitSet.recycle(); + if (bitSet != null) { + bitSet.recycle(); + } messageIdDataBuilder.recycle(); } @@ -937,7 +938,9 @@ public static ByteBuf newAck(long consumerId, long ledgerId, long entryId, BitSe ByteBuf res = serializeWithSize(BaseCommand.newBuilder().setType(Type.ACK).setAck(ack)); ack.recycle(); - ackSet.recycle(); + if (ackSet != null) { + ackSet.recycle(); + } ackBuilder.recycle(); messageIdDataBuilder.recycle(); messageIdData.recycle(); From a297843b47e143a1a26712847932d897c678c336 Mon Sep 17 00:00:00 2001 From: Penghui Li Date: Thu, 20 Feb 2020 15:41:50 +0800 Subject: [PATCH 15/21] fix tests. --- .../mledger/impl/ManagedCursorImpl.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 093ecfc961dee..4970c8318d9e6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -387,7 +387,8 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } - if (config.isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null + && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } recoveredCursor(position, recoveredProperties, lh); @@ -937,7 +938,7 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle); batchDeletedIndexes.clear(); } @@ -1531,7 +1532,7 @@ public void asyncMarkDelete(final Position position, Map propertie PositionImpl newPosition = (PositionImpl) position; - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { if (newPosition.ackSet != null) { batchDeletedIndexes.put(newPosition, BitSetRecyclable.create().resetWords(newPosition.ackSet)); newPosition = ledger.getPreviousPosition(newPosition); @@ -1635,7 +1636,7 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { Map subMap = batchDeletedIndexes.subMap(PositionImpl.earliest, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); @@ -1774,7 +1775,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()) || position.compareTo(markDeletePosition) <= 0) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -1786,7 +1787,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb continue; } if (position.ackSet == null) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -1803,7 +1804,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); } - } else if (config.isDeletionAtBatchIndexLevelEnabled()) { + } else if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> BitSetRecyclable.create().resetWords(position.ackSet)); BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); bitSet.and(givenBitSet); @@ -2377,7 +2378,7 @@ private List buildIndividualDeletedMessageRanges() { } private List buildBatchEntryDeletionIndexInfoList() { - if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo @@ -2792,8 +2793,12 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - BitSetRecyclable bitSet = batchDeletedIndexes.get(position); - return bitSet == null ? null : bitSet.toLongArray(); + if (config.isDeletionAtBatchIndexLevelEnabled() && batchDeletedIndexes != null) { + BitSetRecyclable bitSet = batchDeletedIndexes.get(position); + return bitSet == null ? null : bitSet.toLongArray(); + } else { + return null; + } } private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); From 75aec3966e5f02b3f421fcba92bb771871965eeb Mon Sep 17 00:00:00 2001 From: penghui Date: Fri, 24 Apr 2020 16:00:31 +0800 Subject: [PATCH 16/21] Fix test. --- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 +-- .../org/apache/pulsar/client/impl/BatchMessageIdImplTest.java | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 5a1814ca86c9b..9e11080cda3c0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -718,8 +718,7 @@ private CompletableFuture getNonDurableSubscription(Stri } catch (ManagedLedgerException e) { subscriptionFuture.completeExceptionally(e); } - return new PersistentSubscription(this, subscriptionName, cursor, false, - brokerService.getPulsar().getConfiguration().isBatchIndexAcknowledgeEnable()); + return new PersistentSubscription(this, subscriptionName, cursor, false); }); if (!subscriptionFuture.isDone()) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java index c32a59376f9df..29207378e6666 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageIdImplTest.java @@ -76,7 +76,7 @@ public void equalsTest() { public void deserializationTest() { // initialize BitSet with null BatchMessageAcker ackerDisabled = new BatchMessageAcker(null, 0); - BatchMessageIdImpl batchMsgId = new BatchMessageIdImpl(0, 0, 0, 0, ackerDisabled); + BatchMessageIdImpl batchMsgId = new BatchMessageIdImpl(0, 0, 0, 0, 0, ackerDisabled); ObjectWriter writer = ObjectMapperFactory.create().writerWithDefaultPrettyPrinter(); From cca1a5a0c83104d26730408a827331008f9eaee1 Mon Sep 17 00:00:00 2001 From: penghui Date: Fri, 24 Apr 2020 16:03:06 +0800 Subject: [PATCH 17/21] Remove unused tests. --- .../pulsar/client/impl/KeySharedConsumer.java | 35 ------------------- .../client/impl/KeySharedConsumer1.java | 35 ------------------- .../client/impl/KeySharedConsumer2.java | 35 ------------------- .../pulsar/client/impl/KeySharedProducer.java | 33 ----------------- 4 files changed, 138 deletions(-) delete mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java delete mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java delete mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java delete mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java deleted file mode 100644 index 5129f71db8b77..0000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer.java +++ /dev/null @@ -1,35 +0,0 @@ -package org.apache.pulsar.client.impl; - -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.util.concurrent.TimeUnit; - -public class KeySharedConsumer { - - public static void main(String[] args) throws PulsarClientException { - PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); - - for (int i = 0; i < 3000; i++) { - new Thread(() -> { - try { - Consumer consumer = client.newConsumer(Schema.STRING) - .topic("key_shared_latency-1") - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1000) - .subscriptionName("test") - .subscribe(); - while (true) { - consumer.acknowledge(consumer.receive()); - } - } catch (PulsarClientException e) { - e.printStackTrace(); - } - }).start(); - } - - } -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java deleted file mode 100644 index f965a971fa7d6..0000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer1.java +++ /dev/null @@ -1,35 +0,0 @@ -package org.apache.pulsar.client.impl; - -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.util.concurrent.TimeUnit; - -public class KeySharedConsumer1 { - - public static void main(String[] args) throws PulsarClientException { - PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); - - for (int i = 0; i < 2000; i++) { - new Thread(() -> { - try { - Consumer consumer = client.newConsumer(Schema.STRING) - .topic("key_shared_latency-1") - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1000) - .subscriptionName("test") - .subscribe(); - while (true) { - consumer.acknowledge(consumer.receive()); - } - } catch (PulsarClientException e) { - e.printStackTrace(); - } - }).start(); - } - - } -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java deleted file mode 100644 index edb6c63c812d8..0000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedConsumer2.java +++ /dev/null @@ -1,35 +0,0 @@ -package org.apache.pulsar.client.impl; - -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.util.concurrent.TimeUnit; - -public class KeySharedConsumer2 { - - public static void main(String[] args) throws PulsarClientException { - PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); - - for (int i = 0; i < 2000; i++) { - new Thread(() -> { - try { - Consumer consumer = client.newConsumer(Schema.STRING) - .topic("key_shared_latency-1") - .subscriptionType(SubscriptionType.Key_Shared) - .receiverQueueSize(1000) - .subscriptionName("test") - .subscribe(); - while (true) { - consumer.acknowledge(consumer.receive()); - } - } catch (PulsarClientException e) { - e.printStackTrace(); - } - }).start(); - } - - } -} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java deleted file mode 100644 index 7f737f9391f46..0000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/KeySharedProducer.java +++ /dev/null @@ -1,33 +0,0 @@ -package org.apache.pulsar.client.impl; - -import org.apache.pulsar.client.api.BatcherBuilder; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.util.UUID; -import java.util.concurrent.TimeUnit; - -public class KeySharedProducer { - - public static void main(String[] args) throws PulsarClientException, InterruptedException { - PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://127.0.0.1:6650").statsInterval(5, TimeUnit.SECONDS).build(); - Producer producer = client.newProducer(Schema.STRING) - .topic("key_shared_latency-1") - .enableBatching(false) - .batcherBuilder(BatcherBuilder.KEY_BASED) - .maxPendingMessages(5000) - .create(); - - int i = 0; - while (true) { - producer.newMessage().key(UUID.randomUUID().toString()).value("test").sendAsync(); - if (++i % 20 == 0) { -// Thread.sleep(1); - } - } - } -} From 36d899ac47a4c1b52a1cbfa080b7ba5c4df4f4db Mon Sep 17 00:00:00 2001 From: penghui Date: Fri, 24 Apr 2020 16:05:54 +0800 Subject: [PATCH 18/21] Remove unused tests. --- .../apache/pulsar/client/util/HashTest.java | 22 ------------------- 1 file changed, 22 deletions(-) delete mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java deleted file mode 100644 index c64ca31a63e79..0000000000000 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/util/HashTest.java +++ /dev/null @@ -1,22 +0,0 @@ -package org.apache.pulsar.client.util; - -import org.apache.pulsar.client.impl.Murmur3_32Hash; - -public class HashTest { - - public static void main(String[] args) { - - for (int i = 0; i < 3; i++) { - System.out.println(Murmur3_32Hash.getInstance().makeHash("consumer-name-" + i)); - } - - - System.out.println("-------------------------------------"); - - for (int i = 0; i < 3; i++) { - System.out.println(Murmur3_32Hash.getInstance().makeHash("ff" + i)); - } - - - } -} From ef9929a35ccad0e91d2fd86ff84ddc48de10cc58 Mon Sep 17 00:00:00 2001 From: penghui Date: Tue, 19 May 2020 18:18:43 +0800 Subject: [PATCH 19/21] Fix tests. --- .../PersistentStickyKeyDispatcherMultipleConsumersTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java index 7af5b7910004c..7559091320b2c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentStickyKeyDispatcherMultipleConsumersTest.java @@ -100,6 +100,7 @@ public void setup() throws Exception { doReturn(channelMock).when(consumerMock).sendMessages( anyList(), any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), anyInt(), anyLong(), any(RedeliveryTracker.class) @@ -142,6 +143,7 @@ public void testSendMarkerMessage() { verify(consumerMock, times(2)).sendMessages( anyList(), any(EntryBatchSizes.class), + any(EntryBatchIndexesAcks.class), totalMessagesCaptor.capture(), anyLong(), any(RedeliveryTracker.class) From f248dc8fd279e220bf3aafab7622080625cf69f5 Mon Sep 17 00:00:00 2001 From: penghui Date: Wed, 27 May 2020 17:31:31 +0800 Subject: [PATCH 20/21] Fix tests --- .../apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 4 ++-- .../impl/PersistentAcknowledgmentsGroupingTracker.java | 6 +++++- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 4970c8318d9e6..50a64b87bead5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -1541,8 +1541,8 @@ public void asyncMarkDelete(final Position position, Map propertie subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); } else if (newPosition.ackSet != null) { - callback.markDeleteFailed(new ManagedLedgerException("Batch ack set not support"), ctx); - return; + newPosition = ledger.getPreviousPosition(newPosition); + newPosition.ackSet = null; } if (((PositionImpl) ledger.getLastConfirmedEntry()).compareTo(newPosition) < 0) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java index d163d1a8dedaa..927594c3ff69c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PersistentAcknowledgmentsGroupingTracker.java @@ -161,7 +161,11 @@ private void doCumulativeAck(MessageIdImpl msgId, BitSetRecyclable bitSet) { if (msgId.compareTo(lastCumlativeAck) > 0) { if (LAST_CUMULATIVE_ACK_UPDATER.compareAndSet(this, lastCumlativeAck, msgId) && LAST_CUMULATIVE_ACK_SET_UPDATER.compareAndSet(this, lastBitSet, bitSet)) { if (lastBitSet != null) { - lastBitSet.recycle(); + try { + lastBitSet.recycle(); + } catch (Exception ignore) { + // no-op + } } // Successfully updated the last cumulative ack. Next flush iteration will send this to broker. cumulativeAckFlushRequired = true; From 15f39790e19f8a7904ce5f1e19a432181dc75d4b Mon Sep 17 00:00:00 2001 From: penghui Date: Thu, 28 May 2020 18:30:08 +0800 Subject: [PATCH 21/21] Fix tests. --- .../pulsar/broker/service/AbstractBaseDispatcher.java | 4 ++-- .../pulsar/broker/service/EntryBatchIndexesAcks.java | 9 ++++----- 2 files changed, 6 insertions(+), 7 deletions(-) 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 f5b5da2ecbbb9..849d626a441ef 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 @@ -107,9 +107,9 @@ && trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { if (indexesAcks != null && cursor != null) { long[] ackSet = cursor.getDeletedBatchIndexesAsLongArray(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); if (ackSet != null) { - indexesAcks.setIndexesAcks(Pair.of(batchSize, ackSet)); + indexesAcks.setIndexesAcks(i, Pair.of(batchSize, ackSet)); } else { - indexesAcks.setIndexesAcks(null); + indexesAcks.setIndexesAcks(i,null); } } } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java index 89c0c771c66ec..e41a29092140f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/EntryBatchIndexesAcks.java @@ -28,14 +28,14 @@ public class EntryBatchIndexesAcks { - List> indexesAcks = new ArrayList<>(); + Pair[] indexesAcks = new Pair[100]; - public void setIndexesAcks(Pair indexesAcks) { - this.indexesAcks.add(indexesAcks); + public void setIndexesAcks(int entryIdx, Pair indexesAcks) { + this.indexesAcks[entryIdx] = indexesAcks; } public long[] getAckSet(int entryIdx) { - Pair pair = indexesAcks.get(entryIdx); + Pair pair = indexesAcks[entryIdx]; return pair == null ? null : pair.getRight(); } @@ -50,7 +50,6 @@ public int getTotalAckedIndexCount() { } public void recycle() { - this.indexesAcks.clear(); handle.recycle(this); }