From add770ad603d5f93ea824b4f10f68085325ae623 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 09:05:59 -0700 Subject: [PATCH 1/9] [improve][ml] Migrate managed-ledger from protobuf to LightProto Migrate the managed-ledger module from Google protobuf to LightProto for ManagedLedgerInfo and ManagedCursorInfo proto messages. Key changes: - Replace protobuf Builder pattern with direct LightProto mutable objects - Replace ByteString with byte[] for bytes fields - Replace static parseFrom() with instance parseFrom(ByteBuf, size) - Reuse LightProto objects in ManagedCursorImpl and MetaStoreImpl to reduce allocations on hot paths - Use indexed loops (getXxxCount() + getXxxAt(i)) for repeated fields instead of getXxxList() to avoid list copy overhead - Update all test files to use LightProto API --- managed-ledger/pom.xml | 17 +- .../bookkeeper/mledger/LedgerOffloader.java | 4 +- .../bookkeeper/mledger/ManagedLedger.java | 2 +- .../mledger/ManagedLedgerFactoryConfig.java | 6 +- .../mledger/MetadataCompressionConfig.java | 14 +- .../mledger/impl/EntryCountEstimator.java | 18 +- .../mledger/impl/ManagedCursorImpl.java | 177 +++++++++--------- .../impl/ManagedLedgerFactoryImpl.java | 129 +++++++------ .../mledger/impl/ManagedLedgerImpl.java | 130 +++++++------ .../impl/ManagedLedgerOfflineBacklog.java | 8 +- .../bookkeeper/mledger/impl/MetaStore.java | 4 +- .../mledger/impl/MetaStoreImpl.java | 124 +++++++----- .../mledger/impl/ReadOnlyCursorImpl.java | 4 +- .../impl/ReadOnlyManagedLedgerImpl.java | 23 +-- .../mledger/impl/ShadowManagedLedgerImpl.java | 33 ++-- .../mledger/offload/OffloadUtils.java | 35 ++-- .../mledger/impl/EntryCacheManagerTest.java | 4 +- .../mledger/impl/EntryCacheTest.java | 4 +- .../mledger/impl/EntryCountEstimatorTest.java | 15 +- ...edCursorIndividualDeletedMessagesTest.java | 23 +-- .../impl/ManagedCursorInfoMetadataTest.java | 42 ++--- .../mledger/impl/ManagedCursorTest.java | 32 ++-- .../mledger/impl/ManagedLedgerBkTest.java | 9 +- .../ManagedLedgerFactoryShutdownTest.java | 25 ++- .../impl/ManagedLedgerInfoMetadataTest.java | 53 +++--- .../mledger/impl/ManagedLedgerTest.java | 38 ++-- .../mledger/impl/MetaStoreImplTest.java | 15 +- .../impl/OffloadEvictUnusedLedgersTest.java | 2 +- .../mledger/impl/OffloadLedgerDeleteTest.java | 48 +++-- .../mledger/impl/OffloadPrefixReadTest.java | 24 +-- .../mledger/impl/OffloadPrefixTest.java | 76 ++++---- .../bookkeeper/mledger/impl/PositionTest.java | 7 +- .../mledger/offload/OffloadUtilsTest.java | 26 +-- .../broker/service/BacklogQuotaManager.java | 4 +- .../PersistentMessageExpiryMonitor.java | 10 +- .../persistent/PersistentMessageFinder.java | 2 +- .../service/persistent/PersistentTopic.java | 5 +- .../jcloud/OffloadIndexBlockV2Builder.java | 2 +- .../impl/BlobStoreManagedLedgerOffloader.java | 25 +-- .../impl/OffloadIndexBlockV2BuilderImpl.java | 2 +- .../jcloud/impl/OffloadIndexBlockV2Impl.java | 6 +- ...reManagedLedgerOffloaderStreamingTest.java | 101 +++++----- .../jcloud/impl/MockManagedLedger.java | 6 +- .../offload/jcloud/impl/OffloadIndexTest.java | 4 +- .../jcloud/impl/OffloadIndexV2Test.java | 2 +- 45 files changed, 673 insertions(+), 667 deletions(-) diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index ef6a03d107040..87ebff5655b02 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -43,10 +43,6 @@ - - com.google.protobuf - protobuf-java - ${project.groupId} @@ -168,18 +164,13 @@ - org.xolstice.maven.plugins - protobuf-maven-plugin - ${protobuf-maven-plugin.version} - - com.google.protobuf:protoc:${protoc3.version}:exe:${os.detected.classifier} - true - + io.streamnative.lightproto + lightproto-maven-plugin + ${lightproto-maven-plugin.version} - generate-sources - compile + generate diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java index 11148ef1a59f5..c1740ba9ff3a3 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/LedgerOffloader.java @@ -26,7 +26,7 @@ import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.OffloadContext; import org.apache.pulsar.common.policies.data.OffloadPolicies; /** @@ -198,7 +198,7 @@ CompletableFuture readOffloaded(long ledgerId, UUID uid, CompletableFuture deleteOffloaded(long ledgerId, UUID uid, Map offloadDriverMetadata); - default CompletableFuture readOffloaded(long ledgerId, MLDataFormats.OffloadContext ledgerContext, + default CompletableFuture readOffloaded(long ledgerId, OffloadContext ledgerContext, Map offloadDriverMetadata) { throw new UnsupportedOperationException(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java index 8fb083bcd026c..0455f0efa8bb6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java @@ -35,7 +35,7 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.OpenCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.TerminateCallback; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.jspecify.annotations.Nullable; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java index 592fd2d385e5c..a915d651bd6b2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactoryConfig.java @@ -21,7 +21,7 @@ import lombok.Data; import org.apache.bookkeeper.common.annotation.InterfaceAudience; import org.apache.bookkeeper.common.annotation.InterfaceStability; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.CompressionType; /** * Configuration for a {@link ManagedLedgerFactory}. @@ -134,7 +134,7 @@ public class ManagedLedgerFactoryConfig { /** * ManagedLedgerInfo compression type. If the compression type is null or invalid, don't compress data. */ - private String managedLedgerInfoCompressionType = MLDataFormats.CompressionType.NONE.name(); + private String managedLedgerInfoCompressionType = CompressionType.NONE.name(); /** * ManagedLedgerInfo compression threshold. If the origin metadata size below configuration. @@ -145,7 +145,7 @@ public class ManagedLedgerFactoryConfig { /** * ManagedCursorInfo compression type. If the compression type is null or invalid, don't compress data. */ - private String managedCursorInfoCompressionType = MLDataFormats.CompressionType.NONE.name(); + private String managedCursorInfoCompressionType = CompressionType.NONE.name(); /** * ManagedCursorInfo compression threshold. If the origin metadata size below configuration. diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/MetadataCompressionConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/MetadataCompressionConfig.java index 7ce23192663e1..e246f73acb90c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/MetadataCompressionConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/MetadataCompressionConfig.java @@ -21,14 +21,14 @@ import lombok.AllArgsConstructor; import lombok.Data; import lombok.ToString; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.CompressionType; import org.apache.commons.lang3.StringUtils; @Data @AllArgsConstructor @ToString public class MetadataCompressionConfig { - MLDataFormats.CompressionType compressionType; + CompressionType compressionType; long compressSizeThresholdInBytes; public MetadataCompressionConfig(String compressionType) throws IllegalArgumentException { @@ -41,15 +41,15 @@ public MetadataCompressionConfig(String compressionType, long compressThreshold) } public static MetadataCompressionConfig noCompression = - new MetadataCompressionConfig(MLDataFormats.CompressionType.NONE, 0); + new MetadataCompressionConfig(CompressionType.NONE, 0); - private MLDataFormats.CompressionType parseCompressionType(String value) throws IllegalArgumentException { + private CompressionType parseCompressionType(String value) throws IllegalArgumentException { if (StringUtils.isEmpty(value)) { - return MLDataFormats.CompressionType.NONE; + return CompressionType.NONE; } - MLDataFormats.CompressionType compressionType; - compressionType = MLDataFormats.CompressionType.valueOf(value); + CompressionType compressionType; + compressionType = CompressionType.valueOf(value); return compressionType; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java index a5d41bad6a8d8..7ad45249e3c3c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimator.java @@ -26,7 +26,7 @@ import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; class EntryCountEstimator { // Prevent instantiation, this is a utility class with only static methods @@ -61,7 +61,7 @@ static int estimateEntryCountByBytesSize(int maxEntries, long maxSizeBytes, Posi * @param maxEntries stop further estimation if the number of estimated entries exceeds this value * @param maxSizeBytes the maximum size in bytes for the entries to be estimated * @param readPosition the position in the ledger from where to start reading - * @param ledgersInfo a map of ledger ID to {@link MLDataFormats.ManagedLedgerInfo.LedgerInfo} containing + * @param ledgersInfo a map of ledger ID to {@link ManagedLedgerInfo.LedgerInfo} containing * metadata for ledgers * @param lastLedgerId the ID of the last active ledger in the managed ledger * @param lastLedgerTotalEntries the total number of entries in the last active ledger @@ -69,7 +69,7 @@ static int estimateEntryCountByBytesSize(int maxEntries, long maxSizeBytes, Posi * @return the estimated number of entries that can be read */ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeBytes, Position readPosition, - NavigableMap + NavigableMap ledgersInfo, Long lastLedgerId, long lastLedgerTotalEntries, long lastLedgerTotalSize) { @@ -98,11 +98,11 @@ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeByt long remainingBytesSize = maxSizeBytes; long currentAvgSize = 0; // Get a collection of ledger info starting from the read position - Collection ledgersAfterReadPosition = + Collection ledgersAfterReadPosition = ledgersInfo.tailMap(readPosition.getLedgerId(), true).values(); // calculate the estimated entry count based on the remaining bytes and ledger metadata - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgersAfterReadPosition) { + for (ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgersAfterReadPosition) { if (remainingBytesSize <= 0 || estimatedEntryCount >= maxEntries) { // Stop processing if there are no more bytes remaining to allocate for entries // or if the estimated entry count exceeds the maximum allowed entries @@ -159,9 +159,9 @@ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeByt if (remainingBytesSize > 0 && estimatedEntryCount < maxEntries) { // need to find the previous non-empty ledger to find the average size if (currentAvgSize == 0) { - Collection ledgersBeforeReadPosition = + Collection ledgersBeforeReadPosition = ledgersInfo.headMap(readPosition.getLedgerId(), false).descendingMap().values(); - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgersBeforeReadPosition) { + for (ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgersBeforeReadPosition) { long ledgerTotalSize = ledgerInfo.getSize(); long ledgerTotalEntries = ledgerInfo.getEntries(); // Skip processing ledgers that have no entries or size @@ -184,7 +184,7 @@ static int internalEstimateEntryCountByBytesSize(int maxEntries, long maxSizeByt } private static Position adjustReadPosition(Position readPosition, - NavigableMap + NavigableMap ledgersInfo, Long lastLedgerId, long lastLedgerTotalEntries) { // Adjust the read position to ensure it falls within the valid range of available ledgers. @@ -195,7 +195,7 @@ private static Position adjustReadPosition(Position readPosition, } long lastKey = ledgersInfo.lastKey(); if (lastLedgerId == null && readPosition.getLedgerId() > lastKey) { - Map.Entry lastEntry = ledgersInfo.lastEntry(); + Map.Entry lastEntry = ledgersInfo.lastEntry(); if (lastEntry != null && lastEntry.getKey() == lastKey) { return PositionFactory.create(lastEntry.getKey(), Math.max(lastEntry.getValue().getEntries() - 1, 0)); } 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 7009ac750fe15..4267016ac8383 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 @@ -32,7 +32,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Range; import com.google.common.util.concurrent.RateLimiter; -import com.google.protobuf.InvalidProtocolBufferException; import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; @@ -60,8 +59,8 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; +import java.util.function.IntFunction; import java.util.function.Predicate; -import java.util.stream.Collectors; import java.util.stream.LongStream; import lombok.Getter; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; @@ -94,15 +93,15 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo.Builder; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; +import org.apache.bookkeeper.mledger.proto.BatchedEntryDeletionIndexInfo; +import org.apache.bookkeeper.mledger.proto.LongListMap; +import org.apache.bookkeeper.mledger.proto.LongProperty; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.MessageRange; +import org.apache.bookkeeper.mledger.proto.NestedPositionInfo; +import org.apache.bookkeeper.mledger.proto.PositionInfo; +import org.apache.bookkeeper.mledger.proto.StringProperty; import org.apache.bookkeeper.mledger.util.ManagedLedgerUtils; import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableInt; @@ -212,6 +211,9 @@ public class ManagedCursorImpl implements ManagedCursor { @Nullable protected final ConcurrentSkipListMap batchDeletedIndexes; protected final ReadWriteLock lock = new ReentrantReadWriteLock(); + // Reusable LightProto object for cursor position serialization (used only from persistPositionToLedger) + private final PositionInfo reusablePositionInfo = new PositionInfo(); + private RateLimiter markDeleteLimiter; // The cursor is considered "dirty" when there are mark-delete updates that are only applied in memory, // because of the rate limiting. @@ -428,11 +430,10 @@ private CompletableFuture computeCursorProperties( return updateCursorPropertiesResult; } - ManagedCursorInfo copy = ManagedCursorInfo - .newBuilder(ManagedCursorImpl.this.managedCursorInfo) - .clearCursorProperties() - .addAllCursorProperties(buildStringPropertiesMap(newProperties)) - .build(); + ManagedCursorInfo copy = new ManagedCursorInfo(); + copy.copyFrom(ManagedCursorImpl.this.managedCursorInfo); + copy.clearCursorProperties(); + copy.addAllCursorProperties(buildStringPropertiesMap(newProperties)); final Stat lastCursorLedgerStat = ManagedCursorImpl.this.cursorLedgerStat; ledger.getStore().asyncUpdateCursorInfo(ledger.getName(), name, copy, lastCursorLedgerStat, new MetaStoreCallback<>() { @@ -561,7 +562,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { // Recover properties map recoveredCursorProperties = new HashMap<>(); for (int i = 0; i < info.getCursorPropertiesCount(); i++) { - StringProperty property = info.getCursorProperties(i); + StringProperty property = info.getCursorPropertyAt(i); recoveredCursorProperties.put(property.getName(), property.getValue()); } } @@ -573,7 +574,8 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { Position recoveredPosition = PositionFactory.create(info.getMarkDeleteLedgerId(), info.getMarkDeleteEntryId()); if (info.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesList()); + recoverIndividualDeletedMessages(info.getIndividualDeletedMessagesCount(), + info::getIndividualDeletedMessageAt); } Map recoveredProperties = Collections.emptyMap(); @@ -581,7 +583,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { // Recover properties map recoveredProperties = new HashMap<>(); for (int i = 0; i < info.getPropertiesCount(); i++) { - LongProperty property = info.getProperties(i); + LongProperty property = info.getPropertyAt(i); recoveredProperties.put(property.getName(), property.getValue()); } } @@ -656,10 +658,10 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac LedgerEntry entry = seq.nextElement(); mbean.addReadCursorLedgerSize(entry.getLength()); - PositionInfo positionInfo; + PositionInfo positionInfo = new PositionInfo(); try { - positionInfo = PositionInfo.parseFrom(entry.getEntry()); - } catch (InvalidProtocolBufferException e) { + positionInfo.parseFrom(entry.getEntry()); + } catch (Exception e) { callback.operationFailed(new ManagedLedgerException(e)); return; } @@ -669,7 +671,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac // Recover properties map recoveredProperties = new HashMap<>(); for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { - LongProperty property = positionInfo.getProperties(i); + LongProperty property = positionInfo.getPropertyAt(i); recoveredProperties.put(property.getName(), property.getValue()); } } @@ -677,8 +679,9 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); recoverIndividualDeletedMessages(positionInfo); if (getConfig().isDeletionAtBatchIndexLevelEnabled() - && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { - recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); + && positionInfo.getBatchedEntryDeletionIndexInfosCount() > 0) { + recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfosCount(), + positionInfo::getBatchedEntryDeletionIndexInfoAt); } recoveredCursor(position, recoveredProperties, cursorProperties, lh); callback.operationComplete(); @@ -696,13 +699,20 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesCount(), + positionInfo::getIndividualDeletedMessageAt); } else if (positionInfo.getIndividualDeletedMessageRangesCount() > 0) { - List rangeList = positionInfo.getIndividualDeletedMessageRangesList(); lock.writeLock().lock(); try { - Map rangeMap = rangeList.stream().collect(Collectors.toMap(LongListMap::getKey, - list -> list.getValuesList().stream().mapToLong(i -> i).toArray())); + Map rangeMap = new HashMap<>(positionInfo.getIndividualDeletedMessageRangesCount()); + for (int i = 0; i < positionInfo.getIndividualDeletedMessageRangesCount(); i++) { + LongListMap list = positionInfo.getIndividualDeletedMessageRangeAt(i); + long[] values = new long[list.getValuesCount()]; + for (int idx = 0; idx < values.length; idx++) { + values[idx] = list.getValueAt(idx); + } + rangeMap.put(list.getKey(), values); + } // Guarantee compatability for the config "unackedRangesOpenCacheSetEnabled". if (getConfig().isUnackedRangesOpenCacheSetEnabled()) { individualDeletedMessages.build(rangeMap); @@ -738,12 +748,10 @@ private List buildLongPropertiesMap(Map properties) { if (ranges == null || ranges.length <= 0) { return; } - org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap.Builder lmBuilder = LongListMap.newBuilder() - .setKey(id); + LongListMap lm = new LongListMap().setKey(id); for (long range : ranges) { - lmBuilder.addValues(range); + lm.addValue(range); } - LongListMap lm = lmBuilder.build(); longListMap.add(lm); serializedSize.add(lm.getSerializedSize()); }); @@ -751,13 +759,14 @@ private List buildLongPropertiesMap(Map properties) { return longListMap; } - private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { + private void recoverIndividualDeletedMessages(int count, IntFunction accessor) { lock.writeLock().lock(); try { individualDeletedMessages.clear(); - individualDeletedMessagesList.forEach(messageRange -> { - MLDataFormats.NestedPositionInfo lowerEndpoint = messageRange.getLowerEndpoint(); - MLDataFormats.NestedPositionInfo upperEndpoint = messageRange.getUpperEndpoint(); + for (int i = 0; i < count; i++) { + MessageRange messageRange = accessor.apply(i); + NestedPositionInfo lowerEndpoint = messageRange.getLowerEndpoint(); + NestedPositionInfo upperEndpoint = messageRange.getUpperEndpoint(); if (lowerEndpoint.getLedgerId() == upperEndpoint.getLedgerId()) { individualDeletedMessages.addOpenClosed(lowerEndpoint.getLedgerId(), lowerEndpoint.getEntryId(), @@ -782,29 +791,29 @@ private void recoverIndividualDeletedMessages(List i individualDeletedMessages.addOpenClosed(upperEndpoint.getLedgerId(), -1, upperEndpoint.getLedgerId(), upperEndpoint.getEntryId()); } - }); + } } finally { lock.writeLock().unlock(); } } - private void recoverBatchDeletedIndexes ( - List batchDeletedIndexInfoList) { + private void recoverBatchDeletedIndexes(int count, IntFunction accessor) { Objects.requireNonNull(batchDeletedIndexes); lock.writeLock().lock(); try { this.batchDeletedIndexes.clear(); - batchDeletedIndexInfoList.forEach(batchDeletedIndexInfo -> { - 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); + for (int i = 0; i < count; i++) { + BatchedEntryDeletionIndexInfo batchDeletedIndexInfo = accessor.apply(i); + if (batchDeletedIndexInfo.getDeleteSetsCount() > 0) { + long[] array = new long[batchDeletedIndexInfo.getDeleteSetsCount()]; + for (int j = 0; j < array.length; j++) { + array[j] = batchDeletedIndexInfo.getDeleteSetAt(j); } this.batchDeletedIndexes.put( PositionFactory.create(batchDeletedIndexInfo.getPosition().getLedgerId(), batchDeletedIndexInfo.getPosition().getEntryId()), BitSet.valueOf(array)); } - }); + } } finally { lock.writeLock().unlock(); } @@ -2967,18 +2976,18 @@ private void persistPositionMetaStore(long cursorsLedgerId, Position position, M // When closing we store the last mark-delete position in the z-node itself, so we won't need the cursor ledger, // hence we write it as -1. The cursor ledger is deleted once the z-node write is confirmed. - ManagedCursorInfo.Builder info = ManagedCursorInfo.newBuilder() // - .setCursorsLedgerId(cursorsLedgerId) // - .setMarkDeleteLedgerId(position.getLedgerId()) // - .setMarkDeleteEntryId(position.getEntryId()) // - .setLastActive(lastActive); // + ManagedCursorInfo info = new ManagedCursorInfo() + .setCursorsLedgerId(cursorsLedgerId) + .setMarkDeleteLedgerId(position.getLedgerId()) + .setMarkDeleteEntryId(position.getEntryId()) + .setLastActive(lastActive); info.addAllProperties(buildPropertiesMap(properties)); info.addAllCursorProperties(buildStringPropertiesMap(cursorProperties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { - info.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()); + info.addAllBatchedEntryDeletionIndexInfos(buildBatchEntryDeletionIndexInfoList()); } } @@ -2986,7 +2995,7 @@ private void persistPositionMetaStore(long cursorsLedgerId, Position position, M log.debug("[{}][{}] Closing cursor at md-position: {}", ledger.getName(), name, position); } - ManagedCursorInfo cursorInfo = info.build(); + ManagedCursorInfo cursorInfo = info; ledger.getStore().asyncUpdateCursorInfo(ledger.getName(), name, cursorInfo, lastCursorLedgerStat, new MetaStoreCallback() { @Override @@ -3341,7 +3350,7 @@ private static List buildPropertiesMap(Map propertie List longProperties = new ArrayList<>(); properties.forEach((name, value) -> { - LongProperty lp = LongProperty.newBuilder().setName(name).setValue(value).build(); + LongProperty lp = new LongProperty().setName(name).setValue(value); longProperties.add(lp); }); @@ -3355,14 +3364,14 @@ private static List buildStringPropertiesMap(Map List stringProperties = new ArrayList<>(); properties.forEach((name, value) -> { - StringProperty sp = StringProperty.newBuilder().setName(name).setValue(value).build(); + StringProperty sp = new StringProperty().setName(name).setValue(value); stringProperties.add(sp); }); return stringProperties; } - private List buildIndividualDeletedMessageRanges() { + private List buildIndividualDeletedMessageRanges() { lock.writeLock().lock(); try { if (individualDeletedMessages.isEmpty()) { @@ -3370,30 +3379,17 @@ private List buildIndividualDeletedMessageRanges() { return Collections.emptyList(); } - MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo - .newBuilder(); - - MLDataFormats.MessageRange.Builder messageRangeBuilder = MLDataFormats.MessageRange - .newBuilder(); - AtomicInteger acksSerializedSize = new AtomicInteger(0); List rangeList = new ArrayList<>(); individualDeletedMessages.forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { - MLDataFormats.NestedPositionInfo lowerPosition = nestedPositionBuilder + MessageRange messageRange = new MessageRange(); + messageRange.setLowerEndpoint() .setLedgerId(lowerKey) - .setEntryId(lowerValue) - .build(); - - MLDataFormats.NestedPositionInfo upperPosition = nestedPositionBuilder + .setEntryId(lowerValue); + messageRange.setUpperEndpoint() .setLedgerId(upperKey) - .setEntryId(upperValue) - .build(); - - MessageRange messageRange = messageRangeBuilder - .setLowerEndpoint(lowerPosition) - .setUpperEndpoint(upperPosition) - .build(); + .setEntryId(upperValue); acksSerializedSize.addAndGet(messageRange.getSerializedSize()); rangeList.add(messageRange); @@ -3409,31 +3405,25 @@ private List buildIndividualDeletedMessageRanges() { } } - private List buildBatchEntryDeletionIndexInfoList() { + private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { if (batchDeletedIndexes == null || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } - MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo - .newBuilder(); - MLDataFormats.BatchedEntryDeletionIndexInfo.Builder batchDeletedIndexInfoBuilder = MLDataFormats - .BatchedEntryDeletionIndexInfo.newBuilder(); - List result = new ArrayList<>(); + List result = new ArrayList<>(); final var iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { final var entry = iterator.next(); - nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); - nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); - batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); + BatchedEntryDeletionIndexInfo batchDeletedIndexInfo = new BatchedEntryDeletionIndexInfo(); + batchDeletedIndexInfo.setPosition() + .setLedgerId(entry.getKey().getLedgerId()) + .setEntryId(entry.getKey().getEntryId()); long[] array = entry.getValue().toLongArray(); - List deleteSet = new ArrayList<>(array.length); for (long l : array) { - deleteSet.add(l); + batchDeletedIndexInfo.addDeleteSet(l); } - batchDeletedIndexInfoBuilder.clearDeleteSet(); - batchDeletedIndexInfoBuilder.addAllDeleteSet(deleteSet); - result.add(batchDeletedIndexInfoBuilder.build()); + result.add(batchDeletedIndexInfo); } return result; } finally { @@ -3444,9 +3434,11 @@ private List buildBatchEntryDeletio void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback, boolean ignoreClosedStateAfterFailure) { Position position = mdEntry.newPosition; - Builder piBuilder = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) + PositionInfo pi = reusablePositionInfo; + pi.clear(); + pi.setLedgerId(position.getLedgerId()) .setEntryId(position.getEntryId()) - .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()) + .addAllBatchedEntryDeletionIndexInfos(buildBatchEntryDeletionIndexInfoList()) .addAllProperties(buildPropertiesMap(mdEntry.properties)); Map internalRanges = null; @@ -3470,11 +3462,10 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin } } if (internalRanges != null && !internalRanges.isEmpty()) { - piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges)); + pi.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges)); } else { - piBuilder.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); + pi.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); } - PositionInfo pi = piBuilder.build(); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(), diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java index b8b5b78a62c0d..4e530a185cce1 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java @@ -26,7 +26,6 @@ import com.google.common.collect.BoundType; import com.google.common.collect.Maps; import com.google.common.collect.Range; -import com.google.protobuf.InvalidProtocolBufferException; import io.netty.util.concurrent.DefaultThreadFactory; import io.opentelemetry.api.OpenTelemetry; import java.io.IOException; @@ -72,11 +71,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; -import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.ManagedLedgerInfo.CursorInfo; -import org.apache.bookkeeper.mledger.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.ManagedLedgerInfo.MessageRangeInfo; -import org.apache.bookkeeper.mledger.ManagedLedgerInfo.PositionInfo; import org.apache.bookkeeper.mledger.MetadataCompressionConfig; import org.apache.bookkeeper.mledger.OpenTelemetryManagedLedgerCacheStats; import org.apache.bookkeeper.mledger.Position; @@ -90,10 +86,14 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; import org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl; import org.apache.bookkeeper.mledger.offload.OffloadUtils; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; +import org.apache.bookkeeper.mledger.proto.KeyValue; +import org.apache.bookkeeper.mledger.proto.LongProperty; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.MessageRange; +import org.apache.bookkeeper.mledger.proto.OffloadContext; +import org.apache.bookkeeper.mledger.proto.PositionInfo; import org.apache.bookkeeper.mledger.util.Errors; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -741,16 +741,17 @@ public CompletableFuture asyncExists(String ledgerName) { } @Override - public ManagedLedgerInfo getManagedLedgerInfo(String name) throws InterruptedException, ManagedLedgerException { + public org.apache.bookkeeper.mledger.ManagedLedgerInfo getManagedLedgerInfo(String name) + throws InterruptedException, ManagedLedgerException { class Result { - ManagedLedgerInfo info = null; + org.apache.bookkeeper.mledger.ManagedLedgerInfo info = null; ManagedLedgerException e = null; } final Result r = new Result(); final CountDownLatch latch = new CountDownLatch(1); asyncGetManagedLedgerInfo(name, new ManagedLedgerInfoCallback() { @Override - public void getInfoComplete(ManagedLedgerInfo info, Object ctx) { + public void getInfoComplete(org.apache.bookkeeper.mledger.ManagedLedgerInfo info, Object ctx) { r.info = info; latch.countDown(); } @@ -773,17 +774,18 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncGetManagedLedgerInfo(String name, ManagedLedgerInfoCallback callback, Object ctx) { store.getManagedLedgerInfo(name, false /* createIfMissing */, - new MetaStoreCallback() { + new MetaStoreCallback() { @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo pbInfo, Stat stat) { - ManagedLedgerInfo info = new ManagedLedgerInfo(); + public void operationComplete(ManagedLedgerInfo pbInfo, Stat stat) { + org.apache.bookkeeper.mledger.ManagedLedgerInfo info = + new org.apache.bookkeeper.mledger.ManagedLedgerInfo(); info.version = stat.getVersion(); info.creationDate = DateFormatter.format(stat.getCreationTimestamp()); info.modificationDate = DateFormatter.format(stat.getModificationTimestamp()); - info.ledgers = new ArrayList<>(pbInfo.getLedgerInfoCount()); + info.ledgers = new ArrayList<>(pbInfo.getLedgerInfosCount()); if (pbInfo.hasTerminatedPosition()) { - info.terminatedPosition = new PositionInfo(); + info.terminatedPosition = new org.apache.bookkeeper.mledger.ManagedLedgerInfo.PositionInfo(); info.terminatedPosition.ledgerId = pbInfo.getTerminatedPosition().getLedgerId(); info.terminatedPosition.entryId = pbInfo.getTerminatedPosition().getEntryId(); } @@ -791,21 +793,22 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo pbInfo, Stat stat) if (pbInfo.getPropertiesCount() > 0) { info.properties = new TreeMap(); for (int i = 0; i < pbInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = pbInfo.getProperties(i); + KeyValue property = pbInfo.getPropertyAt(i); info.properties.put(property.getKey(), property.getValue()); } } - for (int i = 0; i < pbInfo.getLedgerInfoCount(); i++) { - MLDataFormats.ManagedLedgerInfo.LedgerInfo pbLedgerInfo = pbInfo.getLedgerInfo(i); - LedgerInfo ledgerInfo = new LedgerInfo(); + for (int i = 0; i < pbInfo.getLedgerInfosCount(); i++) { + LedgerInfo pbLedgerInfo = pbInfo.getLedgerInfoAt(i); + org.apache.bookkeeper.mledger.ManagedLedgerInfo.LedgerInfo ledgerInfo = + new org.apache.bookkeeper.mledger.ManagedLedgerInfo.LedgerInfo(); ledgerInfo.ledgerId = pbLedgerInfo.getLedgerId(); ledgerInfo.entries = pbLedgerInfo.hasEntries() ? pbLedgerInfo.getEntries() : null; ledgerInfo.size = pbLedgerInfo.hasSize() ? pbLedgerInfo.getSize() : null; ledgerInfo.timestamp = pbLedgerInfo.hasTimestamp() ? pbLedgerInfo.getTimestamp() : null; ledgerInfo.isOffloaded = pbLedgerInfo.hasOffloadContext(); if (pbLedgerInfo.hasOffloadContext()) { - MLDataFormats.OffloadContext offloadContext = pbLedgerInfo.getOffloadContext(); + OffloadContext offloadContext = pbLedgerInfo.getOffloadContext(); UUID uuid = new UUID(offloadContext.getUidMsb(), offloadContext.getUidLsb()); ledgerInfo.offloadedContextUuid = uuid.toString(); } @@ -823,7 +826,7 @@ public void operationComplete(List cursorsList, Stat stat) { CompletableFuture cursorFuture = new CompletableFuture<>(); cursorsFutures.add(cursorFuture); store.asyncGetCursorInfo(name, cursorName, - new MetaStoreCallback() { + new MetaStoreCallback() { @Override public void operationComplete(ManagedCursorInfo pbCursorInfo, Stat stat) { CursorInfo cursorInfo = new CursorInfo(); @@ -835,7 +838,9 @@ public void operationComplete(ManagedCursorInfo pbCursorInfo, Stat stat) { cursorInfo.cursorsLedgerId = pbCursorInfo.getCursorsLedgerId(); if (pbCursorInfo.hasMarkDeleteLedgerId()) { - cursorInfo.markDelete = new PositionInfo(); + cursorInfo.markDelete = + new org.apache.bookkeeper.mledger.ManagedLedgerInfo + .PositionInfo(); cursorInfo.markDelete.ledgerId = pbCursorInfo.getMarkDeleteLedgerId(); cursorInfo.markDelete.entryId = pbCursorInfo.getMarkDeleteEntryId(); } @@ -843,7 +848,7 @@ public void operationComplete(ManagedCursorInfo pbCursorInfo, Stat stat) { if (pbCursorInfo.getPropertiesCount() > 0) { cursorInfo.properties = new TreeMap(); for (int i = 0; i < pbCursorInfo.getPropertiesCount(); i++) { - LongProperty property = pbCursorInfo.getProperties(i); + LongProperty property = pbCursorInfo.getPropertyAt(i); cursorInfo.properties.put(property.getName(), property.getValue()); } } @@ -852,7 +857,8 @@ public void operationComplete(ManagedCursorInfo pbCursorInfo, Stat stat) { cursorInfo.individualDeletedMessages = new ArrayList<>(); for (int i = 0; i < pbCursorInfo .getIndividualDeletedMessagesCount(); i++) { - MessageRange range = pbCursorInfo.getIndividualDeletedMessages(i); + MessageRange range = + pbCursorInfo.getIndividualDeletedMessageAt(i); MessageRangeInfo rangeInfo = new MessageRangeInfo(); rangeInfo.from.ledgerId = range.getLowerEndpoint().getLedgerId(); rangeInfo.from.entryId = range.getLowerEndpoint().getEntryId(); @@ -963,7 +969,7 @@ void deleteManagedLedger(String managedLedgerName, CompletableFuture { // First delete all cursors resources List> futures = info.cursors.entrySet().stream() @@ -985,17 +991,19 @@ public void getInfoFailed(ManagedLedgerException exception, Object ctx) { }, ctx); } - private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, ManagedLedgerInfo info, + private void deleteManagedLedgerData(BookKeeper bkc, String managedLedgerName, + org.apache.bookkeeper.mledger.ManagedLedgerInfo info, CompletableFuture mlConfigFuture, DeleteLedgerCallback callback, Object ctx) { - final CompletableFuture> + final CompletableFuture> ledgerInfosFuture = new CompletableFuture<>(); store.getManagedLedgerInfo(managedLedgerName, false, null, new MetaStoreCallback<>() { @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { - Map infos = new HashMap<>(); - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : mlInfo.getLedgerInfoList()) { + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { + Map infos = new HashMap<>(); + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); infos.put(ls.getLedgerId(), ls); } ledgerInfosFuture.complete(infos); @@ -1016,22 +1024,23 @@ public void operationFailed(MetaStoreException e) { .thenCombine(ledgerInfosFuture, Pair::of) .thenCompose(pair -> { ManagedLedgerConfig mlConfig = pair.getLeft(); - Map ledgerInfos = pair.getRight(); + Map ledgerInfos = pair.getRight(); if (mlConfig == null || ledgerInfos == null) { return CompletableFuture.completedFuture(null); } - MLDataFormats.ManagedLedgerInfo.LedgerInfo ls = ledgerInfos.get(li.ledgerId); + LedgerInfo ls = ledgerInfos.get(li.ledgerId); if (ls.getOffloadContext().hasUidMsb()) { - MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); - newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); + LedgerInfo newInfo = new LedgerInfo(); + newInfo.copyFrom(ls); + newInfo.setOffloadContext().setBookkeeperDeleted(true); String driverName = OffloadUtils.getOffloadDriverName(ls, mlConfig.getLedgerOffloader().getOffloadDriverName()); Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, mlConfig.getLedgerOffloader().getOffloadDriverMetadata()); - OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); + OffloadUtils.setOffloadDriverMetadata(newInfo, driverName, driverMetadata); UUID uuid = new UUID(ls.getOffloadContext().getUidMsb(), ls.getOffloadContext().getUidLsb()); @@ -1181,9 +1190,9 @@ public void estimateUnloadedTopicBacklog(PersistentOfflineTopicStats offlineTopi long totalSize = 0; BookKeeper.DigestType digestType = (BookKeeper.DigestType) ((List) ctx).get(0); byte[] password = (byte[]) ((List) ctx).get(1); - NavigableMap ledgers = + NavigableMap ledgers = getManagedLedgersInfo(topicName, accurate, digestType, password); - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers.values()) { + for (LedgerInfo ls : ledgers.values()) { numberOfEntries += ls.getEntries(); totalSize += ls.getSize(); if (accurate) { @@ -1201,20 +1210,21 @@ public void estimateUnloadedTopicBacklog(PersistentOfflineTopicStats offlineTopi offlineTopicStats.statGeneratedAt.setTime(System.currentTimeMillis()); } - private NavigableMap getManagedLedgersInfo( + private NavigableMap getManagedLedgersInfo( final TopicName topicName, boolean accurate, BookKeeper.DigestType digestType, byte[] password) throws Exception { - final NavigableMap ledgers = new ConcurrentSkipListMap<>(); + final NavigableMap ledgers = new ConcurrentSkipListMap<>(); String managedLedgerName = topicName.getPersistenceNamingEncoding(); MetaStore store = getMetaStore(); final CountDownLatch mlMetaCounter = new CountDownLatch(1); store.getManagedLedgerInfo(managedLedgerName, false /* createIfMissing */, - new MetaStore.MetaStoreCallback() { + new MetaStore.MetaStoreCallback() { @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : mlInfo.getLedgerInfoList()) { + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); ledgers.put(ls.getLedgerId(), ls); } @@ -1227,12 +1237,11 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) BKException.getMessage(rc)); } if (rc == BKException.Code.OK) { - MLDataFormats.ManagedLedgerInfo.LedgerInfo info = - MLDataFormats.ManagedLedgerInfo.LedgerInfo - .newBuilder().setLedgerId(id) + LedgerInfo info = + new LedgerInfo() + .setLedgerId(id) .setEntries(lh.getLastAddConfirmed() + 1) - .setSize(lh.getLength()).setTimestamp(System.currentTimeMillis()) - .build(); + .setSize(lh.getLength()).setTimestamp(System.currentTimeMillis()); ledgers.put(id, info); mlMetaCounter.countDown(); } else if (Errors.isNoSuchLedgerExistsException(rc)) { @@ -1281,7 +1290,7 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { } public void calculateCursorBacklogs(final TopicName topicName, - final NavigableMap ledgers, + final NavigableMap ledgers, final PersistentOfflineTopicStats offlineTopicStats, boolean accurate, BookKeeper.DigestType digestType, byte[] password) throws Exception { if (ledgers.isEmpty()) { @@ -1294,7 +1303,7 @@ public void calculateCursorBacklogs(final TopicName topicName, final long errorInReadingCursor = -1; final var ledgerRetryMap = new ConcurrentHashMap(); - final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); + final LedgerInfo ledgerInfo = ledgers.lastEntry().getValue(); final Position lastLedgerPosition = PositionFactory.create(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1); if (log.isDebugEnabled()) { @@ -1369,10 +1378,11 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq, lh.getId()); } else { LedgerEntry entry = seq.nextElement(); - MLDataFormats.PositionInfo positionInfo; + PositionInfo positionInfo; try { - positionInfo = MLDataFormats.PositionInfo.parseFrom(entry.getEntry()); - } catch (InvalidProtocolBufferException e) { + positionInfo = new PositionInfo(); + positionInfo.parseFrom(entry.getEntry()); + } catch (Exception e) { log.warn( "[{}] Error reading position from metadata ledger {} for cursor " + "{}: {}", managedLedgerName, ledgerId, cursorName, e); @@ -1408,9 +1418,9 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq, }; // end of cursor meta read callback store.asyncGetCursorInfo(managedLedgerName, cursorName, - new MetaStore.MetaStoreCallback() { + new MetaStore.MetaStoreCallback() { @Override - public void operationComplete(MLDataFormats.ManagedCursorInfo info, + public void operationComplete(ManagedCursorInfo info, Stat stat) { long cursorLedgerId = info.getCursorsLedgerId(); if (log.isDebugEnabled()) { @@ -1501,7 +1511,7 @@ public void operationFailed(ManagedLedgerException.MetaStoreException e) { // need a better way than to duplicate the functionality below from ML private long getNumberOfEntries(Range range, - NavigableMap ledgers) { + NavigableMap ledgers) { Position fromPosition = range.lowerEndpoint(); boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; Position toPosition = range.upperEndpoint(); @@ -1521,14 +1531,14 @@ private long getNumberOfEntries(Range range, count += toIncluded ? 1 : 0; // 2. Add the entries in the ledger pointed by fromPosition - MLDataFormats.ManagedLedgerInfo.LedgerInfo li = ledgers.get(fromPosition.getLedgerId()); + LedgerInfo li = ledgers.get(fromPosition.getLedgerId()); if (li != null) { count += li.getEntries() - (fromPosition.getEntryId() + 1); count += fromIncluded ? 1 : 0; } // 3. Add the whole ledgers entries in between - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers + for (LedgerInfo ls : ledgers .subMap(fromPosition.getLedgerId(), false, toPosition.getLedgerId(), false).values()) { count += ls.getEntries(); } @@ -1549,7 +1559,8 @@ private Position tryGetMDPosition(BookKeeper bookKeeper, long ledgerId, String c if (log.isDebugEnabled()) { log.debug(" Read entry {} from ledger {} for cursor {}", lastEntry, ledgerId, cursorName); } - MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); + PositionInfo positionInfo = new PositionInfo(); + positionInfo.parseFrom(ledgerEntry.getEntry()); lastAckedMessagePosition = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); if (log.isDebugEnabled()) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java index 0606c6fb074b9..4c2a344d85c44 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java @@ -127,11 +127,11 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.bookkeeper.mledger.offload.OffloadUtils; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; +import org.apache.bookkeeper.mledger.proto.KeyValue; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.NestedPositionInfo; +import org.apache.bookkeeper.mledger.proto.OffloadContext; import org.apache.bookkeeper.mledger.util.CallbackMutex; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; @@ -429,14 +429,15 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}] Recovering managed ledger terminated at {}", name, lastConfirmedEntry); } - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); ledgers.put(ls.getLedgerId(), ls); } if (mlInfo.getPropertiesCount() > 0) { propertiesMap = new HashMap(); for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); + KeyValue property = mlInfo.getPropertyAt(i); propertiesMap.put(property.getKey(), property.getValue()); } } @@ -458,9 +459,9 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { if (State.Terminated.equals(state)) { currentLedger = lh; } - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(id) + LedgerInfo info = new LedgerInfo().setLedgerId(id) .setEntries(lh.getLastAddConfirmed() + 1).setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); + .setTimestamp(clock.millis()); ledgers.put(id, info); if (managedLedgerInterceptor != null) { managedLedgerInterceptor @@ -630,7 +631,7 @@ public void operationFailed(MetaStoreException e) { } } - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build(); + LedgerInfo info = new LedgerInfo().setLedgerId(lh.getId()).setTimestamp(0); ledgers.put(lh.getId(), info); // Save it back to ensure all nodes exist @@ -1425,14 +1426,19 @@ public CompletableFuture asyncAddLedgerProperty(long ledgerId, String key, CompletableFuture f = new CompletableFuture<>(); transformLedgerInfo(ledgerId, oldInfo -> { - List oldProperties = oldInfo.getPropertiesList(); Map newPropertiesMap = new HashMap<>(); - oldProperties.forEach(kv -> newPropertiesMap.put(kv.getKey(), kv.getValue())); + for (int i = 0; i < oldInfo.getPropertiesCount(); i++) { + KeyValue kv = oldInfo.getPropertyAt(i); + newPropertiesMap.put(kv.getKey(), kv.getValue()); + } newPropertiesMap.put(key, value); - List newProperties = newPropertiesMap.entrySet().stream() - .map(e -> MLDataFormats.KeyValue.newBuilder() - .setKey(e.getKey()).setValue(e.getValue()).build()).toList(); - return oldInfo.toBuilder().clearProperties().addAllProperties(newProperties).build(); + List newProperties = newPropertiesMap.entrySet().stream() + .map(e -> new KeyValue().setKey(e.getKey()).setValue(e.getValue())).toList(); + LedgerInfo copy = new LedgerInfo(); + copy.copyFrom(oldInfo); + copy.clearProperties(); + copy.addAllProperties(newProperties); + return copy; }) .thenAccept(v -> f.complete(null)) .exceptionally(t -> { @@ -1460,14 +1466,19 @@ public CompletableFuture asyncRemoveLedgerProperty(long ledgerId, String k CompletableFuture f = new CompletableFuture<>(); transformLedgerInfo(ledgerId, oldInfo -> { - List oldProperties = oldInfo.getPropertiesList(); Map newPropertiesMap = new HashMap<>(); - oldProperties.forEach(kv -> newPropertiesMap.put(kv.getKey(), kv.getValue())); + for (int i = 0; i < oldInfo.getPropertiesCount(); i++) { + KeyValue kv = oldInfo.getPropertyAt(i); + newPropertiesMap.put(kv.getKey(), kv.getValue()); + } newPropertiesMap.remove(key); - List newProperties = newPropertiesMap.entrySet().stream() - .map(e -> MLDataFormats.KeyValue.newBuilder() - .setKey(e.getKey()).setValue(e.getValue()).build()).toList(); - return oldInfo.toBuilder().clearProperties().addAllProperties(newProperties).build(); + List newProperties = newPropertiesMap.entrySet().stream() + .map(e -> new KeyValue().setKey(e.getKey()).setValue(e.getValue())).toList(); + LedgerInfo copy = new LedgerInfo(); + copy.copyFrom(oldInfo); + copy.clearProperties(); + copy.addAllProperties(newProperties); + return copy; }) .thenAccept(v -> f.complete(null)) .exceptionally(t -> { @@ -1494,7 +1505,8 @@ public CompletableFuture asyncGetLedgerProperty(long ledgerId, String ke if (li.getPropertiesCount() <= 0) { return CompletableFuture.completedFuture(null); } - for (MLDataFormats.KeyValue kv : li.getPropertiesList()) { + for (int i = 0; i < li.getPropertiesCount(); i++) { + KeyValue kv = li.getPropertyAt(i); if (kv.getKey().equals(key)) { return CompletableFuture.completedFuture(kv.getValue()); } @@ -1742,7 +1754,7 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct lastLedgerCreationFailureTimestamp = clock.millis(); } else { log.info("[{}] Created new ledger {}", name, lh.getId()); - LedgerInfo newLedger = LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build(); + LedgerInfo newLedger = new LedgerInfo().setLedgerId(lh.getId()).setTimestamp(0); final MetaStoreCallback cb = new MetaStoreCallback() { @Override public void operationComplete(Void v, Stat stat) { @@ -1947,8 +1959,8 @@ synchronized void ledgerClosed(final LedgerHandle lh, Long lastAddConfirmed) { log.debug("[{}] Ledger has been closed id={} entries={}", name, lh.getId(), entriesInLedger); } if (entriesInLedger > 0) { - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lh.getId()).setEntries(entriesInLedger) - .setSize(lh.getLength()).setTimestamp(clock.millis()).build(); + LedgerInfo info = new LedgerInfo().setLedgerId(lh.getId()).setEntries(entriesInLedger) + .setSize(lh.getLength()).setTimestamp(clock.millis()); ledgers.put(lh.getId(), info); } else { // The last ledger was empty, so we can discard it @@ -2221,11 +2233,11 @@ CompletableFuture getLedgerHandle(long ledgerId) { && config.getLedgerOffloader().getOffloadPolicies() .getManagedLedgerOffloadedReadPriority() == OffloadedReadPriority.BOOKKEEPER_FIRST && info != null && info.hasOffloadContext() - && !info.getOffloadContext().getBookkeeperDeleted()) { + && !info.getOffloadContext().isBookkeeperDeleted()) { openFuture = bookKeeper.newOpenLedgerOp().withRecovery(!isReadOnly()).withLedgerId(ledgerId) .withDigestType(config.getDigestType()).withPassword(config.getPassword()).execute(); - } else if (info != null && info.hasOffloadContext() && info.getOffloadContext().getComplete()) { + } else if (info != null && info.hasOffloadContext() && info.getOffloadContext().isComplete()) { UUID uid = new UUID(info.getOffloadContext().getUidMsb(), info.getOffloadContext().getUidLsb()); // TODO: improve this to load ledger offloader by driver name recorded in metadata @@ -2855,7 +2867,7 @@ private void maybeOffload(long offloadThresholdInBytes, long offloadThresholdInS final long now = System.currentTimeMillis(); sizeSummed += size; - final boolean alreadyOffloaded = info.hasOffloadContext() && info.getOffloadContext().getComplete(); + final boolean alreadyOffloaded = info.hasOffloadContext() && info.getOffloadContext().isComplete(); if (alreadyOffloaded) { alreadyOffloadedSize += size; } else { @@ -2895,7 +2907,7 @@ private boolean isLedgerRetentionOverSizeQuota(long retentionSizeInMB, long tota boolean isOffloadedNeedsDelete(OffloadContext offload, Optional offloadPolicies) { long elapsedMs = clock.millis() - offload.getTimestamp(); - return offloadPolicies.filter(policies -> offload.getComplete() && !offload.getBookkeeperDeleted() + return offloadPolicies.filter(policies -> offload.isComplete() && !offload.isBookkeeperDeleted() && policies.getManagedLedgerOffloadDeletionLagInMillis() != null && policies.getManagedLedgerOffloadDeletionLagInMillis() >= 0 && elapsedMs > policies.getManagedLedgerOffloadDeletionLagInMillis()).isPresent(); @@ -3139,14 +3151,15 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { doDeleteLedgers(ledgersToDelete); for (LedgerInfo ls : offloadedLedgersToDelete) { - LedgerInfo.Builder newInfoBuilder = ls.toBuilder(); - newInfoBuilder.getOffloadContextBuilder().setBookkeeperDeleted(true); + LedgerInfo newInfo = new LedgerInfo(); + newInfo.copyFrom(ls); + newInfo.setOffloadContext().setBookkeeperDeleted(true); String driverName = OffloadUtils.getOffloadDriverName(ls, config.getLedgerOffloader().getOffloadDriverName()); Map driverMetadata = OffloadUtils.getOffloadDriverMetadata(ls, config.getLedgerOffloader().getOffloadDriverMetadata()); - OffloadUtils.setOffloadDriverMetadata(newInfoBuilder, driverName, driverMetadata); - ledgers.put(ls.getLedgerId(), newInfoBuilder.build()); + OffloadUtils.setOffloadDriverMetadata(newInfo, driverName, driverMetadata); + ledgers.put(ls.getLedgerId(), newInfo); } if (log.isDebugEnabled()) { @@ -3405,7 +3418,7 @@ private CompletableFuture asyncDeleteLedgerFromBookKeeper(long ledgerId) { } private void asyncDeleteLedger(long ledgerId, LedgerInfo info) { - if (!info.getOffloadContext().getBookkeeperDeleted()) { + if (!info.getOffloadContext().isBookkeeperDeleted()) { // only delete if it hasn't been previously deleted for offload asyncDeleteLedger(ledgerId, DEFAULT_LEDGER_DELETE_RETRIES); } @@ -3614,7 +3627,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct for (LedgerInfo ls : ledgers.headMap(current).values()) { if (requestOffloadTo.getLedgerId() > ls.getLedgerId()) { // don't offload if ledger has already been offloaded, or is empty - if (!ls.getOffloadContext().getComplete() && ls.getSize() > 0) { + if (!ls.getOffloadContext().isComplete() && ls.getSize() > 0) { ledgersToOffload.add(ls); } } else { @@ -3837,16 +3850,17 @@ private CompletableFuture prepareLedgerInfoForOffloaded(long ledgerId, UUI name, scheduledExecutor); } - LedgerInfo.Builder builder = oldInfo.toBuilder(); - builder.getOffloadContextBuilder() + LedgerInfo newInfo = new LedgerInfo(); + newInfo.copyFrom(oldInfo); + newInfo.setOffloadContext() .setUidMsb(uuid.getMostSignificantBits()) .setUidLsb(uuid.getLeastSignificantBits()); OffloadUtils.setOffloadDriverMetadata( - builder, + newInfo, offloadDriverName, offloadDriverMetadata ); - return builder.build(); + return newInfo; }) .whenComplete((result, exception) -> { if (exception != null) { @@ -3865,8 +3879,9 @@ private CompletableFuture completeLedgerInfoForOffloaded(long ledgerId, UU UUID existingUuid = new UUID(oldInfo.getOffloadContext().getUidMsb(), oldInfo.getOffloadContext().getUidLsb()); if (existingUuid.equals(uuid)) { - LedgerInfo.Builder builder = oldInfo.toBuilder(); - builder.getOffloadContextBuilder() + LedgerInfo newInfo = new LedgerInfo(); + newInfo.copyFrom(oldInfo); + newInfo.setOffloadContext() .setTimestamp(clock.millis()) .setComplete(true); @@ -3875,11 +3890,11 @@ private CompletableFuture completeLedgerInfoForOffloaded(long ledgerId, UU Map driverMetadata = OffloadUtils.getOffloadDriverMetadata( oldInfo, config.getLedgerOffloader().getOffloadDriverMetadata()); OffloadUtils.setOffloadDriverMetadata( - builder, + newInfo, driverName, driverMetadata ); - return builder.build(); + return newInfo; } else { throw new OffloadConflict( "Existing UUID(" + existingUuid + ") in metadata for offload" @@ -4370,29 +4385,31 @@ protected ManagedLedgerInfo getManagedLedgerInfo() { } private ManagedLedgerInfo getManagedLedgerInfo(LedgerInfo newLedger) { - ManagedLedgerInfo.Builder mlInfo = ManagedLedgerInfo.newBuilder().addAllLedgerInfo(ledgers.values()) - .addLedgerInfo(newLedger); + ManagedLedgerInfo mlInfo = new ManagedLedgerInfo(); + mlInfo.addAllLedgerInfos(ledgers.values()); + mlInfo.addLedgerInfo().copyFrom(newLedger); return buildManagedLedgerInfo(mlInfo); } private ManagedLedgerInfo buildManagedLedgerInfo(Map ledgers) { - ManagedLedgerInfo.Builder mlInfo = ManagedLedgerInfo.newBuilder().addAllLedgerInfo(ledgers.values()); + ManagedLedgerInfo mlInfo = new ManagedLedgerInfo(); + mlInfo.addAllLedgerInfos(ledgers.values()); return buildManagedLedgerInfo(mlInfo); } - private ManagedLedgerInfo buildManagedLedgerInfo(ManagedLedgerInfo.Builder mlInfo) { + private ManagedLedgerInfo buildManagedLedgerInfo(ManagedLedgerInfo mlInfo) { if (state == State.Terminated) { - mlInfo.setTerminatedPosition(NestedPositionInfo.newBuilder().setLedgerId(lastConfirmedEntry.getLedgerId()) - .setEntryId(lastConfirmedEntry.getEntryId())); + mlInfo.setTerminatedPosition() + .setLedgerId(lastConfirmedEntry.getLedgerId()) + .setEntryId(lastConfirmedEntry.getEntryId()); } if (managedLedgerInterceptor != null) { managedLedgerInterceptor.onUpdateManagedLedgerInfo(propertiesMap); } for (Map.Entry property : propertiesMap.entrySet()) { - mlInfo.addProperties(MLDataFormats.KeyValue.newBuilder() - .setKey(property.getKey()).setValue(property.getValue())); + mlInfo.addProperty().setKey(property.getKey()).setValue(property.getValue()); } - return mlInfo.build(); + return mlInfo; } /** @@ -4693,7 +4710,7 @@ private void checkReadTimeout() { public long getOffloadedSize() { long offloadedSize = 0; for (LedgerInfo li : ledgers.values()) { - if (li.hasOffloadContext() && li.getOffloadContext().getComplete()) { + if (li.hasOffloadContext() && li.getOffloadContext().isComplete()) { offloadedSize += li.getSize(); } } @@ -4956,7 +4973,7 @@ public CompletableFuture getManagedLedgerInternalSta info.ledgerId = li.getLedgerId(); info.entries = li.getEntries(); info.size = li.getSize(); - info.offloaded = li.hasOffloadContext() && li.getOffloadContext().getComplete(); + info.offloaded = li.hasOffloadContext() && li.getOffloadContext().isComplete(); if (includeLedgerMetadata) { // lookup metadata from the hashmap which contains completed async operations LedgerMetadata lm = ledgerMetadataFutures.get(li.getLedgerId()).getNow(null); @@ -4975,7 +4992,8 @@ public CompletableFuture getManagedLedgerInternalSta } if (li.getPropertiesCount() > 0) { Map properties = new HashMap<>(li.getPropertiesCount()); - for (MLDataFormats.KeyValue kv : li.getPropertiesList()) { + for (int i = 0; i < li.getPropertiesCount(); i++) { + KeyValue kv = li.getPropertyAt(i); properties.put(kv.getKey(), kv.getValue()); } info.properties = properties; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java index 9a1753c715eff..7c7b2936cf43b 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerOfflineBacklog.java @@ -27,7 +27,7 @@ import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; import org.slf4j.Logger; @@ -53,7 +53,7 @@ public ManagedLedgerOfflineBacklog(DigestType digestType, byte[] password, Strin // need a better way than to duplicate the functionality below from ML private long getNumberOfEntries(Range range, - NavigableMap ledgers) { + NavigableMap ledgers) { Position fromPosition = range.lowerEndpoint(); boolean fromIncluded = range.lowerBoundType() == BoundType.CLOSED; Position toPosition = range.upperEndpoint(); @@ -73,14 +73,14 @@ private long getNumberOfEntries(Range range, count += toIncluded ? 1 : 0; // 2. Add the entries in the ledger pointed by fromPosition - MLDataFormats.ManagedLedgerInfo.LedgerInfo li = ledgers.get(fromPosition.getLedgerId()); + ManagedLedgerInfo.LedgerInfo li = ledgers.get(fromPosition.getLedgerId()); if (li != null) { count += li.getEntries() - (fromPosition.getEntryId() + 1); count += fromIncluded ? 1 : 0; } // 3. Add the whole ledgers entries in between - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ls : ledgers + for (ManagedLedgerInfo.LedgerInfo ls : ledgers .subMap(fromPosition.getLedgerId(), false, toPosition.getLedgerId(), false).values()) { count += ls.getEntries(); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStore.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStore.java index 21e12d81a727d..9aff7baae9b55 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStore.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStore.java @@ -22,8 +22,8 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.metadata.api.Stat; /** diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java index 611d9d60202cd..a3f0591d81c32 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java @@ -18,7 +18,6 @@ */ package org.apache.bookkeeper.mledger.impl; -import com.google.protobuf.InvalidProtocolBufferException; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; @@ -38,10 +37,11 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetadataNotFoundException; import org.apache.bookkeeper.mledger.MetadataCompressionConfig; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.CompressionType; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfoMetadata; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfoMetadata; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; @@ -67,6 +67,14 @@ public class MetaStoreImpl implements MetaStore, Consumer { private final Map> managedLedgerInfoUpdateCallbackMap; + // Reusable LightProto objects for compression metadata serialization/deserialization. + // These are used from MetaStore callbacks which are dispatched on the ordered executor, + // so each thread gets its own instance via ThreadLocal. + private static final ThreadLocal tlMlInfoMetadata = + ThreadLocal.withInitial(ManagedLedgerInfoMetadata::new); + private static final ThreadLocal tlCursorInfoMetadata = + ThreadLocal.withInitial(ManagedCursorInfoMetadata::new); + public MetaStoreImpl(MetadataStore store, OrderedExecutor executor) { this.store = store; this.executor = executor; @@ -104,7 +112,7 @@ public void getManagedLedgerInfo(String ledgerName, boolean createIfMissing, Map info = parseManagedLedgerInfo(optResult.get().getValue()); info = updateMLInfoTimestamp(info); callback.operationComplete(info, optResult.get().getStat()); - } catch (InvalidProtocolBufferException e) { + } catch (Exception e) { callback.operationFailed(getException(e)); } } else { @@ -114,17 +122,15 @@ public void getManagedLedgerInfo(String ledgerName, boolean createIfMissing, Map store.put(path, new byte[0], Optional.of(-1L)) .thenAccept(stat -> { - ManagedLedgerInfo.Builder ledgerBuilder = ManagedLedgerInfo.newBuilder(); + ManagedLedgerInfo ledgerBuilder = new ManagedLedgerInfo(); if (properties != null) { properties.forEach((k, v) -> { - ledgerBuilder.addProperties( - MLDataFormats.KeyValue.newBuilder() - .setKey(k) - .setValue(v) - .build()); + ledgerBuilder.addProperty() + .setKey(k) + .setValue(v); }); } - callback.operationComplete(ledgerBuilder.build(), stat); + callback.operationComplete(ledgerBuilder, stat); }).exceptionally(ex -> { callback.operationFailed(getException(ex)); return null; @@ -152,12 +158,12 @@ public CompletableFuture> getManagedLedgerPropertiesAsync(St CompletableFuture> result = new CompletableFuture<>(); getManagedLedgerInfo(name, false, new MetaStoreCallback<>() { @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { HashMap propertiesMap = new HashMap<>(mlInfo.getPropertiesCount()); if (mlInfo.getPropertiesCount() > 0) { for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); - propertiesMap.put(property.getKey(), property.getValue()); + propertiesMap.put(mlInfo.getPropertyAt(i).getKey(), + mlInfo.getPropertyAt(i).getValue()); } } result.complete(propertiesMap); @@ -224,7 +230,7 @@ public void asyncGetCursorInfo(String ledgerName, String cursorName, try { ManagedCursorInfo info = parseManagedCursorInfo(optRes.get().getValue()); callback.operationComplete(info, optRes.get().getStat()); - } catch (InvalidProtocolBufferException e) { + } catch (Exception e) { callback.operationFailed(getException(e)); } } else { @@ -359,7 +365,7 @@ public void accept(Notification notification) { info = parseManagedLedgerInfo(optResult.get().getValue()); info = updateMLInfoTimestamp(info); callback.onUpdate(info, optResult.get().getStat()); - } catch (InvalidProtocolBufferException e) { + } catch (Exception e) { log.error("[{}] Error when parseManagedLedgerInfo", ledgerName, e); } } @@ -376,25 +382,29 @@ public void accept(Notification notification) { // - ledger has timestamp which is the normal case now private static ManagedLedgerInfo updateMLInfoTimestamp(ManagedLedgerInfo info) { - List infoList = new ArrayList<>(info.getLedgerInfoCount()); + List infoList = new ArrayList<>(info.getLedgerInfosCount()); long currentTime = System.currentTimeMillis(); - for (ManagedLedgerInfo.LedgerInfo ledgerInfo : info.getLedgerInfoList()) { + for (int i = 0; i < info.getLedgerInfosCount(); i++) { + ManagedLedgerInfo.LedgerInfo ledgerInfo = info.getLedgerInfoAt(i); if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0) { - ManagedLedgerInfo.LedgerInfo.Builder singleInfoBuilder = ledgerInfo.toBuilder(); - singleInfoBuilder.setTimestamp(currentTime); - infoList.add(singleInfoBuilder.build()); + ManagedLedgerInfo.LedgerInfo updatedInfo = new ManagedLedgerInfo.LedgerInfo(); + updatedInfo.copyFrom(ledgerInfo); + updatedInfo.setTimestamp(currentTime); + infoList.add(updatedInfo); } else { infoList.add(ledgerInfo); } } - ManagedLedgerInfo.Builder mlInfo = ManagedLedgerInfo.newBuilder(); - mlInfo.addAllLedgerInfo(infoList); + ManagedLedgerInfo mlInfo = new ManagedLedgerInfo(); + mlInfo.addAllLedgerInfos(infoList); if (info.hasTerminatedPosition()) { - mlInfo.setTerminatedPosition(info.getTerminatedPosition()); + mlInfo.setTerminatedPosition().copyFrom(info.getTerminatedPosition()); + } + for (int i = 0; i < info.getPropertiesCount(); i++) { + mlInfo.addProperty().copyFrom(info.getPropertyAt(i)); } - mlInfo.addAllProperties(info.getPropertiesList()); - return mlInfo.build(); + return mlInfo; } private static MetaStoreException getException(Throwable t) { @@ -416,11 +426,10 @@ public byte[] compressLedgerInfo(ManagedLedgerInfo managedLedgerInfo) { int uncompressedSize = managedLedgerInfo.getSerializedSize(); if (uncompressedSize > ledgerInfoCompressionConfig.getCompressSizeThresholdInBytes()) { - MLDataFormats.ManagedLedgerInfoMetadata mlInfoMetadata = MLDataFormats.ManagedLedgerInfoMetadata - .newBuilder() - .setCompressionType(compressionType) - .setUncompressedSize(uncompressedSize) - .build(); + ManagedLedgerInfoMetadata mlInfoMetadata = tlMlInfoMetadata.get(); + mlInfoMetadata.clear(); + mlInfoMetadata.setCompressionType(compressionType) + .setUncompressedSize(uncompressedSize); return compressManagedInfo(managedLedgerInfo.toByteArray(), mlInfoMetadata.toByteArray(), mlInfoMetadata.getSerializedSize(), compressionType); } @@ -436,11 +445,10 @@ public byte[] compressCursorInfo(ManagedCursorInfo managedCursorInfo) { int uncompressedSize = managedCursorInfo.getSerializedSize(); if (uncompressedSize > cursorInfoCompressionConfig.getCompressSizeThresholdInBytes()) { - MLDataFormats.ManagedCursorInfoMetadata metadata = MLDataFormats.ManagedCursorInfoMetadata - .newBuilder() - .setCompressionType(compressionType) - .setUncompressedSize(uncompressedSize) - .build(); + ManagedCursorInfoMetadata metadata = tlCursorInfoMetadata.get(); + metadata.clear(); + metadata.setCompressionType(compressionType) + .setUncompressedSize(uncompressedSize); return compressManagedInfo(managedCursorInfo.toByteArray(), metadata.toByteArray(), metadata.getSerializedSize(), compressionType); } @@ -448,57 +456,71 @@ public byte[] compressCursorInfo(ManagedCursorInfo managedCursorInfo) { return managedCursorInfo.toByteArray(); } - public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws InvalidProtocolBufferException { + public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws Exception { ByteBuf byteBuf = Unpooled.wrappedBuffer(data); byte[] metadataBytes = extractCompressMetadataBytes(byteBuf); if (metadataBytes != null) { try { - MLDataFormats.ManagedLedgerInfoMetadata metadata = - MLDataFormats.ManagedLedgerInfoMetadata.parseFrom(metadataBytes); + ManagedLedgerInfoMetadata metadata = tlMlInfoMetadata.get(); + metadata.clear(); + metadata.parseFrom(metadataBytes); ByteBuf uncompressed = getCompressionCodec(metadata.getCompressionType()) .decode(byteBuf, metadata.getUncompressedSize()); try { - return ManagedLedgerInfo.parseFrom(uncompressed.nioBuffer()); + ManagedLedgerInfo info = new ManagedLedgerInfo(); + info.parseFrom(uncompressed, uncompressed.readableBytes()); + return info; } finally { uncompressed.release(); } } catch (Exception e) { log.error("Failed to parse managedLedgerInfo metadata, " + "fall back to parse managedLedgerInfo directly.", e); - return ManagedLedgerInfo.parseFrom(data); + ManagedLedgerInfo info = new ManagedLedgerInfo(); + info.parseFrom(data); + return info; } finally { byteBuf.release(); } } else { - return ManagedLedgerInfo.parseFrom(data); + ManagedLedgerInfo info = new ManagedLedgerInfo(); + info.parseFrom(data); + return info; } } - public ManagedCursorInfo parseManagedCursorInfo(byte[] data) throws InvalidProtocolBufferException { + public ManagedCursorInfo parseManagedCursorInfo(byte[] data) throws Exception { ByteBuf byteBuf = Unpooled.wrappedBuffer(data); byte[] metadataBytes = extractCompressMetadataBytes(byteBuf); if (metadataBytes != null) { try { - MLDataFormats.ManagedCursorInfoMetadata metadata = - MLDataFormats.ManagedCursorInfoMetadata.parseFrom(metadataBytes); + ManagedCursorInfoMetadata metadata = tlCursorInfoMetadata.get(); + metadata.clear(); + metadata.parseFrom(metadataBytes); ByteBuf uncompressed = getCompressionCodec(metadata.getCompressionType()) .decode(byteBuf, metadata.getUncompressedSize()); try { - return ManagedCursorInfo.parseFrom(uncompressed.nioBuffer()); + ManagedCursorInfo info = new ManagedCursorInfo(); + info.parseFrom(uncompressed, uncompressed.readableBytes()); + return info; } finally { uncompressed.release(); } } catch (Exception e) { log.error("Failed to parse ManagedCursorInfo metadata, " + "fall back to parse ManagedCursorInfo directly", e); - return ManagedCursorInfo.parseFrom(data); + ManagedCursorInfo info = new ManagedCursorInfo(); + info.parseFrom(data); + return info; } finally { byteBuf.release(); } } else { - return ManagedCursorInfo.parseFrom(data); + ManagedCursorInfo info = new ManagedCursorInfo(); + info.parseFrom(data); + return info; } } @@ -509,7 +531,7 @@ public ManagedCursorInfo parseManagedCursorInfo(byte[] data) throws InvalidProto * [MAGIC_NUMBER](2) + [METADATA_SIZE](4) + [METADATA_PAYLOAD] + [MANAGED_LEDGER_INFO_PAYLOAD] */ private byte[] compressManagedInfo(byte[] info, byte[] metadata, int metadataSerializedSize, - MLDataFormats.CompressionType compressionType) { + CompressionType compressionType) { if (compressionType == null || compressionType.equals(CompressionType.NONE)) { return info; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 8c6bbca3384bd..ff7731e475fed 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -26,7 +26,7 @@ import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; @Slf4j public class ReadOnlyCursorImpl extends ManagedCursorImpl implements ReadOnlyCursor { @@ -67,7 +67,7 @@ public void asyncClose(final AsyncCallbacks.CloseCallback callback, final Object callback.closeComplete(ctx); } - public MLDataFormats.ManagedLedgerInfo.LedgerInfo getCurrentLedgerInfo() { + public ManagedLedgerInfo.LedgerInfo getCurrentLedgerInfo() { return this.ledger.getLedgersInfo().get(this.readPosition.getLedgerId()); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 1fb2aa3629092..8139148fd8fe5 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -34,9 +34,9 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.KeyValue; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.metadata.api.Stat; @Slf4j @@ -57,13 +57,14 @@ public CompletableFuture initialize() { public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { state = State.LedgerOpened; - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); ledgers.put(ls.getLedgerId(), ls); } if (mlInfo.getPropertiesCount() > 0) { for (int i = 0; i < mlInfo.getPropertiesCount(); i++) { - MLDataFormats.KeyValue property = mlInfo.getProperties(i); + KeyValue property = mlInfo.getPropertyAt(i); propertiesMap.put(property.getKey(), property.getValue()); } } @@ -77,9 +78,9 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { .withDigestType(config.getDigestType()).withPassword(config.getPassword()).execute() .thenAccept(readHandle -> { readHandle.readLastAddConfirmedAsync().thenAccept(lastAddConfirmed -> { - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId) + LedgerInfo info = new LedgerInfo().setLedgerId(lastLedgerId) .setEntries(lastAddConfirmed + 1).setSize(readHandle.getLength()) - .setTimestamp(clock.millis()).build(); + .setTimestamp(clock.millis()); ledgers.put(lastLedgerId, info); future.complete(null); @@ -87,8 +88,8 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { if (ex instanceof CompletionException && ex.getCause() instanceof IllegalArgumentException) { // The last ledger was empty, so we cannot read the last add confirmed. - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId) - .setEntries(0).setSize(0).setTimestamp(clock.millis()).build(); + LedgerInfo info = new LedgerInfo().setLedgerId(lastLedgerId) + .setEntries(0).setSize(0).setTimestamp(clock.millis()); ledgers.put(lastLedgerId, info); future.complete(null); } else { @@ -100,8 +101,8 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { if (ex instanceof CompletionException && ex.getCause() instanceof ArrayIndexOutOfBoundsException) { // The last ledger was empty, so we cannot read the last add confirmed. - LedgerInfo info = LedgerInfo.newBuilder().setLedgerId(lastLedgerId).setEntries(0) - .setSize(0).setTimestamp(clock.millis()).build(); + LedgerInfo info = new LedgerInfo().setLedgerId(lastLedgerId).setEntries(0) + .setSize(0).setTimestamp(clock.millis()); ledgers.put(lastLedgerId, info); future.complete(null); } else { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java index 46cd1335a17c0..d17a871586215 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImpl.java @@ -38,8 +38,9 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.NestedPositionInfo; import org.apache.pulsar.metadata.api.Stat; /** @@ -80,7 +81,7 @@ private void doInitialize(ManagedLedgerInitializeLedgerCallback callback, Object ); store.getManagedLedgerInfo(sourceMLName, false, null, new MetaStore.MetaStoreCallback<>() { @Override - public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { + public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { if (log.isDebugEnabled()) { log.debug("[{}][{}] Source ML info:{}", name, sourceMLName, mlInfo); } @@ -90,7 +91,7 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) return; } sourceLedgersStat = stat; - if (mlInfo.getLedgerInfoCount() == 0) { + if (mlInfo.getLedgerInfosCount() == 0) { // Small chance here, since shadow topic is created after source topic exists. log.warn("[{}] Source topic ledger list is empty! source={},mlInfo={},stat={}", name, sourceMLName, mlInfo, stat); @@ -99,14 +100,15 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) } if (mlInfo.hasTerminatedPosition()) { - MLDataFormats.NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); + NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); lastConfirmedEntry = PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}][{}] Recovering managed ledger terminated at {}", name, sourceMLName, lastConfirmedEntry); } - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); ledgers.put(ls.getLedgerId(), ls); } @@ -119,11 +121,11 @@ public void operationComplete(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) } if (rc == BKException.Code.OK) { LedgerInfo info = - LedgerInfo.newBuilder() + new LedgerInfo() .setLedgerId(lastLedgerId) .setEntries(lh.getLastAddConfirmed() + 1) .setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); + .setTimestamp(clock.millis()); ledgers.put(lastLedgerId, info); //Always consider the last ledger is opened in source. @@ -271,7 +273,7 @@ public synchronized void asyncTerminate(AsyncCallbacks.TerminateCallback callbac * 2. old ledgers deleted. * 3. old ledger offload info updated (including ledger deleted from bookie by offloader) */ - private synchronized void processSourceManagedLedgerInfo(MLDataFormats.ManagedLedgerInfo mlInfo, Stat stat) { + private synchronized void processSourceManagedLedgerInfo(ManagedLedgerInfo mlInfo, Stat stat) { if (log.isDebugEnabled()) { log.debug("[{}][{}] new SourceManagedLedgerInfo:{}, prevStat={},stat={}", name, sourceMLName, mlInfo, @@ -285,14 +287,15 @@ private synchronized void processSourceManagedLedgerInfo(MLDataFormats.ManagedLe sourceLedgersStat = stat; if (mlInfo.hasTerminatedPosition()) { - MLDataFormats.NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); + NestedPositionInfo terminatedPosition = mlInfo.getTerminatedPosition(); lastConfirmedEntry = PositionFactory.create(terminatedPosition.getLedgerId(), terminatedPosition.getEntryId()); log.info("[{}][{}] Process managed ledger terminated at {}", name, sourceMLName, lastConfirmedEntry); } TreeMap newLedgerInfos = new TreeMap<>(); - for (LedgerInfo ls : mlInfo.getLedgerInfoList()) { + for (int i = 0; i < mlInfo.getLedgerInfosCount(); i++) { + LedgerInfo ls = mlInfo.getLedgerInfoAt(i); newLedgerInfos.put(ls.getLedgerId(), ls); } @@ -308,9 +311,9 @@ private synchronized void processSourceManagedLedgerInfo(MLDataFormats.ManagedLe log.info("[{}] Old ledger info updated in source,ledgerId={}", name, ledgerId); // ledger deleted from bookkeeper by offloader. if (ledgerInfo.hasOffloadContext() - && ledgerInfo.getOffloadContext().getBookkeeperDeleted() + && ledgerInfo.getOffloadContext().isBookkeeperDeleted() && (!oldLedgerInfo.hasOffloadContext() || !oldLedgerInfo.getOffloadContext() - .getBookkeeperDeleted())) { + .isBookkeeperDeleted())) { log.info("[{}] Old ledger removed from bookkeeper by offloader in source,ledgerId={}", name, ledgerId); invalidateReadHandle(ledgerId); @@ -332,11 +335,11 @@ private synchronized void processSourceManagedLedgerInfo(MLDataFormats.ManagedLe log.debug("[{}] Opened new source ledger {}", name, lastLedgerId); } if (rc == BKException.Code.OK) { - LedgerInfo info = LedgerInfo.newBuilder() + LedgerInfo info = new LedgerInfo() .setLedgerId(lastLedgerId) .setEntries(lh.getLastAddConfirmed() + 1) .setSize(lh.getLength()) - .setTimestamp(clock.millis()).build(); + .setTimestamp(clock.millis()); ledgers.put(lastLedgerId, info); currentLedger = lh; currentLedgerEntries = 0; diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java index 9c9feb2aa7f7c..6faad763c85b2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/offload/OffloadUtils.java @@ -36,10 +36,10 @@ import org.apache.bookkeeper.common.util.Backoff; import org.apache.bookkeeper.common.util.Retries; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.KeyValue; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadDriverMetadata; +import org.apache.bookkeeper.mledger.proto.KeyValue; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.OffloadContext; +import org.apache.bookkeeper.mledger.proto.OffloadDriverMetadata; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.DataFormats; @@ -55,7 +55,10 @@ public static Map getOffloadDriverMetadata(LedgerInfo ledgerInfo if (ctx.hasDriverMetadata()) { OffloadDriverMetadata driverMetadata = ctx.getDriverMetadata(); if (driverMetadata.getPropertiesCount() > 0) { - driverMetadata.getPropertiesList().forEach(kv -> metadata.put(kv.getKey(), kv.getValue())); + for (int i = 0; i < driverMetadata.getPropertiesCount(); i++) { + KeyValue kv = driverMetadata.getPropertyAt(i); + metadata.put(kv.getKey(), kv.getValue()); + } } } } @@ -70,7 +73,10 @@ public static Map getOffloadDriverMetadata(LedgerInfo ledgerInfo OffloadDriverMetadata driverMetadata = ctx.getDriverMetadata(); if (driverMetadata.getPropertiesCount() > 0) { Map metadata = new HashMap(); - driverMetadata.getPropertiesList().forEach(kv -> metadata.put(kv.getKey(), kv.getValue())); + for (int i = 0; i < driverMetadata.getPropertiesCount(); i++) { + KeyValue kv = driverMetadata.getPropertyAt(i); + metadata.put(kv.getKey(), kv.getValue()); + } return metadata; } } @@ -91,20 +97,13 @@ public static String getOffloadDriverName(LedgerInfo ledgerInfo, String defaultD return defaultDriverName; } - public static void setOffloadDriverMetadata(LedgerInfo.Builder infoBuilder, + public static void setOffloadDriverMetadata(LedgerInfo infoBuilder, String driverName, Map offloadDriverMetadata) { - infoBuilder.getOffloadContextBuilder() - .getDriverMetadataBuilder() - .setName(driverName); - infoBuilder.getOffloadContextBuilder().getDriverMetadataBuilder().clearProperties(); - offloadDriverMetadata.forEach((k, v) -> infoBuilder - .getOffloadContextBuilder() - .getDriverMetadataBuilder() - .addProperties(KeyValue.newBuilder() - .setKey(k) - .setValue(v) - .build())); + OffloadDriverMetadata driverMeta = infoBuilder.setOffloadContext().setDriverMetadata(); + driverMeta.setName(driverName); + driverMeta.clearProperties(); + offloadDriverMetadata.forEach((k, v) -> driverMeta.addProperty().setKey(k).setValue(v)); } public static byte[] buildLedgerMetadataFormat(LedgerMetadata metadata) { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 49743d9386c90..2b9f40e7a9593 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheDisabled; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.awaitility.Awaitility; import org.testng.Assert; @@ -411,7 +411,7 @@ void entryCacheDisabledAsyncReadEntry() throws Exception { final CountDownLatch counter = new CountDownLatch(1); when(ml1.getLastConfirmedEntry()).thenReturn(PositionFactory.create(1L, 1L)); when(ml1.getOptionalLedgerInfo(lh.getId())).thenReturn(Optional.of(mock( - MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); + ManagedLedgerInfo.LedgerInfo.class))); entryCache.asyncReadEntry(lh, PositionFactory.create(1L, 1L), new AsyncCallbacks.ReadEntryCallback() { public void readEntryComplete(Entry entry, Object ctx) { Assert.assertNotEquals(entry, null); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java index 0fe889a2da5ec..40b8a1044d62d 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java @@ -48,7 +48,7 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.annotations.Test; @@ -64,7 +64,7 @@ protected void setUpTestCase() throws Exception { when(ml.getMbean()).thenReturn(new ManagedLedgerMBeanImpl(ml)); when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); when(ml.getOptionalLedgerInfo(0L)).thenReturn(Optional.of(mock( - MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); + ManagedLedgerInfo.LedgerInfo.class))); } @Test(timeOut = 5000) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java index 1b7bcbf816dc2..28f69709a31af 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java @@ -29,12 +29,12 @@ import java.util.TreeMap; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class EntryCountEstimatorTest { - private NavigableMap ledgersInfo; + private NavigableMap ledgersInfo; private Position readPosition; private Long lastLedgerId; private long lastLedgerTotalEntries; @@ -66,14 +66,13 @@ public void setup() { readPosition = PositionFactory.create(1L, 0); } - private MLDataFormats.ManagedLedgerInfo.LedgerInfo createLedgerInfo( + private ManagedLedgerInfo.LedgerInfo createLedgerInfo( long ledgerId, long entries, long size) { - return MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder() + return new ManagedLedgerInfo.LedgerInfo() .setLedgerId(ledgerId) .setEntries(entries) .setSize(size) - .setTimestamp(0) - .build(); + .setTimestamp(0); } private int estimateEntryCountByBytesSize(long maxSizeBytes) { @@ -235,9 +234,9 @@ public void testWithOnlySecondLastLedgerAndEmptyLastLedger() { public void testWithMultipleEmptyLedgers() { readPosition = PositionFactory.LATEST; long secondLastLedgerId = ledgersInfo.lowerKey(lastLedgerId); - MLDataFormats.ManagedLedgerInfo.LedgerInfo secondLastLedgerInfo = ledgersInfo.get(secondLastLedgerId); + ManagedLedgerInfo.LedgerInfo secondLastLedgerInfo = ledgersInfo.get(secondLastLedgerId); // make the second last ledger empty - ledgersInfo.put(secondLastLedgerId, secondLastLedgerInfo.toBuilder().setEntries(0).setSize(0).build()); + ledgersInfo.put(secondLastLedgerId, new ManagedLedgerInfo.LedgerInfo().copyFrom(secondLastLedgerInfo).setEntries(0).setSize(0)); lastLedgerTotalEntries = 0; lastLedgerTotalSize = 0; long expectedEntries = 50; diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java index b6c4cc5895db5..529825a340128 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java @@ -32,9 +32,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.NestedPositionInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.MessageRange; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.testng.annotations.Test; @@ -99,24 +98,18 @@ void testRecoverIndividualDeletedMessages() throws Exception { } private static LedgerInfo createLedgerInfo(long ledgerId, long entries, long size) { - return LedgerInfo.newBuilder().setLedgerId(ledgerId).setEntries(entries).setSize(size) - .setTimestamp(System.currentTimeMillis()).build(); + return new LedgerInfo().setLedgerId(ledgerId).setEntries(entries).setSize(size) + .setTimestamp(System.currentTimeMillis()); } private static MessageRange createMessageRange(long lowerLedgerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { - NestedPositionInfo.Builder nestedPositionBuilder = NestedPositionInfo.newBuilder(); - MessageRange.Builder messageRangeBuilder = MessageRange.newBuilder(); + MessageRange messageRange = new MessageRange(); - nestedPositionBuilder.setLedgerId(lowerLedgerId); - nestedPositionBuilder.setEntryId(lowerEntryId); - messageRangeBuilder.setLowerEndpoint(nestedPositionBuilder.build()); + messageRange.setLowerEndpoint().setLedgerId(lowerLedgerId).setEntryId(lowerEntryId); + messageRange.setUpperEndpoint().setLedgerId(upperLedgerId).setEntryId(upperEntryId); - nestedPositionBuilder.setLedgerId(upperLedgerId); - nestedPositionBuilder.setEntryId(upperEntryId); - messageRangeBuilder.setUpperEndpoint(nestedPositionBuilder.build()); - - return messageRangeBuilder.build(); + return messageRange; } private static Range createPositionRange(long lowerLedgerId, long lowerEntryId, long upperLedgerId, diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java index 1f8243469cea5..389d5e4754ca8 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java @@ -22,11 +22,9 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.MetadataCompressionConfig; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; import org.apache.pulsar.common.api.proto.CompressionType; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -51,43 +49,39 @@ private Object[][] compressionTypeProvider() { }; } - private MLDataFormats.ManagedCursorInfo.Builder generateManagedCursorInfo(long ledgerId, int positionNumber) { - MLDataFormats.ManagedCursorInfo.Builder builder = MLDataFormats.ManagedCursorInfo.newBuilder(); + private ManagedCursorInfo generateManagedCursorInfo(long ledgerId, int positionNumber) { + ManagedCursorInfo cursorInfo = new ManagedCursorInfo(); - builder.setCursorsLedgerId(ledgerId); - builder.setMarkDeleteLedgerId(ledgerId); + cursorInfo.setCursorsLedgerId(ledgerId); + cursorInfo.setMarkDeleteLedgerId(ledgerId); - List batchedEntryDeletionIndexInfos = new ArrayList<>(); for (int i = 0; i < positionNumber; i++) { - MLDataFormats.NestedPositionInfo nestedPositionInfo = MLDataFormats.NestedPositionInfo.newBuilder() - .setEntryId(i).setLedgerId(i).build(); - MLDataFormats.BatchedEntryDeletionIndexInfo batchedEntryDeletionIndexInfo = MLDataFormats - .BatchedEntryDeletionIndexInfo.newBuilder().setPosition(nestedPositionInfo).build(); - batchedEntryDeletionIndexInfos.add(batchedEntryDeletionIndexInfo); + cursorInfo.addBatchedEntryDeletionIndexInfo() + .setPosition() + .setEntryId(i) + .setLedgerId(i); } - builder.addAllBatchedEntryDeletionIndexInfo(batchedEntryDeletionIndexInfos); - return builder; + return cursorInfo; } @Test(dataProvider = "compressionTypeProvider") - public void testEncodeAndDecode(String compressionType) throws IOException { + public void testEncodeAndDecode(String compressionType) throws Exception { long ledgerId = 10000; - MLDataFormats.ManagedCursorInfo.Builder builder = generateManagedCursorInfo(ledgerId, 1000); + ManagedCursorInfo managedCursorInfo = generateManagedCursorInfo(ledgerId, 1000); MetaStoreImpl metaStore; if (INVALID_TYPE.equals(compressionType)) { IllegalArgumentException compressionTypeEx = expectThrows(IllegalArgumentException.class, () -> { new MetaStoreImpl(null, null, null, new MetadataCompressionConfig(compressionType)); }); assertEquals(compressionTypeEx.getMessage(), - "No enum constant org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType." + "No enum constant org.apache.bookkeeper.mledger.proto.CompressionType." + compressionType); return; } else { metaStore = new MetaStoreImpl(null, null, null, new MetadataCompressionConfig(compressionType)); } - MLDataFormats.ManagedCursorInfo managedCursorInfo = builder.build(); byte[] compressionBytes = metaStore.compressCursorInfo(managedCursorInfo); log.info("[{}] Uncompressed data size: {}, compressed data size: {}", compressionType, managedCursorInfo.getSerializedSize(), compressionBytes.length); @@ -96,8 +90,8 @@ public void testEncodeAndDecode(String compressionType) throws IOException { } // parse compression data and unCompression data, check their results. - MLDataFormats.ManagedCursorInfo info1 = metaStore.parseManagedCursorInfo(compressionBytes); - MLDataFormats.ManagedCursorInfo info2 = metaStore.parseManagedCursorInfo(managedCursorInfo.toByteArray()); + ManagedCursorInfo info1 = metaStore.parseManagedCursorInfo(compressionBytes); + ManagedCursorInfo info2 = metaStore.parseManagedCursorInfo(managedCursorInfo.toByteArray()); assertEquals(info1, info2); } @@ -107,11 +101,11 @@ public void testCompressionThreshold(String compressionType) throws IOException long ledgerId = 10000; // should not compress - MLDataFormats.ManagedCursorInfo smallInfo = generateManagedCursorInfo(ledgerId, 1).build(); + ManagedCursorInfo smallInfo = generateManagedCursorInfo(ledgerId, 1); assertTrue(smallInfo.getSerializedSize() < compressThreshold); // should compress - MLDataFormats.ManagedCursorInfo bigInfo = generateManagedCursorInfo(ledgerId, 1000).build(); + ManagedCursorInfo bigInfo = generateManagedCursorInfo(ledgerId, 1000); assertTrue(bigInfo.getSerializedSize() > compressThreshold); MetaStoreImpl metaStore; @@ -121,7 +115,7 @@ public void testCompressionThreshold(String compressionType) throws IOException new MetadataCompressionConfig(compressionType, compressThreshold)); }); assertEquals(compressionTypeEx.getMessage(), - "No enum constant org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType." + "No enum constant org.apache.bookkeeper.mledger.proto.CompressionType." + compressionType); return; } else { 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 522e12903201c..8c50e729deca0 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 @@ -113,9 +113,9 @@ import org.apache.bookkeeper.mledger.ScanOutcome; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.PositionInfo; import org.apache.bookkeeper.mledger.util.ManagedLedgerTestUtil; import org.apache.bookkeeper.mledger.util.ManagedLedgerUtils; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; @@ -3634,8 +3634,8 @@ public void operationFailed(MetaStoreException e) { lh.asyncReadEntries(lastEntry, lastEntry, (rc1, lh1, seq, ctx1) -> { try { LedgerEntry entry = seq.nextElement(); - PositionInfo positionInfo; - positionInfo = PositionInfo.parseFrom(entry.getEntry()); + PositionInfo positionInfo = new PositionInfo(); + positionInfo.parseFrom(entry.getEntry()); c1.recoverIndividualDeletedMessages(positionInfo); individualDeletedMessagesCount.set(c1.getIndividuallyDeletedMessagesSet().asRanges().size()); } catch (Exception e) { @@ -3895,9 +3895,9 @@ public void testRecoverCursorAheadOfLastPosition() throws Exception { MetaStore mockMetaStore = mock(MetaStore.class); doAnswer(new Answer() { public Object answer(InvocationOnMock invocation) { - ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(cursorsLedgerId) + ManagedCursorInfo info = new ManagedCursorInfo().setCursorsLedgerId(cursorsLedgerId) .setMarkDeleteLedgerId(markDeleteLedgerId).setMarkDeleteEntryId(markDeleteEntryId) - .setLastActive(0L).build(); + .setLastActive(0L); Stat stat = mock(Stat.class); MetaStoreCallback callback = (MetaStoreCallback) invocation .getArguments()[2]; @@ -3956,12 +3956,11 @@ public void testRecoverCursorAfterResetToLatestForNewEntry() throws Exception { // Trigger the lastConfirmedEntry to move forward ml.addEntry(new byte[1]); - ManagedCursorInfo info = ManagedCursorInfo.newBuilder() + ManagedCursorInfo info = new ManagedCursorInfo() .setCursorsLedgerId(c.getCursorLedger()) .setMarkDeleteLedgerId(markDeleteBeforeRecover.getLedgerId()) .setMarkDeleteEntryId(markDeleteBeforeRecover.getEntryId()) - .setLastActive(0L) - .build(); + .setLastActive(0L); CountDownLatch latch = new CountDownLatch(1); AtomicBoolean failed = new AtomicBoolean(false); @@ -4023,12 +4022,11 @@ public void testRecoverCursorAfterResetToLatestForMultipleEntries() throws Excep final Position markDeleteBeforeRecover = c.getMarkDeletedPosition(); final Position readPositionBeforeRecover = c.getReadPosition(); - ManagedCursorInfo info = ManagedCursorInfo.newBuilder() + ManagedCursorInfo info = new ManagedCursorInfo() .setCursorsLedgerId(c.getCursorLedger()) .setMarkDeleteLedgerId(markDeleteBeforeRecover.getLedgerId()) .setMarkDeleteEntryId(markDeleteBeforeRecover.getEntryId()) - .setLastActive(0L) - .build(); + .setLastActive(0L); CountDownLatch latch = new CountDownLatch(1); AtomicBoolean failed = new AtomicBoolean(false); @@ -4666,7 +4664,7 @@ public void testCursorGetBacklog() throws Exception { Field field = ManagedLedgerImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); - ((ConcurrentSkipListMap) field.get(ledger)) + ((ConcurrentSkipListMap) field.get(ledger)) .remove(position.getLedgerId()); field = ManagedCursorImpl.class.getDeclaredField("markDeletePosition"); field.setAccessible(true); @@ -5657,8 +5655,8 @@ public void testEstimateEntryCountBySize() throws Exception { ml.addEntry(new byte[]{1, 2, 3, 4}); } long ledger3 = ml.getCurrentLedger().getId(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo1 = ml.getLedgersInfo().get(ledger1); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo2 = ml.getLedgersInfo().get(ledger2); + ManagedLedgerInfo.LedgerInfo ledgerInfo1 = ml.getLedgersInfo().get(ledger1); + ManagedLedgerInfo.LedgerInfo ledgerInfo2 = ml.getLedgersInfo().get(ledger2); long average1 = ledgerInfo1.getSize() / ledgerInfo1.getEntries() + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; long average2 = ledgerInfo2.getSize() / ledgerInfo2.getEntries() + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; long average3 = ml.getCurrentLedgerSize() / ml.getCurrentLedgerEntries() + BOOKKEEPER_READ_OVERHEAD_PER_ENTRY; @@ -5766,7 +5764,7 @@ void testForceCursorRecovery() throws Exception { ledger.addEntry("entry-1".getBytes(Encoding)); long invalidLedger = -1L; bk.setErrorCodeMap(invalidLedger, BKException.Code.BookieHandleNotAvailableException); - ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(invalidLedger).build(); + ManagedCursorInfo info = new ManagedCursorInfo().setCursorsLedgerId(invalidLedger); CountDownLatch latch = new CountDownLatch(1); MutableBoolean recovered = new MutableBoolean(false); VoidCallback callback = new VoidCallback() { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java index 77e089be54906..ee6a9c78658e7 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerBkTest.java @@ -59,7 +59,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.PositionInfo; import org.apache.bookkeeper.mledger.util.ThrowableToStringUtil; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.common.policies.data.PersistentOfflineTopicStats; @@ -804,11 +804,12 @@ public void testConfigPersistIndividualAckAsLongArray(boolean enable) throws Exc // Verify: the config affects. long cursorLedgerLac = cursor1.cursorLedger.getLastAddConfirmed(); LedgerEntry ledgerEntry = cursor1.cursorLedger.readEntries(cursorLedgerLac, cursorLedgerLac).nextElement(); - MLDataFormats.PositionInfo positionInfo = MLDataFormats.PositionInfo.parseFrom(ledgerEntry.getEntry()); + PositionInfo positionInfo = new PositionInfo(); + positionInfo.parseFrom(ledgerEntry.getEntry()); if (enable) { - assertNotEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); + assertNotEquals(positionInfo.getIndividualDeletedMessageRangesCount(), 0); } else { - assertEquals(positionInfo.getIndividualDeletedMessageRangesList().size(), 0); + assertEquals(positionInfo.getIndividualDeletedMessageRangesCount(), 0); } // cleanup diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java index 95f0a6b8c775e..df1cf82d9d42d 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryShutdownTest.java @@ -44,7 +44,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyCursor; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.Stat; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -77,11 +78,11 @@ private void setup() { throw new IllegalArgumentException("Path is null."); } if (path.endsWith(ledgerName)) { // ledger - MLDataFormats.ManagedLedgerInfo.Builder mli = MLDataFormats.ManagedLedgerInfo.newBuilder() - .addLedgerInfo(0, MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder() - .setLedgerId(0) - .setEntries(0) - .setTimestamp(System.currentTimeMillis())); + ManagedLedgerInfo mli = new ManagedLedgerInfo(); + mli.addLedgerInfo() + .setLedgerId(0) + .setEntries(0) + .setTimestamp(System.currentTimeMillis()); Stat stat = new Stat(path, version, createTimeMillis, createTimeMillis, false, false); return CompletableFuture.supplyAsync(() -> { try { @@ -89,14 +90,13 @@ private void setup() { } catch (InterruptedException e) { e.printStackTrace(); } - MLDataFormats.ManagedLedgerInfo managedLedgerInfo = mli.build(); - log.info("metadataStore.get({}) returned,managedLedgerInfo={},stat={}", path, managedLedgerInfo, + log.info("metadataStore.get({}) returned,managedLedgerInfo={},stat={}", path, mli, stat); - return Optional.of(new GetResult(managedLedgerInfo.toByteArray(), stat)); + return Optional.of(new GetResult(mli.toByteArray(), stat)); }); } else if (path.contains(ledgerName)) { // cursor - MLDataFormats.ManagedCursorInfo.Builder mci = MLDataFormats.ManagedCursorInfo.newBuilder() + ManagedCursorInfo mci = new ManagedCursorInfo() .setCursorsLedgerId(-1) .setMarkDeleteLedgerId(0) .setMarkDeleteLedgerId(-1); @@ -107,10 +107,9 @@ private void setup() { } catch (InterruptedException e) { e.printStackTrace(); } - MLDataFormats.ManagedCursorInfo managedCursorInfo = mci.build(); - log.info("metadataStore.get({}) returned:managedCursorInfo={},stat={}", path, managedCursorInfo, + log.info("metadataStore.get({}) returned:managedCursorInfo={},stat={}", path, mci, stat); - return Optional.of(new GetResult(managedCursorInfo.toByteArray(), stat)); + return Optional.of(new GetResult(mci.toByteArray(), stat)); }); } else { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java index 7eeac35da4787..567638ec1523c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java @@ -20,17 +20,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import com.google.protobuf.InvalidProtocolBufferException; -import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.UUID; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.MetadataCompressionConfig; import org.apache.bookkeeper.mledger.offload.OffloadUtils; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.common.api.proto.CompressionType; import org.testng.Assert; @@ -56,18 +53,17 @@ private Object[][] compressionTypeProvider() { }; } - private MLDataFormats.ManagedLedgerInfo.Builder generateManagedLedgerInfo(long ledgerId, int ledgerInfoNumber) { - List ledgerInfoList = new ArrayList<>(); + private ManagedLedgerInfo generateManagedLedgerInfo(long ledgerId, int ledgerInfoNumber) { + ManagedLedgerInfo managedLedgerInfo = new ManagedLedgerInfo(); for (int i = 0; i < ledgerInfoNumber; i++) { - MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder builder = - MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder(); - builder.setLedgerId(ledgerId); - builder.setEntries(RandomUtils.nextInt()); - builder.setSize(RandomUtils.nextLong()); - builder.setTimestamp(System.currentTimeMillis()); + LedgerInfo ledgerInfo = managedLedgerInfo.addLedgerInfo(); + ledgerInfo.setLedgerId(ledgerId); + ledgerInfo.setEntries(RandomUtils.nextInt()); + ledgerInfo.setSize(RandomUtils.nextLong()); + ledgerInfo.setTimestamp(System.currentTimeMillis()); UUID uuid = UUID.randomUUID(); - builder.getOffloadContextBuilder() + ledgerInfo.setOffloadContext() .setUidMsb(uuid.getMostSignificantBits()) .setUidLsb(uuid.getLeastSignificantBits()); Map offloadDriverMetadata = new HashMap<>(); @@ -76,24 +72,21 @@ private MLDataFormats.ManagedLedgerInfo.Builder generateManagedLedgerInfo(long l offloadDriverMetadata.put("serviceEndpoint", "https://s3.eu-west-1.amazonaws.com"); offloadDriverMetadata.put("region", "eu-west-1"); OffloadUtils.setOffloadDriverMetadata( - builder, + ledgerInfo, "aws-s3", offloadDriverMetadata ); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = builder.build(); - ledgerInfoList.add(ledgerInfo); ledgerId++; } - return MLDataFormats.ManagedLedgerInfo.newBuilder() - .addAllLedgerInfo(ledgerInfoList); + return managedLedgerInfo; } @Test(dataProvider = "compressionTypeProvider") - public void testEncodeAndDecode(String compressionType) throws IOException { + public void testEncodeAndDecode(String compressionType) throws Exception { long ledgerId = 10000; - MLDataFormats.ManagedLedgerInfo managedLedgerInfo = generateManagedLedgerInfo(ledgerId, 100).build(); + ManagedLedgerInfo managedLedgerInfo = generateManagedLedgerInfo(ledgerId, 100); MetaStoreImpl metaStore; try { @@ -105,7 +98,7 @@ public void testEncodeAndDecode(String compressionType) throws IOException { if ("INVALID_TYPE".equals(compressionType)) { Assert.assertEquals(e.getClass(), IllegalArgumentException.class); Assert.assertEquals( - "No enum constant org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType." + "No enum constant org.apache.bookkeeper.mledger.proto.CompressionType." + compressionType, e.getMessage()); return; } else { @@ -121,15 +114,15 @@ public void testEncodeAndDecode(String compressionType) throws IOException { } // parse compression data and unCompression data, check their results. - MLDataFormats.ManagedLedgerInfo info1 = metaStore.parseManagedLedgerInfo(compressionBytes); - MLDataFormats.ManagedLedgerInfo info2 = metaStore.parseManagedLedgerInfo(managedLedgerInfo.toByteArray()); + ManagedLedgerInfo info1 = metaStore.parseManagedLedgerInfo(compressionBytes); + ManagedLedgerInfo info2 = metaStore.parseManagedLedgerInfo(managedLedgerInfo.toByteArray()); Assert.assertEquals(info1, info2); } @Test - public void testParseEmptyData() throws InvalidProtocolBufferException { + public void testParseEmptyData() throws Exception { MetaStoreImpl metaStore = new MetaStoreImpl(null, null); - MLDataFormats.ManagedLedgerInfo managedLedgerInfo = metaStore.parseManagedLedgerInfo(new byte[0]); + ManagedLedgerInfo managedLedgerInfo = metaStore.parseManagedLedgerInfo(new byte[0]); Assert.assertEquals(managedLedgerInfo.toString(), ""); } @@ -139,14 +132,14 @@ public void testCompressionThreshold(String compressionType) { int compressThreshold = 512; // should not compress - MLDataFormats.ManagedLedgerInfo smallInfo = generateManagedLedgerInfo(ledgerId, 0).build(); + ManagedLedgerInfo smallInfo = generateManagedLedgerInfo(ledgerId, 0); assertTrue(smallInfo.getSerializedSize() < compressThreshold); // should compress - MLDataFormats.ManagedLedgerInfo bigInfo = generateManagedLedgerInfo(ledgerId, 1000).build(); + ManagedLedgerInfo bigInfo = generateManagedLedgerInfo(ledgerId, 1000); assertTrue(bigInfo.getSerializedSize() > compressThreshold); - MLDataFormats.ManagedLedgerInfo managedLedgerInfo = generateManagedLedgerInfo(ledgerId, 100).build(); + ManagedLedgerInfo managedLedgerInfo = generateManagedLedgerInfo(ledgerId, 100); MetaStoreImpl metaStore; try { @@ -160,7 +153,7 @@ public void testCompressionThreshold(String compressionType) { if ("INVALID_TYPE".equals(compressionType)) { Assert.assertEquals(e.getClass(), IllegalArgumentException.class); Assert.assertEquals( - "No enum constant org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType." + "No enum constant org.apache.bookkeeper.mledger.proto.CompressionType." + compressionType, e.getMessage()); return; } else { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java index 32f7bf6b77956..b06d4442ca7ab 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java @@ -135,9 +135,8 @@ import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.exception.ExceptionUtils; @@ -2075,12 +2074,12 @@ public void cursorReadsWithDiscardedEmptyLedgersStillListed() throws Exception { @Override public void operationComplete(ManagedLedgerInfo result, Stat version) { // Update the list - ManagedLedgerInfo.Builder info = ManagedLedgerInfo.newBuilder(result); + ManagedLedgerInfo info = new ManagedLedgerInfo().copyFrom(result); info.clearLedgerInfo(); - info.addLedgerInfo(LedgerInfo.newBuilder().setLedgerId(l1info.getLedgerId()).build()); - info.addLedgerInfo(l2info); + info.addLedgerInfo().setLedgerId(l1info.getLedgerId()); + info.addLedgerInfo().copyFrom(l2info); - store.asyncUpdateLedgerIds("my_test_ledger", info.build(), version, new MetaStoreCallback() { + store.asyncUpdateLedgerIds("my_test_ledger", info, version, new MetaStoreCallback() { @Override public void operationComplete(Void result, Stat version) { counter.countDown(); @@ -2587,16 +2586,17 @@ public void operationFailed(MetaStoreException e) { }); l1.await(); - ManagedLedgerInfo.Builder builder1 = ManagedLedgerInfo.newBuilder(); + ManagedLedgerInfo builder1 = new ManagedLedgerInfo(); // simulate test for old ledger with no timestampl - for (LedgerInfo info : storedMLInfo[0].getLedgerInfoList()) { - LedgerInfo noTimestamp = ManagedLedgerInfo.LedgerInfo.newBuilder().mergeFrom(info).clearTimestamp().build(); + for (int i = 0; i < storedMLInfo[0].getLedgerInfosCount(); i++) { + LedgerInfo info = storedMLInfo[0].getLedgerInfoAt(i); + LedgerInfo noTimestamp = new LedgerInfo().copyFrom(info); + noTimestamp.clearTimestamp(); assertFalse(noTimestamp.hasTimestamp(), "expected old version info with no timestamp"); - builder1.addLedgerInfo(noTimestamp); - + builder1.addLedgerInfo().copyFrom(noTimestamp); } - storedMLInfo[1] = builder1.build(); + storedMLInfo[1] = builder1; // test timestamp on new ledger @@ -3549,12 +3549,12 @@ public void testPropertiesForMeta() throws Exception { factory.open(mLName); MetaStore store = new MetaStoreImpl(metadataStore, executor); - ManagedLedgerInfo.Builder builder = ManagedLedgerInfo.newBuilder(); - builder.addProperties(MLDataFormats.KeyValue.newBuilder().setKey("key1").setValue("value1").build()); - builder.addProperties(MLDataFormats.KeyValue.newBuilder().setKey("key2").setValue("value2").build()); + ManagedLedgerInfo builder = new ManagedLedgerInfo(); + builder.addProperty().setKey("key1").setValue("value1"); + builder.addProperty().setKey("key2").setValue("value2"); CountDownLatch l2 = new CountDownLatch(1); - store.asyncUpdateLedgerIds(mLName, builder.build(), + store.asyncUpdateLedgerIds(mLName, builder, new Stat(mLName, 1, 0, 0, false, true), new MetaStoreCallback() { @Override @@ -4286,7 +4286,7 @@ public void testOffloadTaskCancelled() throws Exception { Awaitility.await().untilAsserted(() -> { CompletableFuture ledgerInfo = ledger.getLedgerInfo(ledgerId); - Assert.assertFalse(ledgerInfo.get(100, TimeUnit.MILLISECONDS).getOffloadContext().getComplete()); + Assert.assertFalse(ledgerInfo.get(100, TimeUnit.MILLISECONDS).getOffloadContext().isComplete()); }); } @@ -4561,7 +4561,7 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce // prepare the arguments for the offloadLoop method CompletableFuture future = new CompletableFuture<>(); Queue ledgersToOffload = new LinkedList<>(); - LedgerInfo ledgerInfo = LedgerInfo.getDefaultInstance().toBuilder().setLedgerId(1).setEntries(10).build(); + LedgerInfo ledgerInfo = new LedgerInfo().setLedgerId(1).setEntries(10); ledgersToOffload.add(ledgerInfo); Position firstUnoffloaded = PositionFactory.create(1, 0); Optional firstError = Optional.empty(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MetaStoreImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MetaStoreImplTest.java index 00ad494a5ca59..e6802906791e3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MetaStoreImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/MetaStoreImplTest.java @@ -31,9 +31,8 @@ import lombok.NoArgsConstructor; import org.apache.bookkeeper.mledger.ManagedLedgerException.MetaStoreException; import org.apache.bookkeeper.mledger.impl.MetaStore.MetaStoreCallback; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedCursorInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.metadata.api.MetadataCache; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -100,7 +99,7 @@ void readMalformedML() throws Exception { final CountDownLatch latch = new CountDownLatch(1); - store.getManagedLedgerInfo("my_test", false, new MetaStoreCallback() { + store.getManagedLedgerInfo("my_test", false, new MetaStoreCallback() { public void operationFailed(MetaStoreException e) { // Ok latch.countDown(); @@ -122,7 +121,7 @@ void readMalformedCursorNode() throws Exception { metadataStore.put("/managed-ledgers/my_test/c1", "non-valid".getBytes(), Optional.empty()).join(); final CountDownLatch latch = new CountDownLatch(1); - store.asyncGetCursorInfo("my_test", "c1", new MetaStoreCallback() { + store.asyncGetCursorInfo("my_test", "c1", new MetaStoreCallback() { public void operationFailed(MetaStoreException e) { // Ok @@ -147,7 +146,7 @@ void failInCreatingMLnode() throws Exception { op == FaultInjectionMetadataStore.OperationType.PUT ); - store.getManagedLedgerInfo("my_test", false, new MetaStoreCallback() { + store.getManagedLedgerInfo("my_test", false, new MetaStoreCallback() { public void operationFailed(MetaStoreException e) { promise.complete(null); } @@ -167,7 +166,7 @@ void updatingCursorNode() throws Exception { final CompletableFuture promise = new CompletableFuture<>(); - ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(1).build(); + ManagedCursorInfo info = new ManagedCursorInfo().setCursorsLedgerId(1); store.asyncUpdateCursorInfo("my_test", "c1", info, null, new MetaStoreCallback() { public void operationFailed(MetaStoreException e) { promise.completeExceptionally(e); @@ -180,7 +179,7 @@ public void operationComplete(Void result, Stat version) { && path.contains("my_test") && path.contains("c1") ); - ManagedCursorInfo info = ManagedCursorInfo.newBuilder().setCursorsLedgerId(2).build(); + ManagedCursorInfo info = new ManagedCursorInfo().setCursorsLedgerId(2); store.asyncUpdateCursorInfo("my_test", "c1", info, version, new MetaStoreCallback() { public void operationFailed(MetaStoreException e) { // ok diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java index 6a449a2b27cf1..41ed696f2c149 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java @@ -66,7 +66,7 @@ public void testEvictUnusedLedgers() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java index a1ef8ad0e2c6e..ad5a0741b1b59 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadLedgerDeleteTest.java @@ -38,7 +38,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.OffloadContext; import org.apache.bookkeeper.mledger.util.MockClock; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; @@ -177,7 +178,7 @@ public void testLaggedDelete() throws Exception { Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); Assert.assertTrue(bkc.getLedgers().contains(firstLedgerId)); @@ -198,7 +199,7 @@ public void testLaggedDelete() throws Exception { // ledger still exists in list Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -275,14 +276,14 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { CompletableFuture trimLedgerFuture = new CompletableFuture(); ml.internalTrimLedgers(false, trimLedgerFuture); trimLedgerFuture.join(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo1 = ml.getLedgerInfo(ledger1).get(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo2 = ml.getLedgerInfo(ledger2).get(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo3 = ml.getLedgerInfo(ledger3).get(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo4 = ml.getLedgerInfo(ledger4).get(); - Assert.assertTrue(ledgerInfo1.hasOffloadContext() && ledgerInfo1.getOffloadContext().getBookkeeperDeleted()); - Assert.assertTrue(ledgerInfo2.hasOffloadContext() && ledgerInfo2.getOffloadContext().getBookkeeperDeleted()); - Assert.assertTrue(ledgerInfo3.hasOffloadContext() && ledgerInfo3.getOffloadContext().getBookkeeperDeleted()); - Assert.assertFalse(ledgerInfo4.hasOffloadContext() || ledgerInfo4.getOffloadContext().getBookkeeperDeleted()); + LedgerInfo ledgerInfo1 = ml.getLedgerInfo(ledger1).get(); + LedgerInfo ledgerInfo2 = ml.getLedgerInfo(ledger2).get(); + LedgerInfo ledgerInfo3 = ml.getLedgerInfo(ledger3).get(); + LedgerInfo ledgerInfo4 = ml.getLedgerInfo(ledger4).get(); + Assert.assertTrue(ledgerInfo1.hasOffloadContext() && ledgerInfo1.getOffloadContext().isBookkeeperDeleted()); + Assert.assertTrue(ledgerInfo2.hasOffloadContext() && ledgerInfo2.getOffloadContext().isBookkeeperDeleted()); + Assert.assertTrue(ledgerInfo3.hasOffloadContext() && ledgerInfo3.getOffloadContext().isBookkeeperDeleted()); + Assert.assertFalse(ledgerInfo4.hasOffloadContext() || ledgerInfo4.getOffloadContext().isBookkeeperDeleted()); Awaitility.await().untilAsserted(() -> { try { @@ -358,14 +359,14 @@ public void testFileSystemOffloadDeletePath() throws Exception { Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); Assert.assertTrue(bkc.getLedgers().contains(firstLedgerId)); // ledger still exists in list Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -406,7 +407,7 @@ public void testLaggedDeleteRetentionSetLower() throws Exception { Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); Assert.assertTrue(bkc.getLedgers().contains(firstLedgerId)); @@ -454,7 +455,7 @@ public void testLaggedDeleteSlowConsumer() throws Exception { Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2); Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); Assert.assertTrue(bkc.getLedgers().contains(firstLedgerId)); @@ -476,7 +477,7 @@ public void testLaggedDeleteSlowConsumer() throws Exception { // ledger still exists in list Assert.assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); } @@ -494,11 +495,10 @@ public void isOffloadedNeedsDeleteTest() throws Exception { ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) factory.open("isOffloadedNeedsDeleteTest", config); - MLDataFormats.OffloadContext offloadContext = MLDataFormats.OffloadContext.newBuilder() + OffloadContext offloadContext = new OffloadContext() .setTimestamp(config.getClock().millis() - 1000) .setComplete(true) - .setBookkeeperDeleted(false) - .build(); + .setBookkeeperDeleted(false); boolean needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertFalse(needsDelete); @@ -515,19 +515,17 @@ public void isOffloadedNeedsDeleteTest() throws Exception { needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertFalse(needsDelete); - offloadContext = MLDataFormats.OffloadContext.newBuilder() + offloadContext = new OffloadContext() .setTimestamp(config.getClock().millis() - 1000) .setComplete(false) - .setBookkeeperDeleted(false) - .build(); + .setBookkeeperDeleted(false); needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertFalse(needsDelete); - offloadContext = MLDataFormats.OffloadContext.newBuilder() + offloadContext = new OffloadContext() .setTimestamp(config.getClock().millis() - 1000) .setComplete(true) - .setBookkeeperDeleted(true) - .build(); + .setBookkeeperDeleted(true); needsDelete = managedLedger.isOffloadedNeedsDelete(offloadContext, Optional.of(offloadPolicies)); Assert.assertFalse(needsDelete); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index 513667308383b..00ee5b7573916 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -60,7 +60,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.OffloadedLedgerHandle; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.util.MockClock; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; @@ -112,9 +112,9 @@ public void testOffloadRead(String offloadType, boolean deleteMl) throws Excepti ledger.offloadPrefix(ledger.getLastConfirmedEntry()); assertEquals(ledger.getLedgersInfoAsList().size(), 3); - Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); - Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete()); - Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete()); + Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); + Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().isComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().isComplete()); if (offloadTypeAppendable.equals(offloadType)) { config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader)); @@ -199,13 +199,13 @@ public void testBookkeeperFirstOffloadRead(String offloadType) throws Exception assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 2); + .filter(e -> e.getOffloadContext().isComplete()).count(), 2); LedgerInfo firstLedger = ledger.getLedgersInfoAsList().get(0); - Assert.assertTrue(firstLedger.getOffloadContext().getComplete()); + Assert.assertTrue(firstLedger.getOffloadContext().isComplete()); LedgerInfo secondLedger; secondLedger = ledger.getLedgersInfoAsList().get(1); - Assert.assertTrue(secondLedger.getOffloadContext().getComplete()); + Assert.assertTrue(secondLedger.getOffloadContext().isComplete()); UUID firstLedgerUUID = new UUID(firstLedger.getOffloadContext().getUidMsb(), firstLedger.getOffloadContext().getUidLsb()); @@ -232,8 +232,8 @@ public void testBookkeeperFirstOffloadRead(String offloadType) throws Exception // assert bk ledger is deleted assertEventuallyTrue(() -> !bkc.getLedgers().contains(firstLedger.getLedgerId())); assertEventuallyTrue(() -> !bkc.getLedgers().contains(secondLedger.getLedgerId())); - Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getBookkeeperDeleted()); - Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getBookkeeperDeleted()); + Assert.assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isBookkeeperDeleted()); + Assert.assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().isBookkeeperDeleted()); if (offloadTypeAppendable.equals(offloadType)) { config.setLedgerOffloader(new NonAppendableLedgerOffloader(offloader)); @@ -292,9 +292,9 @@ public void testSkipOffloadIfReadOnly() throws Exception { } assertEquals(ledger.getLedgersInfoAsList().size(), 3); - Assert.assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); - Assert.assertFalse(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete()); - Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().getComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(1).getOffloadContext().isComplete()); + Assert.assertFalse(ledger.getLedgersInfoAsList().get(2).getOffloadContext().isComplete()); // cleanup. ledger.delete(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java index 09a0522bab43b..38d7c17b9d49c 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixTest.java @@ -49,8 +49,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.OffloadContext; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; @@ -89,7 +89,7 @@ public void testNullOffloader() throws Exception { } assertEquals(ledger.getLedgersInfoAsList().size(), 5); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); try { ledger.offloadPrefix(p); fail("Should have thrown an exception"); @@ -98,7 +98,7 @@ public void testNullOffloader() throws Exception { } assertEquals(ledger.getLedgersInfoAsList().size(), 5); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); // add more entries to ensure we can update the ledger list for (; i < 55; i++) { @@ -107,7 +107,7 @@ public void testNullOffloader() throws Exception { } assertEquals(ledger.getLedgersInfoAsList().size(), 6); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); } @Test @@ -132,7 +132,7 @@ public void testOffload() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -170,7 +170,7 @@ public void testOffloadFenced() throws Exception { // the offloader actually wrote the data on the storage assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -214,7 +214,7 @@ public void testPositionOutOfRange() throws Exception { } assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); assertEquals(offloader.offloadedLedgers().size(), 0); } @@ -247,9 +247,9 @@ public void testPositionOnEdgeOfLedger() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(offloader.offloadedLedgers().size(), 1); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); assertEquals(firstUnoffloaded.getLedgerId(), ledger.getLedgersInfoAsList().get(1).getLedgerId()); assertEquals(firstUnoffloaded.getEntryId(), 0); @@ -259,10 +259,10 @@ public void testPositionOnEdgeOfLedger() throws Exception { assertEquals(offloader.offloadedLedgers().size(), 2); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(1).getLedgerId())); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); - assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().getComplete()); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); + assertTrue(ledger.getLedgersInfoAsList().get(1).getOffloadContext().isComplete()); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 2); + .filter(e -> e.getOffloadContext().isComplete()).count(), 2); assertEquals(firstUnoffloaded2.getLedgerId(), ledger.getLedgersInfoAsList().get(2).getLedgerId()); } @@ -299,9 +299,9 @@ public void testPositionOnLastEmptyLedger() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(offloader.offloadedLedgers().size(), 1); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); assertEquals(firstUnoffloaded.getLedgerId(), ledger.getLedgersInfoAsList().get(1).getLedgerId()); assertEquals(firstUnoffloaded.getEntryId(), 0); } @@ -347,7 +347,7 @@ public CompletableFuture offload(ReadHandle ledger, cursor.markDelete(startOfSecondLedger, new HashMap<>()); assertEventuallyTrue(() -> ledger.getLedgersInfoAsList().size() == 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); // complete offloading blocker.complete(null); @@ -355,8 +355,8 @@ public CompletableFuture offload(ReadHandle ledger, assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); assertEquals(offloader.offloadedLedgers().size(), 1); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); } @@ -413,7 +413,7 @@ public CompletableFuture offload(ReadHandle ledger, assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getLedgerId() == trimmedLedger).count(), 0); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); // complete offloading blocker.complete(trimmedLedger); @@ -421,8 +421,8 @@ public CompletableFuture offload(ReadHandle ledger, assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); assertEquals(offloader.offloadedLedgers().size(), 1); assertTrue(offloader.offloadedLedgers().contains(ledger.getLedgersInfoAsList().get(0).getLedgerId())); } @@ -455,7 +455,7 @@ public void testOffloadClosedManagedLedger() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); assertEquals(offloader.offloadedLedgers().size(), 0); } @@ -481,7 +481,7 @@ public void testOffloadSamePositionTwice() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -489,7 +489,7 @@ public void testOffloadSamePositionTwice() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 3); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); @@ -524,12 +524,12 @@ public void offloadThreeOneFails(int failIndex) throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 4); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 2); - assertFalse(ledger.getLedgersInfoAsList().get(failIndex).getOffloadContext().getComplete()); + .filter(e -> e.getOffloadContext().isComplete()).count(), 2); + assertFalse(ledger.getLedgersInfoAsList().get(failIndex).getOffloadContext().isComplete()); } @Test @@ -632,7 +632,7 @@ public CompletableFuture deleteOffloaded(long ledgerId, UUID uuid, ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidLsb()); assertEquals(failedOffloads.stream().findFirst().get(), Pair.of(expectedFailedLedger, expectedFailedUUID)); - assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + assertFalse(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); // try offload again ledger.offloadPrefix(ledger.getLastConfirmedEntry()); @@ -644,7 +644,7 @@ public CompletableFuture deleteOffloaded(long ledgerId, UUID uuid, UUID successUUID = new UUID(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidMsb(), ledger.getLedgersInfoAsList().get(0).getOffloadContext().getUidLsb()); assertNotEquals(expectedFailedUUID, successUUID); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); } @Test @@ -674,8 +674,8 @@ public void testOffloadDelete() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); long firstLedger = ledger.getLedgersInfoAsList().get(0).getLedgerId(); long secondLedger = ledger.getLedgersInfoAsList().get(1).getLedgerId(); @@ -709,8 +709,8 @@ public void testOffloadDeleteClosedLedger() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 1); - assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().getComplete()); + .filter(e -> e.getOffloadContext().isComplete()).count(), 1); + assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().isComplete()); Set offloadedledgers = Sets.newHashSet(offloader.offloadedLedgers()); assertTrue(offloadedledgers.size() > 0); @@ -784,7 +784,7 @@ public CompletableFuture offload(ReadHandle ledger, assertEquals(ledger.getLedgersInfoAsList().size(), 2); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()).count(), 0); + .filter(e -> e.getOffloadContext().isComplete()).count(), 0); assertEquals(ledger.getLedgersInfoAsList().stream() .filter(e -> e.getOffloadContext().hasUidMsb()).count(), 1); assertTrue(ledger.getLedgersInfoAsList().get(0).getOffloadContext().hasUidMsb()); @@ -827,7 +827,7 @@ public void testDontOffloadEmpty() throws Exception { ledgersField.setAccessible(true); Map ledgers = (Map) ledgersField.get(ledger); ledgers.put(secondLedgerId, - ledgers.get(secondLedgerId).toBuilder().setEntries(0).setSize(0).build()); + new LedgerInfo().copyFrom(ledgers.get(secondLedgerId)).setEntries(0).setSize(0)); Position firstUnoffloaded = ledger.offloadPrefix(ledger.getLastConfirmedEntry()); assertEquals(firstUnoffloaded.getLedgerId(), fourthLedgerId); @@ -835,7 +835,7 @@ public void testDontOffloadEmpty() throws Exception { assertEquals(ledger.getLedgersInfoAsList().size(), 4); assertEquals(ledger.getLedgersInfoAsList().stream() - .filter(e -> e.getOffloadContext().getComplete()) + .filter(e -> e.getOffloadContext().isComplete()) .map(e -> e.getLedgerId()).collect(Collectors.toSet()), offloader.offloadedLedgers()); assertEquals(offloader.offloadedLedgers(), Set.of(firstLedgerId, thirdLedgerId)); @@ -1353,9 +1353,9 @@ public void testFailByZk() throws Exception { } final LedgerInfo ledgerInfo = ledger.getLedgersInfoAsList().get(0); - final MLDataFormats.OffloadContext offloadContext = ledgerInfo.getOffloadContext(); + final OffloadContext offloadContext = ledgerInfo.getOffloadContext(); //should not set complete when - assertEquals(offloadContext.getComplete(), false); + assertEquals(offloadContext.isComplete(), false); } static class ErroringMockLedgerOffloader extends MockLedgerOffloader { diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java index fc60c3765a2eb..26b3d8ee3a264 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionTest.java @@ -22,7 +22,7 @@ import static org.testng.Assert.assertNotEquals; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; +import org.apache.bookkeeper.mledger.proto.PositionInfo; import org.testng.annotations.Test; public class PositionTest { @@ -75,8 +75,9 @@ public void comparisons() { public void hashes() throws Exception { Position p1 = PositionFactory.create(5, 15); PositionInfo positionInfo = - PositionInfo.newBuilder().setLedgerId(p1.getLedgerId()).setEntryId(p1.getEntryId()).build(); - PositionInfo parsed = PositionInfo.parseFrom(positionInfo.toByteArray()); + new PositionInfo().setLedgerId(p1.getLedgerId()).setEntryId(p1.getEntryId()); + PositionInfo parsed = new PositionInfo(); + parsed.parseFrom(positionInfo.toByteArray()); Position p2 = PositionFactory.create(parsed.getLedgerId(), parsed.getEntryId()); assertEquals(p2.getLedgerId(), 5); assertEquals(p2.getEntryId(), 15); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/offload/OffloadUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/offload/OffloadUtilsTest.java index ee207740dfc29..990c186f025bf 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/offload/OffloadUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/offload/OffloadUtilsTest.java @@ -20,7 +20,8 @@ import java.util.HashMap; import java.util.Map; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.OffloadDriverMetadata; import org.testng.Assert; import org.testng.annotations.Test; @@ -28,9 +29,8 @@ public class OffloadUtilsTest { @Test void testOffloadMetadataShouldClearBeforeSet() { - MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder builder = - MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder(); - builder.setLedgerId(1L); + LedgerInfo ledgerInfo = new LedgerInfo(); + ledgerInfo.setLedgerId(1L); Map map = new HashMap<>(); map.put("key1", "value1"); @@ -38,18 +38,18 @@ void testOffloadMetadataShouldClearBeforeSet() { //only one copy of the offload metadata information is stored in metadata store, // and the original properties need to be cleared during offload - OffloadUtils.setOffloadDriverMetadata(builder, "offload", map); + OffloadUtils.setOffloadDriverMetadata(ledgerInfo, "offload", map); - OffloadUtils.setOffloadDriverMetadata(builder, "offload", map); + OffloadUtils.setOffloadDriverMetadata(ledgerInfo, "offload", map); - MLDataFormats.OffloadDriverMetadata offloadDriverMetadata = - builder.build().getOffloadContext().getDriverMetadata(); - Assert.assertEquals(offloadDriverMetadata.getPropertiesList().size(), 2); + OffloadDriverMetadata offloadDriverMetadata = + ledgerInfo.getOffloadContext().getDriverMetadata(); + Assert.assertEquals(offloadDriverMetadata.getPropertiesCount(), 2); - Assert.assertEquals(offloadDriverMetadata.getProperties(0).getKey(), "key1"); - Assert.assertEquals(offloadDriverMetadata.getProperties(1).getKey(), "key2"); - Assert.assertEquals(offloadDriverMetadata.getProperties(0).getValue(), "value1"); - Assert.assertEquals(offloadDriverMetadata.getProperties(1).getValue(), "value2"); + Assert.assertEquals(offloadDriverMetadata.getPropertyAt(0).getKey(), "key1"); + Assert.assertEquals(offloadDriverMetadata.getPropertyAt(1).getKey(), "key2"); + Assert.assertEquals(offloadDriverMetadata.getPropertyAt(0).getValue(), "value1"); + Assert.assertEquals(offloadDriverMetadata.getPropertyAt(1).getValue(), "value2"); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 5e811ff1dce4c..f04570c5441cd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -30,7 +30,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -227,7 +227,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo log.debug("[{}] slowest consumer mark delete position is [{}], read position is [{}]", slowestConsumer.getName(), oldestPosition, slowestConsumer.getReadPosition()); } - MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = + ManagedLedgerInfo.LedgerInfo ledgerInfo = mLedger.getLedgerInfo(oldestPosition.getLedgerId()).get(); if (ledgerInfo == null) { long ledgerId = mLedger.getLedgersInfo().ceilingKey(oldestPosition.getLedgerId() + 1); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index 658e1024f9355..929261783dfff 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -116,11 +116,11 @@ private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTL } ManagedLedger managedLedger = cursor.getManagedLedger(); Position deletedPosition = cursor.getMarkDeletedPosition(); - SortedMap ledgerInfoSortedMap = + SortedMap ledgerInfoSortedMap = managedLedger.getLedgersInfo().subMap(deletedPosition.getLedgerId(), true, managedLedger.getLedgersInfo().lastKey(), true); - MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; - for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { + ManagedLedgerInfo.LedgerInfo info = null; + for (ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { if (!ledgerInfo.hasTimestamp() || ledgerInfo.getTimestamp() == 0L || !MessageImpl.isEntryExpired(messageTTLInSeconds, ledgerInfo.getTimestamp())) { break; @@ -144,7 +144,7 @@ public boolean expireMessages(Position messagePosition) { ManagedLedger managedLedger = cursor.getManagedLedger(); if (managedLedger instanceof ManagedLedgerImpl ml) { // Confirm the position is valid. - Optional ledgerInfoOptional = + Optional ledgerInfoOptional = ml.getOptionalLedgerInfo(messagePosition.getLedgerId()); if (ledgerInfoOptional.isPresent()) { if (messagePosition.getEntryId() >= 0 diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java index 991e6a060295b..4e867e298ef15 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageFinder.java @@ -27,7 +27,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.util.Codec; 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 dfbb2725db44d..cb7157314c080 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 @@ -88,6 +88,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.util.Futures; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; @@ -3953,7 +3954,7 @@ EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } - org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo + ManagedLedgerInfo.LedgerInfo markDeletePositionLedgerInfo = ledger.getLedgerInfo(markDeletePosition.getLedgerId()).get(); // If markDeletePositionLedgerInfo is null (ledger no longer exists due to retention/cleanup), @@ -3964,7 +3965,7 @@ EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( markDeletePosition = nextValidPosition; } - org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo positionToCheckLedgerInfo = + ManagedLedgerInfo.LedgerInfo positionToCheckLedgerInfo = markDeletePositionLedgerInfo; // if the mark-delete position is the last entry it means all entries for diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/OffloadIndexBlockV2Builder.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/OffloadIndexBlockV2Builder.java index 8373b7eb54125..d80e43efecb17 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/OffloadIndexBlockV2Builder.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/OffloadIndexBlockV2Builder.java @@ -23,7 +23,7 @@ import org.apache.bookkeeper.common.annotation.InterfaceAudience.LimitedPrivate; import org.apache.bookkeeper.common.annotation.InterfaceStability.Unstable; import org.apache.bookkeeper.mledger.offload.jcloud.impl.OffloadIndexBlockV2BuilderImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; /** * Interface for builder of index block used for offload a ledger to long term storage. diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java index 33bbc49ee2223..fa0e84efe383f 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloader.java @@ -62,7 +62,9 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2Builder; import org.apache.bookkeeper.mledger.offload.jcloud.provider.BlobStoreLocation; import org.apache.bookkeeper.mledger.offload.jcloud.provider.TieredStorageConfiguration; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.OffloadContext; +import org.apache.bookkeeper.mledger.proto.OffloadSegment; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; @@ -452,17 +454,16 @@ private void buildBlockAndUpload(int blockSize, List entries, long blockL streamingParts.add(blobStore.uploadMultipartPart(streamingMpu, partId, partPayload)); streamingIndexBuilder.withDataBlockHeaderLength(StreamingDataBlockHeaderImpl.getDataStartOffset()); streamingIndexBuilder.addBlock(blockLedgerId, beginEntryId, partId, blockSize); - final MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = ml.getLedgerInfo(blockLedgerId).get(); - final MLDataFormats.ManagedLedgerInfo.LedgerInfo.Builder ledgerInfoBuilder = - MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder(); + final LedgerInfo ledgerInfo = ml.getLedgerInfo(blockLedgerId).get(); + final LedgerInfo ledgerInfoCopy = new LedgerInfo(); if (ledgerInfo != null) { - ledgerInfoBuilder.mergeFrom(ledgerInfo); + ledgerInfoCopy.copyFrom(ledgerInfo); } - if (ledgerInfoBuilder.getEntries() == 0) { + if (!ledgerInfoCopy.hasEntries() || ledgerInfoCopy.getEntries() == 0) { //ledger unclosed, use last entry id of the block - ledgerInfoBuilder.setEntries(payloadStream.getEndEntryId() + 1); + ledgerInfoCopy.setEntries(payloadStream.getEndEntryId() + 1); } - streamingIndexBuilder.addLedgerMeta(blockLedgerId, ledgerInfoBuilder.build()); + streamingIndexBuilder.addLedgerMeta(blockLedgerId, ledgerInfoCopy); log.debug("UploadMultipartPart. container: {}, blobName: {}, partId: {}, mpu: {}", config.getBucket(), streamingDataBlockKey, partId, streamingMpu.id()); } catch (Throwable e) { @@ -584,21 +585,21 @@ public CompletableFuture readOffloaded(long ledgerId, UUID uid, } @Override - public CompletableFuture readOffloaded(long ledgerId, MLDataFormats.OffloadContext ledgerContext, + public CompletableFuture readOffloaded(long ledgerId, OffloadContext ledgerContext, Map offloadDriverMetadata) { BlobStoreLocation bsKey = getBlobStoreLocation(offloadDriverMetadata); String readBucket = bsKey.getBucket(); CompletableFuture promise = new CompletableFuture<>(); - final List offloadSegmentList = ledgerContext.getOffloadSegmentList(); List keys = Lists.newLinkedList(); List indexKeys = Lists.newLinkedList(); - offloadSegmentList.forEach(seg -> { + for (int i = 0; i < ledgerContext.getOffloadSegmentsCount(); i++) { + OffloadSegment seg = ledgerContext.getOffloadSegmentAt(i); final UUID uuid = new UUID(seg.getUidMsb(), seg.getUidLsb()); final String key = uuid.toString(); final String indexKey = DataBlockUtils.indexBlockOffloadKey(uuid); keys.add(key); indexKeys.add(indexKey); - }); + } readExecutor.chooseThread(ledgerId).execute(() -> { try { diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2BuilderImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2BuilderImpl.java index d5761fa0e4f72..9756f77eda817 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2BuilderImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2BuilderImpl.java @@ -34,7 +34,7 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2Builder; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; /** * Interface for builder of index block used for offload a ledger to long term storage. diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2Impl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2Impl.java index 93ae53abce392..dd096c8028e6c 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2Impl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexBlockV2Impl.java @@ -37,7 +37,7 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock.IndexInputStream; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexEntry; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.net.BookieId; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.slf4j.Logger; @@ -217,7 +217,9 @@ public IndexInputStream toStream() throws IOException { } private static LedgerInfo parseLedgerInfo(byte[] bytes) throws IOException { - return LedgerInfo.newBuilder().mergeFrom(bytes).build(); + LedgerInfo ledgerInfo = new LedgerInfo(); + ledgerInfo.parseFrom(bytes); + return ledgerInfo; } private OffloadIndexBlockV2 fromStream(DataInputStream dis) throws IOException { diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java index 85889cde90a57..e887702cd78b4 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java @@ -43,8 +43,7 @@ import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.offload.jcloud.provider.JCloudBlobStoreProvider; import org.apache.bookkeeper.mledger.offload.jcloud.provider.TieredStorageConfiguration; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.OffloadContext; +import org.apache.bookkeeper.mledger.proto.OffloadContext; import org.jclouds.blobstore.BlobStore; import org.mockito.Mockito; import org.slf4j.Logger; @@ -159,15 +158,14 @@ public void testReadAndWrite() throws Exception { final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); assertEquals(offloadResult.endLedger, 0); assertEquals(offloadResult.endEntry, 9); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); @Cleanup final LedgerEntries ledgerEntries = readHandle.readAsync(0, 9).get(); @@ -220,15 +218,14 @@ public void testReadAndWriteAcrossLedger() throws Exception { final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); assertEquals(offloadResult.endLedger, 1); assertEquals(offloadResult.endEntry, 9); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); @Cleanup final LedgerEntries ledgerEntries = readHandle.readAsync(0, 9).get(); @@ -240,7 +237,7 @@ public void testReadAndWriteAcrossLedger() throws Exception { } @Cleanup - final ReadHandle readHandle2 = offloader.readOffloaded(1, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle2 = offloader.readOffloaded(1, context, driverMeta).get(); @Cleanup final LedgerEntries ledgerEntries2 = readHandle2.readAsync(0, 9).get(); @@ -308,20 +305,18 @@ public void testReadAndWriteAcrossSegment() throws Exception { assertEquals(offloadResult2.endLedger, 0); assertEquals(offloadResult2.endEntry, 19); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()).addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid2.getLeastSignificantBits()) - .setUidMsb(uuid2.getMostSignificantBits()) - .setComplete(true).setEndEntryId(19).build() - ); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); + context.addOffloadSegment() + .setUidLsb(uuid2.getLeastSignificantBits()) + .setUidMsb(uuid2.getMostSignificantBits()) + .setComplete(true).setEndEntryId(19); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); @Cleanup final LedgerEntries ledgerEntries = readHandle.readAsync(0, 19).get(); @@ -388,20 +383,18 @@ public void testRandomRead() throws Exception { assertEquals(offloadResult2.endLedger, 0); assertEquals(offloadResult2.endEntry, 19); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()).addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid2.getLeastSignificantBits()) - .setUidMsb(uuid2.getMostSignificantBits()) - .setComplete(true).setEndEntryId(19).build() - ); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); + context.addOffloadSegment() + .setUidLsb(uuid2.getLeastSignificantBits()) + .setUidMsb(uuid2.getMostSignificantBits()) + .setComplete(true).setEndEntryId(19); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); for (int i = 0; i <= 19; i++) { Random seed = new Random(0); @@ -456,15 +449,14 @@ public void testInvalidEntryIds() throws Exception { final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); assertEquals(offloadResult.endLedger, 0); assertEquals(offloadResult.endEntry, 9); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); try { readHandle.read(-1, -1); Assert.fail("Shouldn't be able to read anything"); @@ -511,15 +503,14 @@ public void testReadNotExistLedger() throws Exception { final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); assertEquals(offloadResult.endLedger, 0); assertEquals(offloadResult.endEntry, 9); - final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); - contextBuilder.addOffloadSegment( - MLDataFormats.OffloadSegment.newBuilder() - .setUidLsb(uuid.getLeastSignificantBits()) - .setUidMsb(uuid.getMostSignificantBits()) - .setComplete(true).setEndEntryId(9).build()); + final OffloadContext context = new OffloadContext(); + context.addOffloadSegment() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9); @Cleanup - final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + final ReadHandle readHandle = offloader.readOffloaded(0, context, driverMeta).get(); // delete blob(ledger) blobStore.removeBlob(BUCKET, uuid.toString()); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java index d4f07ed9f9018..91af35cd621f8 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionBound; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; @@ -359,13 +359,13 @@ public ManagedLedgerInterceptor getManagedLedgerInterceptor() { @Override public CompletableFuture getLedgerInfo(long ledgerId) { - final LedgerInfo build = LedgerInfo.newBuilder().setLedgerId(ledgerId).setSize(100).setEntries(20).build(); + final LedgerInfo build = new LedgerInfo().setLedgerId(ledgerId).setSize(100).setEntries(20); return CompletableFuture.completedFuture(build); } @Override public Optional getOptionalLedgerInfo(long ledgerId) { - final LedgerInfo build = LedgerInfo.newBuilder().setLedgerId(ledgerId).setSize(100).setEntries(20).build(); + final LedgerInfo build = new LedgerInfo().setLedgerId(ledgerId).setSize(100).setEntries(20); return Optional.of(build); } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexTest.java index 1f409598f3209..afbf74dc4f70a 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexTest.java @@ -38,7 +38,7 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexEntry; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.testng.annotations.Test; @@ -109,7 +109,7 @@ public static LedgerInfo createLedgerInfo(long id) throws Exception { metadataCustom.put("key1", "value1".getBytes(UTF_8)); metadataCustom.put("key7", "value7".getBytes(UTF_8)); - return LedgerInfo.newBuilder().setLedgerId(id).setEntries(5001).setSize(10000).build(); + return new LedgerInfo().setLedgerId(id).setEntries(5001).setSize(10000); } // prepare metadata, then use builder to build a OffloadIndexBlockImpl diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexV2Test.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexV2Test.java index 9ec4585f14a66..ae1b20962f54f 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexV2Test.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/OffloadIndexV2Test.java @@ -32,7 +32,7 @@ import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2Builder; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexEntry; import org.apache.bookkeeper.mledger.offload.jcloud.impl.OffloadIndexBlockV2Impl.CompatibleMetadata; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.testng.annotations.Test; @Slf4j From 6425ed582c96f005b29eb176738be76a965952c5 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 11:15:29 -0700 Subject: [PATCH 2/9] [improve][ml] Fix checkstyle line length violation in EntryCountEstimatorTest --- .../bookkeeper/mledger/impl/EntryCountEstimatorTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java index 28f69709a31af..e0a08b0f3a439 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCountEstimatorTest.java @@ -236,7 +236,8 @@ public void testWithMultipleEmptyLedgers() { long secondLastLedgerId = ledgersInfo.lowerKey(lastLedgerId); ManagedLedgerInfo.LedgerInfo secondLastLedgerInfo = ledgersInfo.get(secondLastLedgerId); // make the second last ledger empty - ledgersInfo.put(secondLastLedgerId, new ManagedLedgerInfo.LedgerInfo().copyFrom(secondLastLedgerInfo).setEntries(0).setSize(0)); + ledgersInfo.put(secondLastLedgerId, + new ManagedLedgerInfo.LedgerInfo().copyFrom(secondLastLedgerInfo).setEntries(0).setSize(0)); lastLedgerTotalEntries = 0; lastLedgerTotalSize = 0; long expectedEntries = 50; From 10a2a82874b0140aacae62aa879ddbdcd0660976 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 11:53:34 -0700 Subject: [PATCH 3/9] [improve][ml] Fix broker test files to use LightProto API for managed-ledger proto types Update broker test files that referenced the removed MLDataFormats outer class wrapper. Replace MLDataFormats.ManagedLedgerInfo with ManagedLedgerInfo and convert LedgerInfo.newBuilder().build() to new LedgerInfo() constructor pattern. --- ...CustomizedManagedLedgerStorageForTest.java | 8 +- ...ersistentMessageExpiryMonitorMockTest.java | 11 ++- .../broker/service/BrokerBkEnsemblesTest.java | 2 +- .../service/BrokerBookieIsolationTest.java | 2 +- .../service/BrokerEntryMetadataE2ETest.java | 2 +- .../service/ManagedLedgerCompressionTest.java | 12 +-- .../service/PersistentMessageFinderTest.java | 80 +++++++++---------- .../broker/service/PersistentTopicTest.java | 2 +- .../broker/service/SubscriptionSeekTest.java | 4 +- .../SegmentAbortedTxnProcessorTest.java | 6 +- .../TopicTransactionBufferRecoverTest.java | 6 +- 11 files changed, 67 insertions(+), 68 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/CustomizedManagedLedgerStorageForTest.java b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/CustomizedManagedLedgerStorageForTest.java index 56ff0ba65924d..bd42eaf5678e8 100644 --- a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/CustomizedManagedLedgerStorageForTest.java +++ b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/CustomizedManagedLedgerStorageForTest.java @@ -49,7 +49,7 @@ import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.broker.BookKeeperClientFactory; import org.apache.pulsar.broker.ManagedLedgerClientFactory; import org.apache.pulsar.broker.ServiceConfiguration; @@ -616,12 +616,12 @@ public ManagedLedgerInterceptor getManagedLedgerInterceptor() { } @Override - public CompletableFuture getLedgerInfo(long ledgerId) { + public CompletableFuture getLedgerInfo(long ledgerId) { return delegate.getLedgerInfo(ledgerId); } @Override - public Optional getOptionalLedgerInfo(long ledgerId) { + public Optional getOptionalLedgerInfo(long ledgerId) { return delegate.getOptionalLedgerInfo(ledgerId); } @@ -652,7 +652,7 @@ public void asyncReadEntry(Position position, AsyncCallbacks.ReadEntryCallback c } @Override - public NavigableMap getLedgersInfo() { + public NavigableMap getLedgersInfo() { return delegate.getLedgersInfo(); } diff --git a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/PersistentMessageExpiryMonitorMockTest.java b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/PersistentMessageExpiryMonitorMockTest.java index 3b969255f3a9a..fe491ce6adb5e 100644 --- a/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/PersistentMessageExpiryMonitorMockTest.java +++ b/pulsar-broker/src/test/java/org/apache/bookkeeper/mledger/impl/PersistentMessageExpiryMonitorMockTest.java @@ -33,7 +33,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; @@ -99,13 +99,12 @@ public void testExpireMessagesWithMarkDeleteShortCircuitResetsExpirationCheckInP ); // Setup ledger info with expired ledger - NavigableMap ledgerInfo = new TreeMap<>(); - MLDataFormats.ManagedLedgerInfo.LedgerInfo expiredLedger = - MLDataFormats.ManagedLedgerInfo.LedgerInfo.newBuilder() + NavigableMap ledgerInfo = new TreeMap<>(); + ManagedLedgerInfo.LedgerInfo expiredLedger = + new ManagedLedgerInfo.LedgerInfo() .setLedgerId(2) .setEntries(60) - .setTimestamp(System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(10)) // 10 seconds old - .build(); + .setTimestamp(System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(10)); // 10 seconds old ledgerInfo.put(2L, expiredLedger); when(mockManagedLedger.getLedgersInfo()).thenReturn(ledgerInfo); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTest.java index a7d73234989b4..646da7d1fb438 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTest.java @@ -49,7 +49,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.util.StringUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentTopic; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java index 7bef444e7f8e5..f374ab5dec7e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBookieIsolationTest.java @@ -44,7 +44,7 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.versioning.Versioned; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java index 3bfdb3ece4c2e..5118bcbf55654 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java @@ -18,7 +18,7 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.ArrayList; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerCompressionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerCompressionTest.java index fb60ef97320fb..d204424a0d5dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerCompressionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ManagedLedgerCompressionTest.java @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.CompressionType; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -40,7 +40,7 @@ public class ManagedLedgerCompressionTest extends BrokerTestBase { @BeforeClass @Override protected void setup() throws Exception { - conf.setManagedLedgerInfoCompressionType(MLDataFormats.CompressionType.NONE.name()); + conf.setManagedLedgerInfoCompressionType(CompressionType.NONE.name()); super.baseSetup(); } @@ -67,12 +67,12 @@ public void testRestartBrokerEnableManagedLedgerInfoCompression() throws Excepti produceAndConsume(producer, consumer, messageCnt); stopBroker(); - conf.setManagedLedgerInfoCompressionType(MLDataFormats.CompressionType.ZSTD.name()); + conf.setManagedLedgerInfoCompressionType(CompressionType.ZSTD.name()); startBroker(); produceAndConsume(producer, consumer, messageCnt); stopBroker(); - conf.setManagedLedgerInfoCompressionType(MLDataFormats.CompressionType.LZ4.name()); + conf.setManagedLedgerInfoCompressionType(CompressionType.LZ4.name()); startBroker(); produceAndConsume(producer, consumer, messageCnt); @@ -85,11 +85,11 @@ public void testRestartBrokerEnableManagedLedgerInfoCompression() throws Excepti Throwable e = rte.getCause(); Assert.assertEquals(e.getCause().getClass(), IllegalArgumentException.class); Assert.assertEquals( - "No enum constant org.apache.bookkeeper.mledger.proto.MLDataFormats.CompressionType.INVALID", + "No enum constant org.apache.bookkeeper.mledger.proto.CompressionType.INVALID", e.getCause().getMessage()); } - conf.setManagedLedgerInfoCompressionType(MLDataFormats.CompressionType.NONE.name()); + conf.setManagedLedgerInfoCompressionType(CompressionType.NONE.name()); startBroker(); produceAndConsume(producer, consumer, messageCnt); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java index 5707a6cca60b8..d8dc64d049023 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentMessageFinderTest.java @@ -56,7 +56,7 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -878,8 +878,8 @@ public void testGetFindPositionRange_EmptyLedgerInfos() { @Test public void testGetFindPositionRange_AllTimestampsLessThanTarget() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(1500)); Position lastConfirmedEntry = PositionFactory.create(2, 9); long targetTimestamp = 2000; @@ -895,9 +895,9 @@ public void testGetFindPositionRange_AllTimestampsLessThanTarget() { @Test public void testGetFindPositionRange_LastTimestampIsZero() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(1500)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(3, 5); long targetTimestamp = 2000; @@ -913,9 +913,9 @@ public void testGetFindPositionRange_LastTimestampIsZero() { @Test public void testGetFindPositionRange_LastTimestampIsZeroWithNoEntries() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(1500).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(1500)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(2, 9); long targetTimestamp = 2000; @@ -932,8 +932,8 @@ public void testGetFindPositionRange_LastTimestampIsZeroWithNoEntries() { @Test public void testGetFindPositionRange_AllTimestampsGreaterThanTarget() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(3000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(3000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(4000)); Position lastConfirmedEntry = PositionFactory.create(2, 9); long targetTimestamp = 2000; @@ -949,9 +949,9 @@ public void testGetFindPositionRange_AllTimestampsGreaterThanTarget() { @Test public void testGetFindPositionRange_MixedTimestamps() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(3000)); Position lastConfirmedEntry = PositionFactory.create(3, 9); long targetTimestamp = 2500; @@ -968,10 +968,10 @@ public void testGetFindPositionRange_MixedTimestamps() { @Test public void testGetFindPositionRange_TimestampAtBoundary() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(3000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(4).setEntries(10).setTimestamp(4000)); Position lastConfirmedEntry = PositionFactory.create(4, 9); long targetTimestamp = 3000; @@ -990,11 +990,11 @@ public void testGetFindPositionRange_TimestampAtBoundary() { @Test public void testGetFindPositionRange_ClockSkew() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(2010).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(2010)); + ledgerInfos.add(new LedgerInfo().setLedgerId(4).setEntries(10).setTimestamp(4000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(5).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(5, 5); long targetTimestamp = 2009; @@ -1011,11 +1011,11 @@ public void testGetFindPositionRange_ClockSkew() { @Test public void testGetFindPositionRange_ClockSkewCase2() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(3000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(4).setEntries(10).setTimestamp(4000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(5).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(5, 5); long targetTimestamp = 2995; @@ -1032,11 +1032,11 @@ public void testGetFindPositionRange_ClockSkewCase2() { @Test public void testGetFindPositionRange_ClockSkewCase3() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(3000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(3000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(4).setEntries(10).setTimestamp(4000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(5).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(5, 5); long targetTimestamp = 3005; @@ -1053,11 +1053,11 @@ public void testGetFindPositionRange_ClockSkewCase3() { @Test public void testGetFindPositionRange_FeatureDisabledWithNegativeClockSkew() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(2).setEntries(10).setTimestamp(2000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(3).setEntries(10).setTimestamp(2010).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(4).setEntries(10).setTimestamp(4000).build()); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(5).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(2).setEntries(10).setTimestamp(2000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(3).setEntries(10).setTimestamp(2010)); + ledgerInfos.add(new LedgerInfo().setLedgerId(4).setEntries(10).setTimestamp(4000)); + ledgerInfos.add(new LedgerInfo().setLedgerId(5).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(5, 5); long targetTimestamp = 2009; @@ -1072,7 +1072,7 @@ public void testGetFindPositionRange_FeatureDisabledWithNegativeClockSkew() { @Test public void testGetFindPositionRange_SingleLedger() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setTimestamp(0).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setTimestamp(0)); Position lastConfirmedEntry = PositionFactory.create(1, 5); long targetTimestamp = 2500; @@ -1087,7 +1087,7 @@ public void testGetFindPositionRange_SingleLedger() { @Test public void testGetFindPositionRange_SingleClosedLedger() { List ledgerInfos = new ArrayList<>(); - ledgerInfos.add(LedgerInfo.newBuilder().setLedgerId(1).setEntries(10).setTimestamp(1000).build()); + ledgerInfos.add(new LedgerInfo().setLedgerId(1).setEntries(10).setTimestamp(1000)); Position lastConfirmedEntry = PositionFactory.create(1, 9); long targetTimestamp = 2500; 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 93bb6f86ffaf2..a84764a5f4a88 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 @@ -2372,7 +2372,7 @@ public void testLastPublishTimestampCaching() throws Exception { Position lastPosition = PositionFactory.create(1, 0); when(ledgerMock.getLastConfirmedEntry()).thenReturn(lastPosition); when(ledgerMock.getLedgersInfo()).thenReturn(new java.util.TreeMap<>(Map.of(1L, - mock(org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo.class)))); + mock(org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo.class)))); // Mock the last entry to return a timestamp Entry entryMock = mock(Entry.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java index 549c1e990f861..b01ef91d99d0e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SubscriptionSeekTest.java @@ -53,7 +53,7 @@ import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.commons.lang3.ArraySorter; import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -765,7 +765,7 @@ public void testSeekByTimestampWithLedgerTrim() throws Exception { ledger.trimConsumedLedgersInBackground(trimFuture); trimFuture.get(); Position readPosition = cursor.getReadPosition(); - Map.Entry firstLedger = + Map.Entry firstLedger = ledger.getLedgersInfo().firstEntry(); Assert.assertNotNull(firstLedger); if (firstLedger.getKey() > messageId.getLedgerId()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java index 82f1015d7cd63..30e308b30ebae 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/SegmentAbortedTxnProcessorTest.java @@ -36,7 +36,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarService; @@ -137,8 +137,8 @@ public void testPutAbortedTxnIntoProcessor() throws Exception { //3. Delete the ledgers and then verify the date. Field ledgersField = ManagedLedgerImpl.class.getDeclaredField("ledgers"); ledgersField.setAccessible(true); - NavigableMap ledgers = - (NavigableMap) + NavigableMap ledgers = + (NavigableMap) ledgersField.get(persistentTopic.getManagedLedger()); ledgers.forEach((k, v) -> { ledgers.remove(k); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java index 149d73aede67b..b127c632009ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TopicTransactionBufferRecoverTest.java @@ -51,7 +51,7 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.ReadOnlyManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo; import org.apache.commons.collections4.map.LinkedMap; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.PulsarService; @@ -461,8 +461,8 @@ private void testTopicTransactionBufferDeleteAbort(Boolean enableSnapshotSegment PersistentTopic persistentTopic = (PersistentTopic) topic.get(); var field = ManagedLedgerImpl.class.getDeclaredField("ledgers"); field.setAccessible(true); - NavigableMap ledgers = - (NavigableMap) + NavigableMap ledgers = + (NavigableMap) field.get(persistentTopic.getManagedLedger()); ledgers.remove(((MessageIdImpl) messageId1).getLedgerId()); From 296600b0fb3eeb4e30863d992ae2e5f8b9e14366 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 12:38:40 -0700 Subject: [PATCH 4/9] [improve][ml] Fix checkstyle import order in BrokerEntryMetadataE2ETest --- .../pulsar/broker/service/BrokerEntryMetadataE2ETest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java index 5118bcbf55654..9b07b46c38854 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java @@ -18,8 +18,8 @@ */ package org.apache.pulsar.broker.service; -import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import java.time.Duration; import java.util.ArrayList; import java.util.List; From b5c4300463b65fe88ae892af23115c3dc9f808b8 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 13:03:54 -0700 Subject: [PATCH 5/9] [improve][ml] Fix checkstyle import order in BrokerEntryMetadataE2ETest --- .../pulsar/broker/service/BrokerEntryMetadataE2ETest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java index 9b07b46c38854..9a09a3267f2b1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerEntryMetadataE2ETest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.broker.service; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import java.time.Duration; import java.util.ArrayList; import java.util.List; @@ -30,6 +29,7 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; From 3ea03662d71fe17033dd70b0f197dc9e75965358 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 15:18:02 -0700 Subject: [PATCH 6/9] [improve][build] Upgrade LightProto to 0.6.2 LightProto 0.6.2 fixes optional field behavior to return default values instead of throwing exceptions, matching standard protobuf behavior. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index c3a9e357f3826..cc47242f89dfc 100644 --- a/pom.xml +++ b/pom.xml @@ -363,7 +363,7 @@ flexible messaging model and an intuitive client API. 2.45.0 0.1.29 1.3 - 0.6.1 + 0.6.2 3.6.0 12.1.0 1.6.9 From 859ad11db008fb7177fc7dea80404902a8d6bab8 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 15:49:21 -0700 Subject: [PATCH 7/9] [improve][ml] Fix ManagedCursorIndividualDeletedMessagesTest for LightProto The test was using reflection to call a private method with a List parameter that no longer exists. Updated to use the public recoverIndividualDeletedMessages(PositionInfo) method directly. --- ...edCursorIndividualDeletedMessagesTest.java | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java index 529825a340128..c408fa16349b0 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java @@ -23,7 +23,6 @@ import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import com.google.common.collect.Range; -import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.NavigableMap; @@ -34,6 +33,7 @@ import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.proto.MessageRange; +import org.apache.bookkeeper.mledger.proto.PositionInfo; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.testng.annotations.Test; @@ -59,40 +59,34 @@ void testRecoverIndividualDeletedMessages() throws Exception { ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, ledger, "test-cursor")); LongPairRangeSet deletedMessages = cursor.getIndividuallyDeletedMessagesSet(); - Method recoverMethod = ManagedCursorImpl.class.getDeclaredMethod("recoverIndividualDeletedMessages", - List.class); - recoverMethod.setAccessible(true); - // (1) [(1:5..1:10]] - List messageRangeList = new ArrayList(); - messageRangeList.add(createMessageRange(1, 5, 1, 10)); - List> expectedRangeList = new ArrayList(); + PositionInfo positionInfo = createPositionInfo(createMessageRange(1, 5, 1, 10)); + List> expectedRangeList = new ArrayList<>(); expectedRangeList.add(createPositionRange(1, 5, 1, 10)); - recoverMethod.invoke(cursor, messageRangeList); + cursor.recoverIndividualDeletedMessages(positionInfo); assertEquals(deletedMessages.size(), 1); assertEquals(deletedMessages.asRanges(), expectedRangeList); // (2) [(1:10..3:0]] - messageRangeList.clear(); - messageRangeList.add(createMessageRange(1, 10, 3, 0)); + positionInfo = createPositionInfo(createMessageRange(1, 10, 3, 0)); expectedRangeList.clear(); expectedRangeList.add(createPositionRange(1, 10, 1, 99)); expectedRangeList.add(createPositionRange(3, -1, 3, 0)); - recoverMethod.invoke(cursor, messageRangeList); + cursor.recoverIndividualDeletedMessages(positionInfo); assertEquals(deletedMessages.size(), 2); assertEquals(deletedMessages.asRanges(), expectedRangeList); // (3) [(1:20..10:1],(20:2..20:9]] - messageRangeList.clear(); - messageRangeList.add(createMessageRange(1, 20, 10, 1)); - messageRangeList.add(createMessageRange(20, 2, 20, 9)); + positionInfo = createPositionInfo( + createMessageRange(1, 20, 10, 1), + createMessageRange(20, 2, 20, 9)); expectedRangeList.clear(); expectedRangeList.add(createPositionRange(1, 20, 1, 99)); expectedRangeList.add(createPositionRange(3, -1, 3, 49)); expectedRangeList.add(createPositionRange(5, -1, 5, 199)); expectedRangeList.add(createPositionRange(10, -1, 10, 1)); expectedRangeList.add(createPositionRange(20, 2, 20, 9)); - recoverMethod.invoke(cursor, messageRangeList); + cursor.recoverIndividualDeletedMessages(positionInfo); assertEquals(deletedMessages.size(), 5); assertEquals(deletedMessages.asRanges(), expectedRangeList); } @@ -102,6 +96,14 @@ private static LedgerInfo createLedgerInfo(long ledgerId, long entries, long siz .setTimestamp(System.currentTimeMillis()); } + private static PositionInfo createPositionInfo(MessageRange... ranges) { + PositionInfo positionInfo = new PositionInfo(); + for (MessageRange range : ranges) { + positionInfo.addIndividualDeletedMessage().copyFrom(range); + } + return positionInfo; + } + private static MessageRange createMessageRange(long lowerLedgerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { MessageRange messageRange = new MessageRange(); From 5d15b02ed487eebe0eb8478379385290aa1e30f1 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 16:27:47 -0700 Subject: [PATCH 8/9] [improve][ml] Fix ManagedCursorInfoMetadataTest for LightProto LightProto does not implement equals(), so compare serialized bytes instead of object equality. --- .../bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java index 389d5e4754ca8..42247941df973 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorInfoMetadataTest.java @@ -92,7 +92,7 @@ public void testEncodeAndDecode(String compressionType) throws Exception { // parse compression data and unCompression data, check their results. ManagedCursorInfo info1 = metaStore.parseManagedCursorInfo(compressionBytes); ManagedCursorInfo info2 = metaStore.parseManagedCursorInfo(managedCursorInfo.toByteArray()); - assertEquals(info1, info2); + assertEquals(info1.toByteArray(), info2.toByteArray()); } @Test(dataProvider = "compressionTypeProvider") From e3210363059e486b2813992d766b87a469823409 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 17 Mar 2026 18:52:37 -0700 Subject: [PATCH 9/9] [improve][ml] Fix ManagedLedgerInfoMetadataTest for LightProto Compare serialized bytes instead of object equality since LightProto does not implement equals(). Also call materialize() after parsing from ByteBuf to ensure the data is fully deserialized before releasing the buffer. --- .../org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java | 1 + .../mledger/impl/ManagedLedgerInfoMetadataTest.java | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java index a3f0591d81c32..4e7adf9aeaf52 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/MetaStoreImpl.java @@ -470,6 +470,7 @@ public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws Exception { try { ManagedLedgerInfo info = new ManagedLedgerInfo(); info.parseFrom(uncompressed, uncompressed.readableBytes()); + info.materialize(); return info; } finally { uncompressed.release(); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java index 567638ec1523c..008acf12a8811 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerInfoMetadataTest.java @@ -116,14 +116,14 @@ public void testEncodeAndDecode(String compressionType) throws Exception { // parse compression data and unCompression data, check their results. ManagedLedgerInfo info1 = metaStore.parseManagedLedgerInfo(compressionBytes); ManagedLedgerInfo info2 = metaStore.parseManagedLedgerInfo(managedLedgerInfo.toByteArray()); - Assert.assertEquals(info1, info2); + Assert.assertEquals(info1.toByteArray(), info2.toByteArray()); } @Test public void testParseEmptyData() throws Exception { MetaStoreImpl metaStore = new MetaStoreImpl(null, null); ManagedLedgerInfo managedLedgerInfo = metaStore.parseManagedLedgerInfo(new byte[0]); - Assert.assertEquals(managedLedgerInfo.toString(), ""); + Assert.assertEquals(managedLedgerInfo.toByteArray(), new byte[0]); } @Test(dataProvider = "compressionTypeProvider")