From e0143569a3dff3e4987200da36a9ca5bbedf47ba Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 10 May 2024 11:23:14 +0200 Subject: [PATCH 01/22] ManagedCursor: compress data written to BookKeeper (cherry picked from commit 1ef96646b1aa2f8da169447a4e51cb01004e5a47) --- .../mledger/impl/LedgerMetadataUtils.java | 10 +- .../mledger/impl/ManagedCursorImpl.java | 100 +++++++++++++++++- .../mledger/impl/MetaStoreImpl.java | 18 +++- ...edCursorIndividualDeletedMessagesTest.java | 16 +++ .../mledger/impl/ManagedCursorTest.java | 6 ++ 5 files changed, 141 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java index 4ac409a2e9bfe..6d64b4ec6be27 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerMetadataUtils.java @@ -44,6 +44,7 @@ public final class LedgerMetadataUtils { private static final String METADATA_PROPERTY_MANAGED_LEDGER_NAME = "pulsar/managed-ledger"; private static final String METADATA_PROPERTY_CURSOR_NAME = "pulsar/cursor"; + public static final String METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE = "pulsar/cursor-compressionType"; private static final String METADATA_PROPERTY_COMPACTEDTOPIC = "pulsar/compactedTopic"; private static final String METADATA_PROPERTY_COMPACTEDTO = "pulsar/compactedTo"; private static final String METADATA_PROPERTY_SCHEMAID = "pulsar/schemaId"; @@ -72,8 +73,13 @@ static Map buildBaseManagedLedgerMetadata(String name) { * @return an immutable map which describes the cursor * @see #buildBaseManagedLedgerMetadata(java.lang.String) */ - static Map buildAdditionalMetadataForCursor(String name) { - return Map.of(METADATA_PROPERTY_CURSOR_NAME, name.getBytes(StandardCharsets.UTF_8)); + static Map buildAdditionalMetadataForCursor(String name, String compressionType) { + if (compressionType != null) { + return Map.of(METADATA_PROPERTY_CURSOR_NAME, name.getBytes(StandardCharsets.UTF_8), + METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE, compressionType.getBytes(StandardCharsets.UTF_8)); + } else { + return Map.of(METADATA_PROPERTY_CURSOR_NAME, name.getBytes(StandardCharsets.UTF_8)); + } } /** 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 e27814eadd0b5..59f25297c40fe 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 @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; import static org.apache.bookkeeper.mledger.ManagedLedgerException.getManagedLedgerException; +import static org.apache.bookkeeper.mledger.impl.LedgerMetadataUtils.METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_RETRIES; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; @@ -32,6 +33,14 @@ import com.google.common.collect.Range; import com.google.common.util.concurrent.RateLimiter; import com.google.protobuf.InvalidProtocolBufferException; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.Unpooled; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; @@ -98,6 +107,9 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.api.proto.CompressionType; +import org.apache.pulsar.common.compression.CompressionCodec; +import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; @@ -125,6 +137,7 @@ public class ManagedCursorImpl implements ManagedCursor { protected final BookKeeper bookkeeper; protected final ManagedLedgerImpl ledger; private final String name; + private final String cursorInfoCompressionType; private volatile Map cursorProperties; private final BookKeeper.DigestType digestType; @@ -328,6 +341,7 @@ public interface VoidCallback { markDeleteLimiter = null; } this.mbean = new ManagedCursorMXBeanImpl(this); + this.cursorInfoCompressionType = ledger.getFactory().getConfig().getManagedCursorInfoCompressionType(); } private void updateCursorLedgerStat(ManagedCursorInfo cursorInfo, Stat stat) { @@ -589,11 +603,14 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac mbean.addReadCursorLedgerSize(entry.getLength()); PositionInfo positionInfo; try { - positionInfo = PositionInfo.parseFrom(entry.getEntry()); + byte[] data = entry.getEntry(); + data = decompressDataIfNeeded(data, lh); + positionInfo = PositionInfo.parseFrom(data); } catch (InvalidProtocolBufferException e) { callback.operationFailed(new ManagedLedgerException(e)); return; } + log.info("[{}] Cursor {} recovered to position {}", ledger.getName(), name, positionInfo); Map recoveredProperties = Collections.emptyMap(); if (positionInfo.getPropertiesCount() > 0) { @@ -605,6 +622,9 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } + log.info("[{}] Cursor {} recovered with recoveredProperties {}, individualDeletedMessagesCount {}", + ledger.getName(), name, recoveredProperties, positionInfo.getIndividualDeletedMessagesCount()); + Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); @@ -628,6 +648,8 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { + log.info("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}", + ledger.getName(), name, individualDeletedMessagesList.size()); lock.writeLock().lock(); try { individualDeletedMessages.clear(); @@ -2996,7 +3018,7 @@ private CompletableFuture doCreateNewMetadataLedger() { } future.complete(lh); }); - }, LedgerMetadataUtils.buildAdditionalMetadataForCursor(name)); + }, LedgerMetadataUtils.buildAdditionalMetadataForCursor(name, cursorInfoCompressionType)); return future; } @@ -3047,6 +3069,8 @@ private static List buildStringPropertiesMap(Map private List buildIndividualDeletedMessageRanges() { lock.writeLock().lock(); try { + log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {}", + ledger.getName(), name, individualDeletedMessages.size()); if (individualDeletedMessages.isEmpty()) { this.individualDeletedMessagesSerializedSize = 0; return Collections.emptyList(); @@ -3085,6 +3109,13 @@ private List buildIndividualDeletedMessageRanges() { this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); + log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {} " + + "individualDeletedMessagesSerializedSize {} rangeListSize {} " + + "maxUnackedRangesToPersist {}", + ledger.getName(), name, individualDeletedMessages.size(), + individualDeletedMessagesSerializedSize, rangeList.size(), + getConfig().getMaxUnackedRangesToPersist()); + return rangeList; } finally { lock.writeLock().unlock(); @@ -3138,7 +3169,14 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin } requireNonNull(lh); - byte[] data = pi.toByteArray(); + byte[] rawData = pi.toByteArray(); + + byte[] data = compressDataIfNeeded(rawData, lh); + + log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes", + ledger.getName(), name, lh.getId(), + position, data.length); + lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> { if (rc == BKException.Code.OK) { if (log.isDebugEnabled()) { @@ -3164,6 +3202,62 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin }, null); } + private byte[] compressDataIfNeeded(byte[] data, LedgerHandle lh) { + byte[] pulsarCursorInfoCompression = + lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE); + if (pulsarCursorInfoCompression != null) { + String pulsarCursorInfoCompressionString = new String(pulsarCursorInfoCompression); + CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( + CompressionType.valueOf(pulsarCursorInfoCompressionString)); + ByteBuf encode = compressionCodec.encode(Unpooled.wrappedBuffer(data)); + try { + int uncompressedSize = data.length; + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputStream dataOutputStream = new DataOutputStream(out); + dataOutputStream.writeInt(uncompressedSize); + dataOutputStream.write(ByteBufUtil.getBytes(encode)); + dataOutputStream.flush(); + byte[] result = out.toByteArray(); + int ratio = (int) (result.length * 100.0 / uncompressedSize); + log.info("[{}] Cursor {} Compressed data size {} bytes (with {}, original size {} bytes, ratio {}%)", + ledger.getName(), name, result.length, pulsarCursorInfoCompressionString, data.length, ratio); + return result; + } catch (IOException error) { + throw new RuntimeException(error); + } finally { + encode.release(); + } + } else { + return data; + } + } + + private static byte[] decompressDataIfNeeded(byte[] data, LedgerHandle lh) { + byte[] pulsarCursorInfoCompression = + lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE); + if (pulsarCursorInfoCompression != null) { + String pulsarCursorInfoCompressionString = new String(pulsarCursorInfoCompression); + CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( + CompressionType.valueOf(pulsarCursorInfoCompressionString)); + ByteArrayInputStream input = new ByteArrayInputStream(data); + DataInputStream dataInputStream = new DataInputStream(input); + try { + int uncompressedSize = dataInputStream.readInt(); + byte[] compressedData = dataInputStream.readNBytes(uncompressedSize); + ByteBuf decode = compressionCodec.decode(Unpooled.wrappedBuffer(compressedData), uncompressedSize); + try { + return ByteBufUtil.getBytes(decode); + } finally { + decode.release(); + } + } catch (IOException error) { + throw new RuntimeException(error); + } + } else { + return data; + } + } + public boolean periodicRollover() { LedgerHandle lh = cursorLedger; if (State.Open.equals(STATE_UPDATER.get(this)) 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 d9269ec83b179..a9117848e58bc 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 @@ -453,8 +453,13 @@ public ManagedLedgerInfo parseManagedLedgerInfo(byte[] data) throws InvalidProto try { MLDataFormats.ManagedLedgerInfoMetadata metadata = MLDataFormats.ManagedLedgerInfoMetadata.parseFrom(metadataBytes); - return ManagedLedgerInfo.parseFrom(getCompressionCodec(metadata.getCompressionType()) - .decode(byteBuf, metadata.getUncompressedSize()).nioBuffer()); + ByteBuf decode = getCompressionCodec(metadata.getCompressionType()) + .decode(byteBuf, metadata.getUncompressedSize()); + try { + return ManagedLedgerInfo.parseFrom(decode.nioBuffer()); + } finally { + decode.release(); + } } catch (Exception e) { log.error("Failed to parse managedLedgerInfo metadata, " + "fall back to parse managedLedgerInfo directly.", e); @@ -475,8 +480,13 @@ public ManagedCursorInfo parseManagedCursorInfo(byte[] data) throws InvalidProto try { MLDataFormats.ManagedCursorInfoMetadata metadata = MLDataFormats.ManagedCursorInfoMetadata.parseFrom(metadataBytes); - return ManagedCursorInfo.parseFrom(getCompressionCodec(metadata.getCompressionType()) - .decode(byteBuf, metadata.getUncompressedSize()).nioBuffer()); + ByteBuf decode = getCompressionCodec(metadata.getCompressionType()) + .decode(byteBuf, metadata.getUncompressedSize()); + try { + return ManagedCursorInfo.parseFrom(decode.nioBuffer()); + } finally { + decode.release(); + } } catch (Exception e) { log.error("Failed to parse ManagedCursorInfo metadata, " + "fall back to parse ManagedCursorInfo directly", e); 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 3d4de5b1f4975..27643bbd6f550 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 @@ -33,6 +33,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; +import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; 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; @@ -40,8 +41,18 @@ import org.testng.annotations.Test; public class ManagedCursorIndividualDeletedMessagesTest { + @Test(timeOut = 10000) void testRecoverIndividualDeletedMessages() throws Exception { + testRecoverIndividualDeletedMessages(null); + } + + @Test(timeOut = 10000) + void testRecoverIndividualDeletedMessagesWithZSTDCompression() throws Exception { + testRecoverIndividualDeletedMessages("ZSTD"); + } + + void testRecoverIndividualDeletedMessages(String compression) throws Exception { BookKeeper bookkeeper = mock(BookKeeper.class); ManagedLedgerConfig config = new ManagedLedgerConfig(); @@ -54,8 +65,13 @@ void testRecoverIndividualDeletedMessages() throws Exception { ledgersInfo.put(10L, createLedgerInfo(10, 2, 32)); ledgersInfo.put(20L, createLedgerInfo(20, 10, 256)); + ManagedLedgerFactoryImpl factory = mock(ManagedLedgerFactoryImpl.class); + ManagedLedgerFactoryConfig factoryConfig = new ManagedLedgerFactoryConfig(); + factoryConfig.setManagedCursorInfoCompressionType(compression); + doReturn(factoryConfig).when(factory).getConfig(); ManagedLedgerImpl ledger = mock(ManagedLedgerImpl.class); doReturn(ledgersInfo).when(ledger).getLedgersInfo(); + doReturn(factory).when(ledger).getFactory(); doReturn(config).when(ledger).getConfig(); ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, ledger, "test-cursor")); 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 8913c4013b4ab..b42e5fe283c73 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 @@ -21,6 +21,7 @@ import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -3468,6 +3469,11 @@ public Object answer(InvocationOnMock invocation) { when(ml.ledgerExists(markDeleteLedgerId)).thenReturn(false); when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); + ManagedLedgerFactoryImpl factory = mock(ManagedLedgerFactoryImpl.class); + ManagedLedgerFactoryConfig factoryConfig = new ManagedLedgerFactoryConfig(); + doReturn(factoryConfig).when(factory).getConfig(); + when(ml.getFactory()).thenReturn(factory); + BookKeeper mockBookKeeper = mock(BookKeeper.class); final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, ml, cursorName); From 33e4c718082e765e4d313eaf4b638657d25a27a1 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Wed, 15 May 2024 22:24:07 -0700 Subject: [PATCH 02/22] serialize/compress without intermediate byte arrays (#268) * serialize/compress without intermediate byte arrays * use lightproto for cursor serialization to the ledger * Reuse PositionInfo (cherry picked from commit 1887c447acd9183d8e6b3862a3040b42b249d939) --- managed-ledger/pom.xml | 16 ++ .../mledger/impl/ManagedCursorImpl.java | 260 +++++++++++++++--- 2 files changed, 241 insertions(+), 35 deletions(-) diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 22b093f7aafd7..f326f9a782993 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -191,6 +191,22 @@ + + com.github.splunk.lightproto + lightproto-maven-plugin + ${lightproto-maven-plugin.version} + + true + Light + + + + + generate + + + + org.apache.maven.plugins maven-checkstyle-plugin 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 59f25297c40fe..8d1924fe73114 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 @@ -26,6 +26,7 @@ import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_RETRIES; import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException; import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException; +import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.Collections2; @@ -35,11 +36,11 @@ import com.google.protobuf.InvalidProtocolBufferException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; +import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; +import io.netty.util.concurrent.FastThreadLocal; import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.time.Clock; import java.util.ArrayDeque; @@ -69,6 +70,11 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.LongStream; +import lombok.AllArgsConstructor; +import lombok.Data; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.ToString; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; @@ -99,6 +105,7 @@ 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.LightMLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; @@ -107,12 +114,14 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.proto.CompressionType; import org.apache.pulsar.common.compression.CompressionCodec; import org.apache.pulsar.common.compression.CompressionCodecProvider; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.common.util.collections.LongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet.LongPairConsumer; @@ -123,6 +132,14 @@ @SuppressWarnings("checkstyle:javadoctype") public class ManagedCursorImpl implements ManagedCursor { + + private static final FastThreadLocal piThreadLocal = new FastThreadLocal<>() { + @Override + protected LightMLDataFormats.PositionInfo initialValue() { + return new LightMLDataFormats.PositionInfo(); + } + }; + private static final Comparator ENTRY_COMPARATOR = (e1, e2) -> { if (e1.getLedgerId() != e2.getLedgerId()) { return e1.getLedgerId() < e2.getLedgerId() ? -1 : 1; @@ -647,6 +664,25 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac } } + @AllArgsConstructor + @NoArgsConstructor + @Getter + @ToString + @Data + public static final class ChunkSequenceFooter { + private static final ChunkSequenceFooter NOT_CHUNKED = new ChunkSequenceFooter(0, 0); + private int numParts; + private int length; + } + + private ChunkSequenceFooter parseChunkSequenceFooter(byte[] data) throws IOException { + if (data.length == 0 || data[0] != '{') { + // this is not JSON + return ChunkSequenceFooter.NOT_CHUNKED; + } + return ObjectMapperFactory.getMapper().getObjectMapper().readValue(data, ChunkSequenceFooter.class); + } + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { log.info("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}", ledger.getName(), name, individualDeletedMessagesList.size()); @@ -3038,6 +3074,18 @@ private CompletableFuture deleteLedgerAsync(LedgerHandle ledgerHandle) { } + private static void addAllProperties(LightMLDataFormats.PositionInfo lpi, Map properties) { + if (properties.isEmpty()) { + return; + } + + properties.forEach((name, value) -> { + lpi.addProperty() + .setName(name) + .setValue(value); + }); + } + private static List buildPropertiesMap(Map properties) { if (properties.isEmpty()) { return Collections.emptyList(); @@ -3122,6 +3170,44 @@ private List buildIndividualDeletedMessageRanges() { } } + private void addIndividualDeletedMessageRanges(LightMLDataFormats.PositionInfo lpi) { + lock.readLock().lock(); + try { + if (individualDeletedMessages.isEmpty()) { + this.individualDeletedMessagesSerializedSize = 0; + return; + } + + AtomicInteger acksSerializedSize = new AtomicInteger(0); + AtomicInteger rangeCount = new AtomicInteger(0); + + individualDeletedMessages.forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { + LightMLDataFormats.MessageRange messageRange = lpi.addIndividualDeletedMessage(); + messageRange.setLowerEndpoint() + .setLedgerId(lowerKey) + .setEntryId(lowerValue); + messageRange.setUpperEndpoint() + .setLedgerId(upperKey) + .setEntryId(upperValue); + + acksSerializedSize.addAndGet(messageRange.getSerializedSize()); + + return rangeCount.incrementAndGet() <= getConfig().getMaxUnackedRangesToPersist(); + }); + + this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); + individualDeletedMessages.resetDirtyKeys(); + log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {} " + + "individualDeletedMessagesSerializedSize {} rangeListSize {} " + + "maxUnackedRangesToPersist {}", + ledger.getName(), name, individualDeletedMessages.size(), + individualDeletedMessagesSerializedSize, rangeCount.get(), + getConfig().getMaxUnackedRangesToPersist()); + } finally { + lock.readLock().unlock(); + } + } + private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { @@ -3154,14 +3240,52 @@ private List buildBatchEntryDeletio } } + private void addAllBatchedEntryDeletionIndexInfo(LightMLDataFormats.PositionInfo lpi) { + lock.readLock().lock(); + try { + if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + return; + } + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + int count = 0; + while (iterator.hasNext() && count < getConfig().getMaxBatchDeletedIndexToPersist()) { + Map.Entry entry = iterator.next(); + + LightMLDataFormats.BatchedEntryDeletionIndexInfo batchInfo = lpi.addBatchedEntryDeletionIndexInfo(); + batchInfo.setPosition() + .setLedgerId(entry.getKey().getLedgerId()) + .setEntryId(entry.getKey().getEntryId()); + + long[] array = entry.getValue().toLongArray(); + List deleteSet = new ArrayList<>(array.length); + for (long l : array) { + batchInfo.addDeleteSet(l); + } + count++; + } + } finally { + lock.readLock().unlock(); + } + } + + private static ByteBuf toByteBuf(LightMLDataFormats.PositionInfo pi) { + int size = pi.getSerializedSize(); + ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(size, size); + pi.writeTo(buf); + return buf; + } + void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { Position position = mdEntry.newPosition; - PositionInfo pi = PositionInfo.newBuilder().setLedgerId(position.getLedgerId()) - .setEntryId(position.getEntryId()) - .addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()) - .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()) - .addAllProperties(buildPropertiesMap(mdEntry.properties)).build(); + LightMLDataFormats.PositionInfo pi = piThreadLocal.get(); + pi.clear(); + + pi.setLedgerId(position.getLedgerId()) + .setEntryId(position.getEntryId()); + addIndividualDeletedMessageRanges(pi); + addAllBatchedEntryDeletionIndexInfo(pi); + addAllProperties(pi, mdEntry.properties); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(), @@ -3169,14 +3293,75 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin } requireNonNull(lh); - byte[] rawData = pi.toByteArray(); + ByteBuf rawData = toByteBuf(pi); + + // rawData is released by compressDataIfNeeded if needed + ByteBuf data = compressDataIfNeeded(rawData, lh); + + int maxSize = 1024 * 1024; + int offset = 0; + final int len = data.readableBytes(); + int numParts = 1 + (len / maxSize); + + if (log.isDebugEnabled()) { + log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {}", + ledger.getName(), name, lh.getId(), + position, len, numParts); + } + + if (numParts == 1) { + // no need for chunking + // asyncAddEntry will release data ByteBuf + writeToBookKeeperLastChunk(lh, mdEntry, callback, data, position, () -> {}); + } else { + // chunking + int part = 0; + while (part != numParts) { + int remaining = len - offset; + int currentLen = Math.min(maxSize, remaining); + boolean isLast = part == numParts - 1; + + if (log.isDebugEnabled()) { + log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {} " + + "part {} offset {} len {}", + ledger.getName(), name, lh.getId(), + position, len, numParts, part, offset, currentLen); + } - byte[] data = compressDataIfNeeded(rawData, lh); + // just send the addEntry, BK client guarantees that each entry succeeds only if all + // the previous entries succeeded + // asyncAddEntry takes ownership of the buffer + lh.asyncAddEntry(data.retainedSlice(offset, currentLen), (rc, lh1, entryId, ctx) -> { + }, null); - log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes", - ledger.getName(), name, lh.getId(), - position, data.length); + if (isLast) { + // last, send a footer with the number of parts + ChunkSequenceFooter footer = new ChunkSequenceFooter(numParts, len); + byte[] footerData; + try { + footerData = ObjectMapperFactory.getMapper() + .getObjectMapper().writeValueAsBytes(footer); + } catch (JsonProcessingException e) { + // this is almost impossible to happen + log.error("Cannot serialize footer {}", footer); + return; + } + // need to explicitly release data ByteBuf + writeToBookKeeperLastChunk(lh, mdEntry, callback, + Unpooled.wrappedBuffer(footerData), position, data::release); + } + offset += currentLen; + part++; + } + } + } + private void writeToBookKeeperLastChunk(LedgerHandle lh, + MarkDeleteEntry mdEntry, + VoidCallback callback, + ByteBuf data, + PositionImpl position, + Runnable onFinished) { lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> { if (rc == BKException.Code.OK) { if (log.isDebugEnabled()) { @@ -3187,8 +3372,9 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin rolloverLedgerIfNeeded(lh1); mbean.persistToLedger(true); - mbean.addWriteCursorLedgerSize(data.length); + mbean.addWriteCursorLedgerSize(data.readableBytes()); callback.operationComplete(); + onFinished.run(); } else { log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}: {}", ledger.getName(), name, position, lh1.getId(), BKException.getMessage(rc)); @@ -3198,37 +3384,41 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin // Before giving up, try to persist the position in the metadata store. persistPositionToMetaStore(mdEntry, callback); + onFinished.run(); } }, null); } - private byte[] compressDataIfNeeded(byte[] data, LedgerHandle lh) { + private ByteBuf compressDataIfNeeded(ByteBuf data, LedgerHandle lh) { byte[] pulsarCursorInfoCompression = lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE); - if (pulsarCursorInfoCompression != null) { + if (pulsarCursorInfoCompression == null) { + return data; + } + + try { + int uncompressedSize = data.readableBytes(); String pulsarCursorInfoCompressionString = new String(pulsarCursorInfoCompression); CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( CompressionType.valueOf(pulsarCursorInfoCompressionString)); - ByteBuf encode = compressionCodec.encode(Unpooled.wrappedBuffer(data)); - try { - int uncompressedSize = data.length; - ByteArrayOutputStream out = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(out); - dataOutputStream.writeInt(uncompressedSize); - dataOutputStream.write(ByteBufUtil.getBytes(encode)); - dataOutputStream.flush(); - byte[] result = out.toByteArray(); - int ratio = (int) (result.length * 100.0 / uncompressedSize); - log.info("[{}] Cursor {} Compressed data size {} bytes (with {}, original size {} bytes, ratio {}%)", - ledger.getName(), name, result.length, pulsarCursorInfoCompressionString, data.length, ratio); - return result; - } catch (IOException error) { - throw new RuntimeException(error); - } finally { - encode.release(); - } - } else { - return data; + ByteBuf encode = compressionCodec.encode(data); + + int compressedSize = encode.readableBytes(); + + ByteBuf szBuf = PulsarByteBufAllocator.DEFAULT.buffer(4).writeInt(uncompressedSize); + + CompositeByteBuf result = PulsarByteBufAllocator.DEFAULT.compositeBuffer(2); + result.addComponent(szBuf) + .addComponent(encode); + result.readerIndex(0) + .writerIndex(4 + compressedSize); + + int ratio = (int) (compressedSize * 100.0 / uncompressedSize); + log.info("[{}] Cursor {} Compressed data size {} bytes (with {}, original size {} bytes, ratio {}%)", + ledger.getName(), name, compressedSize, pulsarCursorInfoCompressionString, uncompressedSize, ratio); + return result; + } finally { + data.release(); } } From 6d1b93a396b9d262596cfc952929530b63d6ccab Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 16 May 2024 08:56:57 +0200 Subject: [PATCH 03/22] Print time (cherry picked from commit 98a3d254eb74f2e389c7b079bc6d0655abf679de) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 8d1924fe73114..8feb5171cf0a0 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 @@ -3276,6 +3276,7 @@ private static ByteBuf toByteBuf(LightMLDataFormats.PositionInfo pi) { } void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { + long now = System.nanoTime(); Position position = mdEntry.newPosition; LightMLDataFormats.PositionInfo pi = piThreadLocal.get(); @@ -3294,10 +3295,13 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin requireNonNull(lh); ByteBuf rawData = toByteBuf(pi); + long endSer = System.nanoTime(); // rawData is released by compressDataIfNeeded if needed ByteBuf data = compressDataIfNeeded(rawData, lh); + long endCompress = System.nanoTime(); + int maxSize = 1024 * 1024; int offset = 0; final int len = data.readableBytes(); @@ -3308,6 +3312,12 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin ledger.getName(), name, lh.getId(), position, len, numParts); } + log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, " + + "numParts {}, serializeTime {} ms" + + " compressTime {} ms, total {} ms", ledger.getName(), name, lh.getId(), + position, len, numParts, + (endSer - now) / 1000000, + (endCompress - endSer) / 1000000, (endCompress - now) / 1000000); if (numParts == 1) { // no need for chunking @@ -3322,7 +3332,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin boolean isLast = part == numParts - 1; if (log.isDebugEnabled()) { - log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {} " + log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {} " + "part {} offset {} len {}", ledger.getName(), name, lh.getId(), position, len, numParts, part, offset, currentLen); From 568d446d16f1d2bff34f797656f21cca96f7170f Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 16 May 2024 12:09:02 +0200 Subject: [PATCH 04/22] ManagedCursor: manually serialise PositionInfo (#270) * ManagedCursor: manually serialise PositionInfo * Add tests and save last serialized side to prevent reallocations (cherry picked from commit 8a365d097c8a73fd71410676cfce43c7bd368b99) --- .../main/resources/pulsar/suppressions.xml | 1 + .../mledger/impl/ManagedCursorImpl.java | 116 +- .../mledger/impl/PositionInfoUtils.java | 1514 +++++++++++++++++ .../mledger/impl/PositionInfoUtilsTest.java | 91 + 4 files changed, 1679 insertions(+), 43 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java diff --git a/buildtools/src/main/resources/pulsar/suppressions.xml b/buildtools/src/main/resources/pulsar/suppressions.xml index 57a01c60f6a27..ab83286ef9f97 100644 --- a/buildtools/src/main/resources/pulsar/suppressions.xml +++ b/buildtools/src/main/resources/pulsar/suppressions.xml @@ -36,6 +36,7 @@ + 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 8feb5171cf0a0..f72a02289ff56 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -160,6 +160,7 @@ protected LightMLDataFormats.PositionInfo initialValue() { private final BookKeeper.DigestType digestType; protected volatile Position markDeletePosition; + private int lastSerializedSize; // this position is have persistent mark delete position protected volatile Position persistentMarkDeletePosition; @@ -246,7 +247,7 @@ protected LightMLDataFormats.PositionInfo initialValue() { // active state cache in ManagedCursor. It should be in sync with the state in activeCursors in ManagedLedger. private volatile boolean isActive = false; - class MarkDeleteEntry { + static class MarkDeleteEntry { final Position newPosition; final MarkDeleteCallback callback; final Object ctx; @@ -683,6 +684,52 @@ private ChunkSequenceFooter parseChunkSequenceFooter(byte[] data) throws IOExcep return ObjectMapperFactory.getMapper().getObjectMapper().readValue(data, ChunkSequenceFooter.class); } + private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byte[] data) { + mbean.addReadCursorLedgerSize(data.length); + + try { + data = decompressDataIfNeeded(data, lh); + } catch (Throwable e) { + callback.operationFailed(new ManagedLedgerException(e)); + return; + } + + PositionInfo positionInfo; + try { + positionInfo = PositionInfo.parseFrom(data); + } catch (InvalidProtocolBufferException e) { + log.error("[{}] Failed to parse position info from ledger {} for cursor {}: {}", ledger.getName(), + lh.getId(), name, e); + // Rewind to oldest entry available + positionInfo = PositionInfo + .newBuilder() + .setLedgerId(-1) + .setEntryId(-1) + .build(); + } + + Map recoveredProperties = Collections.emptyMap(); + if (positionInfo.getPropertiesCount() > 0) { + // Recover properties map + recoveredProperties = new HashMap<>(); + for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { + LongProperty property = positionInfo.getProperties(i); + recoveredProperties.put(property.getName(), property.getValue()); + } + } + + PositionImpl position = new PositionImpl(positionInfo); + if (positionInfo.getIndividualDeletedMessagesCount() > 0) { + recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + } + if (getConfig().isDeletionAtBatchIndexLevelEnabled() + && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { + recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); + } + recoveredCursor(position, recoveredProperties, cursorProperties, lh); + callback.operationComplete(); + } + private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { log.info("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}", ledger.getName(), name, individualDeletedMessagesList.size()); @@ -3170,7 +3217,12 @@ private List buildIndividualDeletedMessageRanges() { } } - private void addIndividualDeletedMessageRanges(LightMLDataFormats.PositionInfo lpi) { + private void scanIndividualDeletedMessageRanges( + PositionInfoUtils.IndividuallyDeletedMessagesRangeConsumer consumer) { + final int maxUnackedRangesToPersist = getConfig().getMaxUnackedRangesToPersist(); + AtomicInteger acksSerializedSize = new AtomicInteger(0); + AtomicInteger rangeCount = new AtomicInteger(0); + lock.readLock().lock(); try { if (individualDeletedMessages.isEmpty()) { @@ -3178,26 +3230,15 @@ private void addIndividualDeletedMessageRanges(LightMLDataFormats.PositionInfo l return; } - AtomicInteger acksSerializedSize = new AtomicInteger(0); - AtomicInteger rangeCount = new AtomicInteger(0); - individualDeletedMessages.forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { - LightMLDataFormats.MessageRange messageRange = lpi.addIndividualDeletedMessage(); - messageRange.setLowerEndpoint() - .setLedgerId(lowerKey) - .setEntryId(lowerValue); - messageRange.setUpperEndpoint() - .setLedgerId(upperKey) - .setEntryId(upperValue); - - acksSerializedSize.addAndGet(messageRange.getSerializedSize()); - - return rangeCount.incrementAndGet() <= getConfig().getMaxUnackedRangesToPersist(); + acksSerializedSize.addAndGet(16 * 4); + consumer.acceptRange(lowerKey, lowerValue, upperKey, upperValue); + return rangeCount.incrementAndGet() <= maxUnackedRangesToPersist; }); this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); - log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {} " + log.info("[{}] [{}] scanIndividualDeletedMessageRanges, numRanges {} " + "individualDeletedMessagesSerializedSize {} rangeListSize {} " + "maxUnackedRangesToPersist {}", ledger.getName(), name, individualDeletedMessages.size(), @@ -3222,9 +3263,6 @@ private List buildBatchEntryDeletio Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { Map.Entry entry = iterator.next(); - nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); - nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); - batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); long[] array = entry.getValue().toLongArray(); List deleteSet = new ArrayList<>(array.length); for (long l : array) { @@ -3240,27 +3278,23 @@ private List buildBatchEntryDeletio } } - private void addAllBatchedEntryDeletionIndexInfo(LightMLDataFormats.PositionInfo lpi) { + private void buildBatchEntryDeletionIndexInfoList( + PositionInfoUtils.BatchedEntryDeletionIndexInfoConsumer consumer) { + if (!getConfig().isDeletionAtBatchIndexLevelEnabled()) { + return; + } + int maxBatchDeletedIndexToPersist = getConfig().getMaxBatchDeletedIndexToPersist(); lock.readLock().lock(); try { if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { return; } - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); int count = 0; - while (iterator.hasNext() && count < getConfig().getMaxBatchDeletedIndexToPersist()) { + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + while (iterator.hasNext() && count < maxBatchDeletedIndexToPersist) { Map.Entry entry = iterator.next(); - - LightMLDataFormats.BatchedEntryDeletionIndexInfo batchInfo = lpi.addBatchedEntryDeletionIndexInfo(); - batchInfo.setPosition() - .setLedgerId(entry.getKey().getLedgerId()) - .setEntryId(entry.getKey().getEntryId()); - long[] array = entry.getValue().toLongArray(); - List deleteSet = new ArrayList<>(array.length); - for (long l : array) { - batchInfo.addDeleteSet(l); - } + consumer.acceptRange(entry.getKey().getLedgerId(), entry.getKey().getEntryId(), array); count++; } } finally { @@ -3279,23 +3313,17 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin long now = System.nanoTime(); Position position = mdEntry.newPosition; - LightMLDataFormats.PositionInfo pi = piThreadLocal.get(); - pi.clear(); - - pi.setLedgerId(position.getLedgerId()) - .setEntryId(position.getEntryId()); - addIndividualDeletedMessageRanges(pi); - addAllBatchedEntryDeletionIndexInfo(pi); - addAllProperties(pi, mdEntry.properties); - if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(), position); } requireNonNull(lh); - ByteBuf rawData = toByteBuf(pi); + ByteBuf rawData = PositionInfoUtils.serializePositionInfo(mdEntry, position, + this::scanIndividualDeletedMessageRanges, this::buildBatchEntryDeletionIndexInfoList, + lastSerializedSize); long endSer = System.nanoTime(); + this.lastSerializedSize = rawData.readableBytes(); // rawData is released by compressDataIfNeeded if needed ByteBuf data = compressDataIfNeeded(rawData, lh); @@ -3366,6 +3394,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin } } + private void writeToBookKeeperLastChunk(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback, @@ -4065,4 +4094,5 @@ public ManagedLedgerInternalStats.CursorStats getCursorStats() { cs.properties = getProperties(); return cs; } + } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java new file mode 100644 index 0000000000000..28643ba797004 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java @@ -0,0 +1,1514 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; + +final class PositionInfoUtils { + + interface IndividuallyDeletedMessagesRangeConsumer { + void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId, long upperEntryId); + } + + interface BatchedEntryDeletionIndexInfoConsumer { + void acceptRange(long ledgerId, long entryId, long[] array); + } + + static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, PositionImpl position, + Consumer rangeScanner, + Consumer batchDeletedIndexesScanner, + int lastSerializedSize) { + int size = Math.max(lastSerializedSize, 64 * 1024); + ByteBuf _b = PulsarByteBufAllocator.DEFAULT.buffer(size); + + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, PositionInfo._LEDGER_ID_TAG); + LightProtoCodec.writeVarInt64(_b, position.getLedgerId()); + LightProtoCodec.writeVarInt(_b, PositionInfo._ENTRY_ID_TAG); + LightProtoCodec.writeVarInt64(_b, position.getEntryId()); + + MessageRange _item = new MessageRange(); + NestedPositionInfo lower = _item.setLowerEndpoint(); + NestedPositionInfo upper = _item.setUpperEndpoint(); + rangeScanner.accept(new IndividuallyDeletedMessagesRangeConsumer() { + @Override + public void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { + lower.setLedgerId(lowerLegerId); + lower.setEntryId(lowerEntryId); + upper.setLedgerId(upperLedgerId); + upper.setEntryId(upperEntryId); + LightProtoCodec.writeVarInt(_b, PositionInfo._INDIVIDUAL_DELETED_MESSAGES_TAG); + LightProtoCodec.writeVarInt(_b, _item.getSerializedSize()); + _item.writeTo(_b); + } + }); + + final LongProperty longProperty = new LongProperty(); + Map properties = mdEntry.properties; + if (properties != null) { + properties.forEach((k, v) -> { + longProperty.setName(k); + longProperty.setValue(v); + LightProtoCodec.writeVarInt(_b, PositionInfo._PROPERTIES_TAG); + LightProtoCodec.writeVarInt(_b, longProperty.getSerializedSize()); + longProperty.writeTo(_b); + }); + } + + final BatchedEntryDeletionIndexInfo batchDeletedIndexInfo = new BatchedEntryDeletionIndexInfo(); + final NestedPositionInfo nestedPositionInfo = batchDeletedIndexInfo.setPosition(); + + batchDeletedIndexesScanner.accept(new BatchedEntryDeletionIndexInfoConsumer() { + @Override + public void acceptRange(long ledgerId, long entryId, long[] array) { + nestedPositionInfo.setLedgerId(ledgerId); + nestedPositionInfo.setEntryId(entryId); + List deleteSet = new ArrayList<>(array.length); + batchDeletedIndexInfo.clearDeleteSet(); + for (long l : array) { + batchDeletedIndexInfo.addDeleteSet(l); + } + LightProtoCodec.writeVarInt(_b, PositionInfo._BATCHED_ENTRY_DELETION_INDEX_INFO_TAG); + LightProtoCodec.writeVarInt(_b, batchDeletedIndexInfo.getSerializedSize()); + batchDeletedIndexInfo.writeTo(_b); + } + }); + + return _b; + } + + public static final class PositionInfo { + private long ledgerId; + private static final int _LEDGER_ID_FIELD_NUMBER = 1; + private static final int _LEDGER_ID_TAG = (_LEDGER_ID_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _LEDGER_ID_TAG_SIZE = LightProtoCodec.computeVarIntSize(_LEDGER_ID_TAG); + private static final int _LEDGER_ID_MASK = 1 << (0 % 32); + public boolean hasLedgerId() { + return (_bitField0 & _LEDGER_ID_MASK) != 0; + } + public long getLedgerId() { + if (!hasLedgerId()) { + throw new IllegalStateException("Field 'ledgerId' is not set"); + } + return ledgerId; + } + public PositionInfo setLedgerId(long ledgerId) { + this.ledgerId = ledgerId; + _bitField0 |= _LEDGER_ID_MASK; + _cachedSize = -1; + return this; + } + public PositionInfo clearLedgerId() { + _bitField0 &= ~_LEDGER_ID_MASK; + return this; + } + + private long entryId; + private static final int _ENTRY_ID_FIELD_NUMBER = 2; + private static final int _ENTRY_ID_TAG = (_ENTRY_ID_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _ENTRY_ID_TAG_SIZE = LightProtoCodec.computeVarIntSize(_ENTRY_ID_TAG); + private static final int _ENTRY_ID_MASK = 1 << (1 % 32); + public boolean hasEntryId() { + return (_bitField0 & _ENTRY_ID_MASK) != 0; + } + public long getEntryId() { + if (!hasEntryId()) { + throw new IllegalStateException("Field 'entryId' is not set"); + } + return entryId; + } + public PositionInfo setEntryId(long entryId) { + this.entryId = entryId; + _bitField0 |= _ENTRY_ID_MASK; + _cachedSize = -1; + return this; + } + public PositionInfo clearEntryId() { + _bitField0 &= ~_ENTRY_ID_MASK; + return this; + } + + private java.util.List individualDeletedMessages = null; + private int _individualDeletedMessagesCount = 0; + private static final int _INDIVIDUAL_DELETED_MESSAGES_FIELD_NUMBER = 3; + private static final int _INDIVIDUAL_DELETED_MESSAGES_TAG = (_INDIVIDUAL_DELETED_MESSAGES_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _INDIVIDUAL_DELETED_MESSAGES_TAG_SIZE = LightProtoCodec + .computeVarIntSize(_INDIVIDUAL_DELETED_MESSAGES_TAG); + public int getIndividualDeletedMessagesCount() { + return _individualDeletedMessagesCount; + } + public MessageRange getIndividualDeletedMessageAt(int idx) { + if (idx < 0 || idx >= _individualDeletedMessagesCount) { + throw new IndexOutOfBoundsException("Index " + idx + " is out of the list size (" + + _individualDeletedMessagesCount + ") for field 'individualDeletedMessages'"); + } + return individualDeletedMessages.get(idx); + } + public java.util.List getIndividualDeletedMessagesList() { + if (_individualDeletedMessagesCount == 0) { + return java.util.Collections.emptyList(); + } else { + return individualDeletedMessages.subList(0, _individualDeletedMessagesCount); + } + } + public MessageRange addIndividualDeletedMessage() { + if (individualDeletedMessages == null) { + individualDeletedMessages = new java.util.ArrayList(); + } + if (individualDeletedMessages.size() == _individualDeletedMessagesCount) { + individualDeletedMessages.add(new MessageRange()); + } + _cachedSize = -1; + return individualDeletedMessages.get(_individualDeletedMessagesCount++); + } + public PositionInfo addAllIndividualDeletedMessages(Iterable individualDeletedMessages) { + for (MessageRange _o : individualDeletedMessages) { + addIndividualDeletedMessage().copyFrom(_o); + } + return this; + } + public PositionInfo clearIndividualDeletedMessages() { + for (int i = 0; i < _individualDeletedMessagesCount; i++) { + individualDeletedMessages.get(i).clear(); + } + _individualDeletedMessagesCount = 0; + return this; + } + + private java.util.List properties = null; + private int _propertiesCount = 0; + private static final int _PROPERTIES_FIELD_NUMBER = 4; + private static final int _PROPERTIES_TAG = (_PROPERTIES_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _PROPERTIES_TAG_SIZE = LightProtoCodec.computeVarIntSize(_PROPERTIES_TAG); + public int getPropertiesCount() { + return _propertiesCount; + } + public LongProperty getPropertyAt(int idx) { + if (idx < 0 || idx >= _propertiesCount) { + throw new IndexOutOfBoundsException( + "Index " + idx + " is out of the list size (" + _propertiesCount + ") for field 'properties'"); + } + return properties.get(idx); + } + public java.util.List getPropertiesList() { + if (_propertiesCount == 0) { + return java.util.Collections.emptyList(); + } else { + return properties.subList(0, _propertiesCount); + } + } + public LongProperty addProperty() { + if (properties == null) { + properties = new java.util.ArrayList(); + } + if (properties.size() == _propertiesCount) { + properties.add(new LongProperty()); + } + _cachedSize = -1; + return properties.get(_propertiesCount++); + } + public PositionInfo addAllProperties(Iterable properties) { + for (LongProperty _o : properties) { + addProperty().copyFrom(_o); + } + return this; + } + public PositionInfo clearProperties() { + for (int i = 0; i < _propertiesCount; i++) { + properties.get(i).clear(); + } + _propertiesCount = 0; + return this; + } + + private java.util.List batchedEntryDeletionIndexInfos = null; + private int _batchedEntryDeletionIndexInfosCount = 0; + private static final int _BATCHED_ENTRY_DELETION_INDEX_INFO_FIELD_NUMBER = 5; + private static final int _BATCHED_ENTRY_DELETION_INDEX_INFO_TAG = (_BATCHED_ENTRY_DELETION_INDEX_INFO_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _BATCHED_ENTRY_DELETION_INDEX_INFO_TAG_SIZE = LightProtoCodec + .computeVarIntSize(_BATCHED_ENTRY_DELETION_INDEX_INFO_TAG); + public int getBatchedEntryDeletionIndexInfosCount() { + return _batchedEntryDeletionIndexInfosCount; + } + public BatchedEntryDeletionIndexInfo getBatchedEntryDeletionIndexInfoAt(int idx) { + if (idx < 0 || idx >= _batchedEntryDeletionIndexInfosCount) { + throw new IndexOutOfBoundsException("Index " + idx + " is out of the list size (" + + _batchedEntryDeletionIndexInfosCount + ") for field 'batchedEntryDeletionIndexInfo'"); + } + return batchedEntryDeletionIndexInfos.get(idx); + } + public java.util.List getBatchedEntryDeletionIndexInfosList() { + if (_batchedEntryDeletionIndexInfosCount == 0) { + return java.util.Collections.emptyList(); + } else { + return batchedEntryDeletionIndexInfos.subList(0, _batchedEntryDeletionIndexInfosCount); + } + } + public BatchedEntryDeletionIndexInfo addBatchedEntryDeletionIndexInfo() { + if (batchedEntryDeletionIndexInfos == null) { + batchedEntryDeletionIndexInfos = new java.util.ArrayList(); + } + if (batchedEntryDeletionIndexInfos.size() == _batchedEntryDeletionIndexInfosCount) { + batchedEntryDeletionIndexInfos.add(new BatchedEntryDeletionIndexInfo()); + } + _cachedSize = -1; + return batchedEntryDeletionIndexInfos.get(_batchedEntryDeletionIndexInfosCount++); + } + public PositionInfo addAllBatchedEntryDeletionIndexInfos( + Iterable batchedEntryDeletionIndexInfos) { + for (BatchedEntryDeletionIndexInfo _o : batchedEntryDeletionIndexInfos) { + addBatchedEntryDeletionIndexInfo().copyFrom(_o); + } + return this; + } + public PositionInfo clearBatchedEntryDeletionIndexInfo() { + for (int i = 0; i < _batchedEntryDeletionIndexInfosCount; i++) { + batchedEntryDeletionIndexInfos.get(i).clear(); + } + _batchedEntryDeletionIndexInfosCount = 0; + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _LEDGER_ID_MASK | _ENTRY_ID_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _LEDGER_ID_TAG); + LightProtoCodec.writeVarInt64(_b, ledgerId); + LightProtoCodec.writeVarInt(_b, _ENTRY_ID_TAG); + LightProtoCodec.writeVarInt64(_b, entryId); + for (int i = 0; i < _individualDeletedMessagesCount; i++) { + MessageRange _item = individualDeletedMessages.get(i); + LightProtoCodec.writeVarInt(_b, _INDIVIDUAL_DELETED_MESSAGES_TAG); + LightProtoCodec.writeVarInt(_b, _item.getSerializedSize()); + _item.writeTo(_b); + } + for (int i = 0; i < _propertiesCount; i++) { + LongProperty _item = properties.get(i); + LightProtoCodec.writeVarInt(_b, _PROPERTIES_TAG); + LightProtoCodec.writeVarInt(_b, _item.getSerializedSize()); + _item.writeTo(_b); + } + for (int i = 0; i < _batchedEntryDeletionIndexInfosCount; i++) { + BatchedEntryDeletionIndexInfo _item = batchedEntryDeletionIndexInfos.get(i); + LightProtoCodec.writeVarInt(_b, _BATCHED_ENTRY_DELETION_INDEX_INFO_TAG); + LightProtoCodec.writeVarInt(_b, _item.getSerializedSize()); + _item.writeTo(_b); + } + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += _LEDGER_ID_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(ledgerId); + _size += _ENTRY_ID_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(entryId); + for (int i = 0; i < _individualDeletedMessagesCount; i++) { + MessageRange _item = individualDeletedMessages.get(i); + _size += _INDIVIDUAL_DELETED_MESSAGES_TAG_SIZE; + int MsgsizeIndividualDeletedMessages = _item.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizeIndividualDeletedMessages) + + MsgsizeIndividualDeletedMessages; + } + for (int i = 0; i < _propertiesCount; i++) { + LongProperty _item = properties.get(i); + _size += _PROPERTIES_TAG_SIZE; + int MsgsizeProperties = _item.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizeProperties) + MsgsizeProperties; + } + for (int i = 0; i < _batchedEntryDeletionIndexInfosCount; i++) { + BatchedEntryDeletionIndexInfo _item = batchedEntryDeletionIndexInfos.get(i); + _size += _BATCHED_ENTRY_DELETION_INDEX_INFO_TAG_SIZE; + int MsgsizeBatchedEntryDeletionIndexInfo = _item.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizeBatchedEntryDeletionIndexInfo) + + MsgsizeBatchedEntryDeletionIndexInfo; + } + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _LEDGER_ID_TAG : + _bitField0 |= _LEDGER_ID_MASK; + ledgerId = LightProtoCodec.readVarInt64(_buffer); + break; + case _ENTRY_ID_TAG : + _bitField0 |= _ENTRY_ID_MASK; + entryId = LightProtoCodec.readVarInt64(_buffer); + break; + case _INDIVIDUAL_DELETED_MESSAGES_TAG : + int _individualDeletedMessagesSize = LightProtoCodec.readVarInt(_buffer); + addIndividualDeletedMessage().parseFrom(_buffer, _individualDeletedMessagesSize); + break; + case _PROPERTIES_TAG : + int _propertiesSize = LightProtoCodec.readVarInt(_buffer); + addProperty().parseFrom(_buffer, _propertiesSize); + break; + case _BATCHED_ENTRY_DELETION_INDEX_INFO_TAG : + int _batchedEntryDeletionIndexInfoSize = LightProtoCodec.readVarInt(_buffer); + addBatchedEntryDeletionIndexInfo().parseFrom(_buffer, _batchedEntryDeletionIndexInfoSize); + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public PositionInfo clear() { + for (int i = 0; i < _individualDeletedMessagesCount; i++) { + individualDeletedMessages.get(i).clear(); + } + _individualDeletedMessagesCount = 0; + for (int i = 0; i < _propertiesCount; i++) { + properties.get(i).clear(); + } + _propertiesCount = 0; + for (int i = 0; i < _batchedEntryDeletionIndexInfosCount; i++) { + batchedEntryDeletionIndexInfos.get(i).clear(); + } + _batchedEntryDeletionIndexInfosCount = 0; + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public PositionInfo copyFrom(PositionInfo _other) { + _cachedSize = -1; + if (_other.hasLedgerId()) { + setLedgerId(_other.ledgerId); + } + if (_other.hasEntryId()) { + setEntryId(_other.entryId); + } + for (int i = 0; i < _other.getIndividualDeletedMessagesCount(); i++) { + addIndividualDeletedMessage().copyFrom(_other.getIndividualDeletedMessageAt(i)); + } + for (int i = 0; i < _other.getPropertiesCount(); i++) { + addProperty().copyFrom(_other.getPropertyAt(i)); + } + for (int i = 0; i < _other.getBatchedEntryDeletionIndexInfosCount(); i++) { + addBatchedEntryDeletionIndexInfo().copyFrom(_other.getBatchedEntryDeletionIndexInfoAt(i)); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + public static final class NestedPositionInfo { + private long ledgerId; + private static final int _LEDGER_ID_FIELD_NUMBER = 1; + private static final int _LEDGER_ID_TAG = (_LEDGER_ID_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _LEDGER_ID_TAG_SIZE = LightProtoCodec.computeVarIntSize(_LEDGER_ID_TAG); + private static final int _LEDGER_ID_MASK = 1 << (0 % 32); + public boolean hasLedgerId() { + return (_bitField0 & _LEDGER_ID_MASK) != 0; + } + public long getLedgerId() { + if (!hasLedgerId()) { + throw new IllegalStateException("Field 'ledgerId' is not set"); + } + return ledgerId; + } + public NestedPositionInfo setLedgerId(long ledgerId) { + this.ledgerId = ledgerId; + _bitField0 |= _LEDGER_ID_MASK; + _cachedSize = -1; + return this; + } + public NestedPositionInfo clearLedgerId() { + _bitField0 &= ~_LEDGER_ID_MASK; + return this; + } + + private long entryId; + private static final int _ENTRY_ID_FIELD_NUMBER = 2; + private static final int _ENTRY_ID_TAG = (_ENTRY_ID_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _ENTRY_ID_TAG_SIZE = LightProtoCodec.computeVarIntSize(_ENTRY_ID_TAG); + private static final int _ENTRY_ID_MASK = 1 << (1 % 32); + public boolean hasEntryId() { + return (_bitField0 & _ENTRY_ID_MASK) != 0; + } + public long getEntryId() { + if (!hasEntryId()) { + throw new IllegalStateException("Field 'entryId' is not set"); + } + return entryId; + } + public NestedPositionInfo setEntryId(long entryId) { + this.entryId = entryId; + _bitField0 |= _ENTRY_ID_MASK; + _cachedSize = -1; + return this; + } + public NestedPositionInfo clearEntryId() { + _bitField0 &= ~_ENTRY_ID_MASK; + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _LEDGER_ID_MASK | _ENTRY_ID_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _LEDGER_ID_TAG); + LightProtoCodec.writeVarInt64(_b, ledgerId); + LightProtoCodec.writeVarInt(_b, _ENTRY_ID_TAG); + LightProtoCodec.writeVarInt64(_b, entryId); + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += _LEDGER_ID_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(ledgerId); + _size += _ENTRY_ID_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(entryId); + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _LEDGER_ID_TAG : + _bitField0 |= _LEDGER_ID_MASK; + ledgerId = LightProtoCodec.readVarInt64(_buffer); + break; + case _ENTRY_ID_TAG : + _bitField0 |= _ENTRY_ID_MASK; + entryId = LightProtoCodec.readVarInt64(_buffer); + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public NestedPositionInfo clear() { + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public NestedPositionInfo copyFrom(NestedPositionInfo _other) { + _cachedSize = -1; + if (_other.hasLedgerId()) { + setLedgerId(_other.ledgerId); + } + if (_other.hasEntryId()) { + setEntryId(_other.entryId); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + public static final class MessageRange { + private NestedPositionInfo lowerEndpoint; + private static final int _LOWER_ENDPOINT_FIELD_NUMBER = 1; + private static final int _LOWER_ENDPOINT_TAG = (_LOWER_ENDPOINT_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _LOWER_ENDPOINT_TAG_SIZE = LightProtoCodec.computeVarIntSize(_LOWER_ENDPOINT_TAG); + private static final int _LOWER_ENDPOINT_MASK = 1 << (0 % 32); + public boolean hasLowerEndpoint() { + return (_bitField0 & _LOWER_ENDPOINT_MASK) != 0; + } + public NestedPositionInfo getLowerEndpoint() { + if (!hasLowerEndpoint()) { + throw new IllegalStateException("Field 'lowerEndpoint' is not set"); + } + return lowerEndpoint; + } + public NestedPositionInfo setLowerEndpoint() { + if (lowerEndpoint == null) { + lowerEndpoint = new NestedPositionInfo(); + } + _bitField0 |= _LOWER_ENDPOINT_MASK; + _cachedSize = -1; + return lowerEndpoint; + } + public MessageRange clearLowerEndpoint() { + _bitField0 &= ~_LOWER_ENDPOINT_MASK; + if (hasLowerEndpoint()) { + lowerEndpoint.clear(); + } + return this; + } + + private NestedPositionInfo upperEndpoint; + private static final int _UPPER_ENDPOINT_FIELD_NUMBER = 2; + private static final int _UPPER_ENDPOINT_TAG = (_UPPER_ENDPOINT_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _UPPER_ENDPOINT_TAG_SIZE = LightProtoCodec.computeVarIntSize(_UPPER_ENDPOINT_TAG); + private static final int _UPPER_ENDPOINT_MASK = 1 << (1 % 32); + public boolean hasUpperEndpoint() { + return (_bitField0 & _UPPER_ENDPOINT_MASK) != 0; + } + public NestedPositionInfo getUpperEndpoint() { + if (!hasUpperEndpoint()) { + throw new IllegalStateException("Field 'upperEndpoint' is not set"); + } + return upperEndpoint; + } + public NestedPositionInfo setUpperEndpoint() { + if (upperEndpoint == null) { + upperEndpoint = new NestedPositionInfo(); + } + _bitField0 |= _UPPER_ENDPOINT_MASK; + _cachedSize = -1; + return upperEndpoint; + } + public MessageRange clearUpperEndpoint() { + _bitField0 &= ~_UPPER_ENDPOINT_MASK; + if (hasUpperEndpoint()) { + upperEndpoint.clear(); + } + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _LOWER_ENDPOINT_MASK | _UPPER_ENDPOINT_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _LOWER_ENDPOINT_TAG); + LightProtoCodec.writeVarInt(_b, lowerEndpoint.getSerializedSize()); + lowerEndpoint.writeTo(_b); + LightProtoCodec.writeVarInt(_b, _UPPER_ENDPOINT_TAG); + LightProtoCodec.writeVarInt(_b, upperEndpoint.getSerializedSize()); + upperEndpoint.writeTo(_b); + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += LightProtoCodec.computeVarIntSize(_LOWER_ENDPOINT_TAG); + int MsgsizeLowerEndpoint = lowerEndpoint.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizeLowerEndpoint) + MsgsizeLowerEndpoint; + _size += LightProtoCodec.computeVarIntSize(_UPPER_ENDPOINT_TAG); + int MsgsizeUpperEndpoint = upperEndpoint.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizeUpperEndpoint) + MsgsizeUpperEndpoint; + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _LOWER_ENDPOINT_TAG : + _bitField0 |= _LOWER_ENDPOINT_MASK; + int lowerEndpointSize = LightProtoCodec.readVarInt(_buffer); + setLowerEndpoint().parseFrom(_buffer, lowerEndpointSize); + break; + case _UPPER_ENDPOINT_TAG : + _bitField0 |= _UPPER_ENDPOINT_MASK; + int upperEndpointSize = LightProtoCodec.readVarInt(_buffer); + setUpperEndpoint().parseFrom(_buffer, upperEndpointSize); + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public MessageRange clear() { + if (hasLowerEndpoint()) { + lowerEndpoint.clear(); + } + if (hasUpperEndpoint()) { + upperEndpoint.clear(); + } + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public MessageRange copyFrom(MessageRange _other) { + _cachedSize = -1; + if (_other.hasLowerEndpoint()) { + setLowerEndpoint().copyFrom(_other.lowerEndpoint); + } + if (_other.hasUpperEndpoint()) { + setUpperEndpoint().copyFrom(_other.upperEndpoint); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + public static final class BatchedEntryDeletionIndexInfo { + private NestedPositionInfo position; + private static final int _POSITION_FIELD_NUMBER = 1; + private static final int _POSITION_TAG = (_POSITION_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _POSITION_TAG_SIZE = LightProtoCodec.computeVarIntSize(_POSITION_TAG); + private static final int _POSITION_MASK = 1 << (0 % 32); + public boolean hasPosition() { + return (_bitField0 & _POSITION_MASK) != 0; + } + public NestedPositionInfo getPosition() { + if (!hasPosition()) { + throw new IllegalStateException("Field 'position' is not set"); + } + return position; + } + public NestedPositionInfo setPosition() { + if (position == null) { + position = new NestedPositionInfo(); + } + _bitField0 |= _POSITION_MASK; + _cachedSize = -1; + return position; + } + public BatchedEntryDeletionIndexInfo clearPosition() { + _bitField0 &= ~_POSITION_MASK; + if (hasPosition()) { + position.clear(); + } + return this; + } + + private long[] deleteSets = null; + private int _deleteSetsCount = 0; + private static final int _DELETE_SET_FIELD_NUMBER = 2; + private static final int _DELETE_SET_TAG = (_DELETE_SET_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _DELETE_SET_TAG_SIZE = LightProtoCodec.computeVarIntSize(_DELETE_SET_TAG); + private static final int _DELETE_SET_TAG_PACKED = (_DELETE_SET_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + public int getDeleteSetsCount() { + return _deleteSetsCount; + } + public long getDeleteSetAt(int idx) { + if (idx < 0 || idx >= _deleteSetsCount) { + throw new IndexOutOfBoundsException( + "Index " + idx + " is out of the list size (" + _deleteSetsCount + ") for field 'deleteSet'"); + } + return deleteSets[idx]; + } + public void addDeleteSet(long deleteSet) { + if (deleteSets == null) { + deleteSets = new long[4]; + } + if (deleteSets.length == _deleteSetsCount) { + deleteSets = java.util.Arrays.copyOf(deleteSets, _deleteSetsCount * 2); + } + _cachedSize = -1; + deleteSets[_deleteSetsCount++] = deleteSet; + } + public BatchedEntryDeletionIndexInfo clearDeleteSet() { + _deleteSetsCount = 0; + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _POSITION_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _POSITION_TAG); + LightProtoCodec.writeVarInt(_b, position.getSerializedSize()); + position.writeTo(_b); + for (int i = 0; i < _deleteSetsCount; i++) { + long _item = deleteSets[i]; + LightProtoCodec.writeVarInt(_b, _DELETE_SET_TAG); + LightProtoCodec.writeVarInt64(_b, _item); + } + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += LightProtoCodec.computeVarIntSize(_POSITION_TAG); + int MsgsizePosition = position.getSerializedSize(); + _size += LightProtoCodec.computeVarIntSize(MsgsizePosition) + MsgsizePosition; + for (int i = 0; i < _deleteSetsCount; i++) { + long _item = deleteSets[i]; + _size += _DELETE_SET_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(_item); + } + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _POSITION_TAG : + _bitField0 |= _POSITION_MASK; + int positionSize = LightProtoCodec.readVarInt(_buffer); + setPosition().parseFrom(_buffer, positionSize); + break; + case _DELETE_SET_TAG : + addDeleteSet(LightProtoCodec.readVarInt64(_buffer)); + break; + case _DELETE_SET_TAG_PACKED : + int _deleteSetSize = LightProtoCodec.readVarInt(_buffer); + int _deleteSetEndIdx = _buffer.readerIndex() + _deleteSetSize; + while (_buffer.readerIndex() < _deleteSetEndIdx) { + addDeleteSet(LightProtoCodec.readVarInt64(_buffer)); + } + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public BatchedEntryDeletionIndexInfo clear() { + if (hasPosition()) { + position.clear(); + } + _deleteSetsCount = 0; + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public BatchedEntryDeletionIndexInfo copyFrom( + BatchedEntryDeletionIndexInfo _other) { + _cachedSize = -1; + if (_other.hasPosition()) { + setPosition().copyFrom(_other.position); + } + for (int i = 0; i < _other.getDeleteSetsCount(); i++) { + addDeleteSet(_other.getDeleteSetAt(i)); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + public static final class LongProperty { + private String name; + private int _nameBufferIdx = -1; + private int _nameBufferLen = -1; + private static final int _NAME_FIELD_NUMBER = 1; + private static final int _NAME_TAG = (_NAME_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _NAME_TAG_SIZE = LightProtoCodec.computeVarIntSize(_NAME_TAG); + private static final int _NAME_MASK = 1 << (0 % 32); + public boolean hasName() { + return (_bitField0 & _NAME_MASK) != 0; + } + public String getName() { + if (!hasName()) { + throw new IllegalStateException("Field 'name' is not set"); + } + if (name == null) { + name = LightProtoCodec.readString(_parsedBuffer, _nameBufferIdx, _nameBufferLen); + } + return name; + } + public LongProperty setName(String name) { + this.name = name; + _bitField0 |= _NAME_MASK; + _nameBufferIdx = -1; + _nameBufferLen = LightProtoCodec.computeStringUTF8Size(name); + _cachedSize = -1; + return this; + } + public LongProperty clearName() { + _bitField0 &= ~_NAME_MASK; + name = null; + _nameBufferIdx = -1; + _nameBufferLen = -1; + return this; + } + + private long value; + private static final int _VALUE_FIELD_NUMBER = 2; + private static final int _VALUE_TAG = (_VALUE_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_VARINT; + private static final int _VALUE_TAG_SIZE = LightProtoCodec.computeVarIntSize(_VALUE_TAG); + private static final int _VALUE_MASK = 1 << (1 % 32); + public boolean hasValue() { + return (_bitField0 & _VALUE_MASK) != 0; + } + public long getValue() { + if (!hasValue()) { + throw new IllegalStateException("Field 'value' is not set"); + } + return value; + } + public LongProperty setValue(long value) { + this.value = value; + _bitField0 |= _VALUE_MASK; + _cachedSize = -1; + return this; + } + public LongProperty clearValue() { + _bitField0 &= ~_VALUE_MASK; + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _NAME_MASK | _VALUE_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _NAME_TAG); + LightProtoCodec.writeVarInt(_b, _nameBufferLen); + if (_nameBufferIdx == -1) { + LightProtoCodec.writeString(_b, name, _nameBufferLen); + } else { + _parsedBuffer.getBytes(_nameBufferIdx, _b, _nameBufferLen); + } + LightProtoCodec.writeVarInt(_b, _VALUE_TAG); + LightProtoCodec.writeVarInt64(_b, value); + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += _NAME_TAG_SIZE; + _size += LightProtoCodec.computeVarIntSize(_nameBufferLen); + _size += _nameBufferLen; + _size += _VALUE_TAG_SIZE; + _size += LightProtoCodec.computeVarInt64Size(value); + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _NAME_TAG : + _bitField0 |= _NAME_MASK; + _nameBufferLen = LightProtoCodec.readVarInt(_buffer); + _nameBufferIdx = _buffer.readerIndex(); + _buffer.skipBytes(_nameBufferLen); + break; + case _VALUE_TAG : + _bitField0 |= _VALUE_MASK; + value = LightProtoCodec.readVarInt64(_buffer); + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public LongProperty clear() { + name = null; + _nameBufferIdx = -1; + _nameBufferLen = -1; + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public LongProperty copyFrom(LongProperty _other) { + _cachedSize = -1; + if (_other.hasName()) { + setName(_other.getName()); + } + if (_other.hasValue()) { + setValue(_other.value); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + public static final class StringProperty { + private String name; + private int _nameBufferIdx = -1; + private int _nameBufferLen = -1; + private static final int _NAME_FIELD_NUMBER = 1; + private static final int _NAME_TAG = (_NAME_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _NAME_TAG_SIZE = LightProtoCodec.computeVarIntSize(_NAME_TAG); + private static final int _NAME_MASK = 1 << (0 % 32); + public boolean hasName() { + return (_bitField0 & _NAME_MASK) != 0; + } + public String getName() { + if (!hasName()) { + throw new IllegalStateException("Field 'name' is not set"); + } + if (name == null) { + name = LightProtoCodec.readString(_parsedBuffer, _nameBufferIdx, _nameBufferLen); + } + return name; + } + public StringProperty setName(String name) { + this.name = name; + _bitField0 |= _NAME_MASK; + _nameBufferIdx = -1; + _nameBufferLen = LightProtoCodec.computeStringUTF8Size(name); + _cachedSize = -1; + return this; + } + public StringProperty clearName() { + _bitField0 &= ~_NAME_MASK; + name = null; + _nameBufferIdx = -1; + _nameBufferLen = -1; + return this; + } + + private String value; + private int _valueBufferIdx = -1; + private int _valueBufferLen = -1; + private static final int _VALUE_FIELD_NUMBER = 2; + private static final int _VALUE_TAG = (_VALUE_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS) + | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED; + private static final int _VALUE_TAG_SIZE = LightProtoCodec.computeVarIntSize(_VALUE_TAG); + private static final int _VALUE_MASK = 1 << (1 % 32); + public boolean hasValue() { + return (_bitField0 & _VALUE_MASK) != 0; + } + public String getValue() { + if (!hasValue()) { + throw new IllegalStateException("Field 'value' is not set"); + } + if (value == null) { + value = LightProtoCodec.readString(_parsedBuffer, _valueBufferIdx, _valueBufferLen); + } + return value; + } + public StringProperty setValue(String value) { + this.value = value; + _bitField0 |= _VALUE_MASK; + _valueBufferIdx = -1; + _valueBufferLen = LightProtoCodec.computeStringUTF8Size(value); + _cachedSize = -1; + return this; + } + public StringProperty clearValue() { + _bitField0 &= ~_VALUE_MASK; + value = null; + _valueBufferIdx = -1; + _valueBufferLen = -1; + return this; + } + + private int _bitField0; + private static final int _REQUIRED_FIELDS_MASK0 = 0 | _NAME_MASK | _VALUE_MASK; + public int writeTo(io.netty.buffer.ByteBuf _b) { + checkRequiredFields(); + int _writeIdx = _b.writerIndex(); + LightProtoCodec.writeVarInt(_b, _NAME_TAG); + LightProtoCodec.writeVarInt(_b, _nameBufferLen); + if (_nameBufferIdx == -1) { + LightProtoCodec.writeString(_b, name, _nameBufferLen); + } else { + _parsedBuffer.getBytes(_nameBufferIdx, _b, _nameBufferLen); + } + LightProtoCodec.writeVarInt(_b, _VALUE_TAG); + LightProtoCodec.writeVarInt(_b, _valueBufferLen); + if (_valueBufferIdx == -1) { + LightProtoCodec.writeString(_b, value, _valueBufferLen); + } else { + _parsedBuffer.getBytes(_valueBufferIdx, _b, _valueBufferLen); + } + return (_b.writerIndex() - _writeIdx); + } + public int getSerializedSize() { + if (_cachedSize > -1) { + return _cachedSize; + } + + int _size = 0; + _size += _NAME_TAG_SIZE; + _size += LightProtoCodec.computeVarIntSize(_nameBufferLen); + _size += _nameBufferLen; + _size += _VALUE_TAG_SIZE; + _size += LightProtoCodec.computeVarIntSize(_valueBufferLen); + _size += _valueBufferLen; + _cachedSize = _size; + return _size; + } + public void parseFrom(io.netty.buffer.ByteBuf _buffer, int _size) { + clear(); + int _endIdx = _buffer.readerIndex() + _size; + while (_buffer.readerIndex() < _endIdx) { + int _tag = LightProtoCodec.readVarInt(_buffer); + switch (_tag) { + case _NAME_TAG : + _bitField0 |= _NAME_MASK; + _nameBufferLen = LightProtoCodec.readVarInt(_buffer); + _nameBufferIdx = _buffer.readerIndex(); + _buffer.skipBytes(_nameBufferLen); + break; + case _VALUE_TAG : + _bitField0 |= _VALUE_MASK; + _valueBufferLen = LightProtoCodec.readVarInt(_buffer); + _valueBufferIdx = _buffer.readerIndex(); + _buffer.skipBytes(_valueBufferLen); + break; + default : + LightProtoCodec.skipUnknownField(_tag, _buffer); + } + } + checkRequiredFields(); + _parsedBuffer = _buffer; + } + private void checkRequiredFields() { + if ((_bitField0 & _REQUIRED_FIELDS_MASK0) != _REQUIRED_FIELDS_MASK0) { + throw new IllegalStateException("Some required fields are missing"); + } + } + public StringProperty clear() { + name = null; + _nameBufferIdx = -1; + _nameBufferLen = -1; + value = null; + _valueBufferIdx = -1; + _valueBufferLen = -1; + _parsedBuffer = null; + _cachedSize = -1; + _bitField0 = 0; + return this; + } + public StringProperty copyFrom(StringProperty _other) { + _cachedSize = -1; + if (_other.hasName()) { + setName(_other.getName()); + } + if (_other.hasValue()) { + setValue(_other.getValue()); + } + return this; + } + public byte[] toByteArray() { + byte[] a = new byte[getSerializedSize()]; + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a).writerIndex(0); + this.writeTo(b); + return a; + } + public void parseFrom(byte[] a) { + io.netty.buffer.ByteBuf b = io.netty.buffer.Unpooled.wrappedBuffer(a); + this.parseFrom(b, b.readableBytes()); + } + private int _cachedSize; + + private io.netty.buffer.ByteBuf _parsedBuffer; + + } + + + + static final class LightProtoCodec { + static final int TAG_TYPE_MASK = 7; + static final int TAG_TYPE_BITS = 3; + static final int WIRETYPE_VARINT = 0; + static final int WIRETYPE_FIXED64 = 1; + static final int WIRETYPE_LENGTH_DELIMITED = 2; + static final int WIRETYPE_START_GROUP = 3; + static final int WIRETYPE_END_GROUP = 4; + static final int WIRETYPE_FIXED32 = 5; + private LightProtoCodec() { + } + + private static int getTagType(int tag) { + return tag & TAG_TYPE_MASK; + } + + static int getFieldId(int tag) { + return tag >>> TAG_TYPE_BITS; + } + + static void writeVarInt(ByteBuf b, int n) { + if (n >= 0) { + _writeVarInt(b, n); + } else { + writeVarInt64(b, n); + } + } + + static void writeSignedVarInt(ByteBuf b, int n) { + writeVarInt(b, encodeZigZag32(n)); + } + + static int readSignedVarInt(ByteBuf b) { + return decodeZigZag32(readVarInt(b)); + } + + static long readSignedVarInt64(ByteBuf b) { + return decodeZigZag64(readVarInt64(b)); + } + + static void writeFloat(ByteBuf b, float n) { + writeFixedInt32(b, Float.floatToRawIntBits(n)); + } + + static void writeDouble(ByteBuf b, double n) { + writeFixedInt64(b, Double.doubleToRawLongBits(n)); + } + + static float readFloat(ByteBuf b) { + return Float.intBitsToFloat(readFixedInt32(b)); + } + + static double readDouble(ByteBuf b) { + return Double.longBitsToDouble(readFixedInt64(b)); + } + + private static void _writeVarInt(ByteBuf b, int n) { + while (true) { + if ((n & ~0x7F) == 0) { + b.writeByte(n); + return; + } else { + b.writeByte((n & 0x7F) | 0x80); + n >>>= 7; + } + } + } + + static void writeVarInt64(ByteBuf b, long value) { + while (true) { + if ((value & ~0x7FL) == 0) { + b.writeByte((int) value); + return; + } else { + b.writeByte(((int) value & 0x7F) | 0x80); + value >>>= 7; + } + } + } + + static void writeFixedInt32(ByteBuf b, int n) { + b.writeIntLE(n); + } + + static void writeFixedInt64(ByteBuf b, long n) { + b.writeLongLE(n); + } + + static int readFixedInt32(ByteBuf b) { + return b.readIntLE(); + } + + static long readFixedInt64(ByteBuf b) { + return b.readLongLE(); + } + + static void writeSignedVarInt64(ByteBuf b, long n) { + writeVarInt64(b, encodeZigZag64(n)); + } + + private static int encodeZigZag32(final int n) { + return (n << 1) ^ (n >> 31); + } + + private static long encodeZigZag64(final long n) { + return (n << 1) ^ (n >> 63); + } + + private static int decodeZigZag32(int n) { + return n >>> 1 ^ -(n & 1); + } + + private static long decodeZigZag64(long n) { + return n >>> 1 ^ -(n & 1L); + } + + static int readVarInt(ByteBuf buf) { + byte tmp = buf.readByte(); + if (tmp >= 0) { + return tmp; + } + int result = tmp & 0x7f; + if ((tmp = buf.readByte()) >= 0) { + result |= tmp << 7; + } else { + result |= (tmp & 0x7f) << 7; + if ((tmp = buf.readByte()) >= 0) { + result |= tmp << 14; + } else { + result |= (tmp & 0x7f) << 14; + if ((tmp = buf.readByte()) >= 0) { + result |= tmp << 21; + } else { + result |= (tmp & 0x7f) << 21; + result |= (tmp = buf.readByte()) << 28; + if (tmp < 0) { + // Discard upper 32 bits. + for (int i = 0; i < 5; i++) { + if (buf.readByte() >= 0) { + return result; + } + } + throw new IllegalArgumentException("Encountered a malformed varint."); + } + } + } + } + return result; + } + + static long readVarInt64(ByteBuf buf) { + int shift = 0; + long result = 0; + while (shift < 64) { + final byte b = buf.readByte(); + result |= (long) (b & 0x7F) << shift; + if ((b & 0x80) == 0) { + return result; + } + shift += 7; + } + throw new IllegalArgumentException("Encountered a malformed varint."); + } + + static int computeSignedVarIntSize(final int value) { + return computeVarUIntSize(encodeZigZag32(value)); + } + + static int computeSignedVarInt64Size(final long value) { + return computeVarInt64Size(encodeZigZag64(value)); + } + + static int computeVarIntSize(final int value) { + if (value < 0) { + return 10; + } else { + return computeVarUIntSize(value); + } + } + + static int computeVarUIntSize(final int value) { + if ((value & (0xffffffff << 7)) == 0) { + return 1; + } else if ((value & (0xffffffff << 14)) == 0) { + return 2; + } else if ((value & (0xffffffff << 21)) == 0) { + return 3; + } else if ((value & (0xffffffff << 28)) == 0) { + return 4; + } else { + return 5; + } + } + + static int computeVarInt64Size(final long value) { + if ((value & (0xffffffffffffffffL << 7)) == 0) { + return 1; + } else if ((value & (0xffffffffffffffffL << 14)) == 0) { + return 2; + } else if ((value & (0xffffffffffffffffL << 21)) == 0) { + return 3; + } else if ((value & (0xffffffffffffffffL << 28)) == 0) { + return 4; + } else if ((value & (0xffffffffffffffffL << 35)) == 0) { + return 5; + } else if ((value & (0xffffffffffffffffL << 42)) == 0) { + return 6; + } else if ((value & (0xffffffffffffffffL << 49)) == 0) { + return 7; + } else if ((value & (0xffffffffffffffffL << 56)) == 0) { + return 8; + } else if ((value & (0xffffffffffffffffL << 63)) == 0) { + return 9; + } else { + return 10; + } + } + + static int computeStringUTF8Size(String s) { + return ByteBufUtil.utf8Bytes(s); + } + + static void writeString(ByteBuf b, String s, int bytesCount) { + ByteBufUtil.reserveAndWriteUtf8(b, s, bytesCount); + } + + static String readString(ByteBuf b, int index, int len) { + return b.toString(index, len, StandardCharsets.UTF_8); + } + + static void skipUnknownField(int tag, ByteBuf buffer) { + int tagType = getTagType(tag); + switch (tagType) { + case WIRETYPE_VARINT : + readVarInt(buffer); + break; + case WIRETYPE_FIXED64 : + buffer.skipBytes(8); + break; + case WIRETYPE_LENGTH_DELIMITED : + int len = readVarInt(buffer); + buffer.skipBytes(len); + break; + case WIRETYPE_FIXED32 : + buffer.skipBytes(4); + break; + default : + throw new IllegalArgumentException("Invalid unknonwn tag type: " + tagType); + } + } + + static final class StringHolder { + String s; + int idx; + int len; + } + + static final class BytesHolder { + ByteBuf b; + int idx; + int len; + } + } + +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java new file mode 100644 index 0000000000000..84dfa6277622f --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.mledger.impl; + +import static org.testng.Assert.*; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufUtil; +import java.util.Map; +import java.util.List; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.testng.annotations.Test; + +public class PositionInfoUtilsTest { + @Test + public void testSerializeDeserialize() throws Exception { + PositionImpl position = new PositionImpl(1, 2); + ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, + Map.of("foo", 1L), null, null); + + ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { + scanner.acceptRange(1, 2, 3, 4); + scanner.acceptRange(5, 6, 7, 8); + }, (scanner) -> { + long[] array = {7L, 8L}; + scanner.acceptRange(1, 2, array); + }); + + byte[] data = ByteBufUtil.getBytes(result); + MLDataFormats.PositionInfo positionInfoParsed = MLDataFormats.PositionInfo.parseFrom(data); + assertEquals(1, positionInfoParsed.getLedgerId()); + assertEquals(2, positionInfoParsed.getEntryId()); + + assertEquals(1, positionInfoParsed.getPropertiesCount()); + assertEquals("foo", positionInfoParsed.getProperties(0).getName()); + assertEquals(1, positionInfoParsed.getProperties(0).getValue()); + + assertEquals(2, positionInfoParsed.getIndividualDeletedMessagesCount()); + assertEquals(1, positionInfoParsed.getIndividualDeletedMessages(0).getLowerEndpoint().getLedgerId()); + assertEquals(2, positionInfoParsed.getIndividualDeletedMessages(0).getLowerEndpoint().getEntryId()); + assertEquals(3, positionInfoParsed.getIndividualDeletedMessages(0).getUpperEndpoint().getLedgerId()); + assertEquals(4, positionInfoParsed.getIndividualDeletedMessages(0).getUpperEndpoint().getEntryId()); + + assertEquals(5, positionInfoParsed.getIndividualDeletedMessages(1).getLowerEndpoint().getLedgerId()); + assertEquals(6, positionInfoParsed.getIndividualDeletedMessages(1).getLowerEndpoint().getEntryId()); + assertEquals(7, positionInfoParsed.getIndividualDeletedMessages(1).getUpperEndpoint().getLedgerId()); + assertEquals(8, positionInfoParsed.getIndividualDeletedMessages(1).getUpperEndpoint().getEntryId()); + + assertEquals(1, positionInfoParsed.getBatchedEntryDeletionIndexInfoCount()); + assertEquals(1, positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getPosition().getLedgerId()); + assertEquals(2, positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getPosition().getEntryId()); + assertEquals(List.of(7L, 8L), positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getDeleteSetList()); + result.release(); + } + + @Test + public void testSerializeDeserializeEmpty() throws Exception { + PositionImpl position = new PositionImpl(1, 2); + ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, + null, null, null); + + ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { + }, (scanner) -> { + }); + + byte[] data = ByteBufUtil.getBytes(result); + MLDataFormats.PositionInfo positionInfoParsed = MLDataFormats.PositionInfo.parseFrom(data); + assertEquals(1, positionInfoParsed.getLedgerId()); + assertEquals(2, positionInfoParsed.getEntryId()); + + assertEquals(0, positionInfoParsed.getPropertiesCount()); + assertEquals(0, positionInfoParsed.getIndividualDeletedMessagesCount()); + assertEquals(0, positionInfoParsed.getBatchedEntryDeletionIndexInfoCount()); + result.release(); + } +} \ No newline at end of file From 02402509f4db0ecb88fd96cfd1e4a1e98e8897cb Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Thu, 16 May 2024 17:14:29 +0200 Subject: [PATCH 05/22] Fix PositionInfoUtilsTest (cherry picked from commit 44ba614adc3a747ebf2715c5efb6a206ad9b4352) --- .../apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java index 84dfa6277622f..39aa2d6299a8a 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java @@ -39,7 +39,7 @@ public void testSerializeDeserialize() throws Exception { }, (scanner) -> { long[] array = {7L, 8L}; scanner.acceptRange(1, 2, array); - }); + }, 1024); byte[] data = ByteBufUtil.getBytes(result); MLDataFormats.PositionInfo positionInfoParsed = MLDataFormats.PositionInfo.parseFrom(data); @@ -76,7 +76,7 @@ public void testSerializeDeserializeEmpty() throws Exception { ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { }, (scanner) -> { - }); + }, 1024); byte[] data = ByteBufUtil.getBytes(result); MLDataFormats.PositionInfo positionInfoParsed = MLDataFormats.PositionInfo.parseFrom(data); From ed8df4d5376021823eda2c8c5c553e82fc08c548 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Thu, 16 May 2024 23:37:50 -0700 Subject: [PATCH 06/22] PositionInfo Util serialization fix and test (#272) (cherry picked from commit f1323c646862b14db3c2843d8c6146656aec7eec) --- .../mledger/impl/ManagedCursorImpl.java | 22 +--- .../mledger/impl/PositionInfoUtils.java | 16 +-- .../mledger/impl/PositionInfoUtilsTest.java | 123 ++++++++++++++++++ 3 files changed, 131 insertions(+), 30 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f72a02289ff56..f1f1b106dbd00 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 @@ -38,7 +38,6 @@ import io.netty.buffer.ByteBufUtil; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import io.netty.util.concurrent.FastThreadLocal; import java.io.ByteArrayInputStream; import java.io.DataInputStream; import java.io.IOException; @@ -133,13 +132,6 @@ @SuppressWarnings("checkstyle:javadoctype") public class ManagedCursorImpl implements ManagedCursor { - private static final FastThreadLocal piThreadLocal = new FastThreadLocal<>() { - @Override - protected LightMLDataFormats.PositionInfo initialValue() { - return new LightMLDataFormats.PositionInfo(); - } - }; - private static final Comparator ENTRY_COMPARATOR = (e1, e2) -> { if (e1.getLedgerId() != e2.getLedgerId()) { return e1.getLedgerId() < e2.getLedgerId() ? -1 : 1; @@ -493,7 +485,7 @@ public boolean removeProperty(String key) { if (lastMarkDeleteEntry != null) { LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> { Map properties = last.properties; - if (properties != null && properties.containsKey(key)) { + if (properties != null) { properties.remove(key); } return last; @@ -2070,7 +2062,7 @@ public void asyncMarkDelete(final Position position, Map propertie } callback.markDeleteFailed( new ManagedLedgerException("Reset cursor in progress - unable to mark delete position " - + position.toString()), + + position), ctx); return; } @@ -3302,13 +3294,6 @@ private void buildBatchEntryDeletionIndexInfoList( } } - private static ByteBuf toByteBuf(LightMLDataFormats.PositionInfo pi) { - int size = pi.getSerializedSize(); - ByteBuf buf = PulsarByteBufAllocator.DEFAULT.buffer(size, size); - pi.writeTo(buf); - return buf; - } - void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { long now = System.nanoTime(); Position position = mdEntry.newPosition; @@ -3547,7 +3532,7 @@ boolean shouldCloseLedger(LedgerHandle lh) { long now = clock.millis(); if (ledger.getFactory().isMetadataServiceAvailable() && (lh.getLastAddConfirmed() >= getConfig().getMetadataMaxEntriesPerLedger() - || lastLedgerSwitchTimestamp < (now - getConfig().getLedgerRolloverTimeout() * 1000)) + || lastLedgerSwitchTimestamp < (now - getConfig().getLedgerRolloverTimeout() * 1000L)) && (STATE_UPDATER.get(this) != State.Closed && STATE_UPDATER.get(this) != State.Closing)) { // It's safe to modify the timestamp since this method will be only called from a callback, implying that // calls will be serialized on one single thread @@ -3686,7 +3671,6 @@ private void asyncDeleteLedger(final LedgerHandle lh, int retry) { ledger.getScheduledExecutor().schedule(() -> asyncDeleteLedger(lh, retry - 1), DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC, TimeUnit.SECONDS); } - return; } else { log.info("[{}][{}] Successfully closed & deleted ledger {} in cursor", ledger.getName(), name, lh.getId()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java index 28643ba797004..2cf68ac92b57d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java @@ -21,14 +21,9 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.function.Consumer; -import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; -import org.apache.pulsar.common.util.collections.BitSetRecyclable; final class PositionInfoUtils { @@ -47,18 +42,18 @@ static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, int size = Math.max(lastSerializedSize, 64 * 1024); ByteBuf _b = PulsarByteBufAllocator.DEFAULT.buffer(size); - int _writeIdx = _b.writerIndex(); LightProtoCodec.writeVarInt(_b, PositionInfo._LEDGER_ID_TAG); LightProtoCodec.writeVarInt64(_b, position.getLedgerId()); LightProtoCodec.writeVarInt(_b, PositionInfo._ENTRY_ID_TAG); LightProtoCodec.writeVarInt64(_b, position.getEntryId()); MessageRange _item = new MessageRange(); - NestedPositionInfo lower = _item.setLowerEndpoint(); - NestedPositionInfo upper = _item.setUpperEndpoint(); rangeScanner.accept(new IndividuallyDeletedMessagesRangeConsumer() { @Override public void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { + _item.clear(); + NestedPositionInfo lower = _item.setLowerEndpoint(); + NestedPositionInfo upper = _item.setUpperEndpoint(); lower.setLedgerId(lowerLegerId); lower.setEntryId(lowerEntryId); upper.setLedgerId(upperLedgerId); @@ -82,15 +77,14 @@ public void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId } final BatchedEntryDeletionIndexInfo batchDeletedIndexInfo = new BatchedEntryDeletionIndexInfo(); - final NestedPositionInfo nestedPositionInfo = batchDeletedIndexInfo.setPosition(); batchDeletedIndexesScanner.accept(new BatchedEntryDeletionIndexInfoConsumer() { @Override public void acceptRange(long ledgerId, long entryId, long[] array) { + batchDeletedIndexInfo.clear(); + final NestedPositionInfo nestedPositionInfo = batchDeletedIndexInfo.setPosition(); nestedPositionInfo.setLedgerId(ledgerId); nestedPositionInfo.setEntryId(entryId); - List deleteSet = new ArrayList<>(array.length); - batchDeletedIndexInfo.clearDeleteSet(); for (long l : array) { batchDeletedIndexInfo.addDeleteSet(l); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java index 39aa2d6299a8a..dc953676ca947 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java @@ -19,14 +19,23 @@ package org.apache.bookkeeper.mledger.impl; import static org.testng.Assert.*; + +import com.google.protobuf.InvalidProtocolBufferException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import java.util.Map; import java.util.List; + +import org.apache.bookkeeper.mledger.proto.LightMLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.testng.annotations.Test; public class PositionInfoUtilsTest { + private static final Logger log = LoggerFactory.getLogger(PositionInfoUtilsTest.class); + @Test public void testSerializeDeserialize() throws Exception { PositionImpl position = new PositionImpl(1, 2); @@ -88,4 +97,118 @@ public void testSerializeDeserializeEmpty() throws Exception { assertEquals(0, positionInfoParsed.getBatchedEntryDeletionIndexInfoCount()); result.release(); } + + @Test + public void testSerializeDeserialize2() throws Exception { + PositionImpl position = new PositionImpl(1, 2); + ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, + Map.of("foo", 1L), null, null); + + final int numRanges = 10000; + ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { + for (int i = 0; i < numRanges; i++) { + scanner.acceptRange(i*4 + 1, i*4 + 2, i*4 + 3, i*4 + 4); + } + }, (scanner) -> { + long[] array = {7L, 8L}; + for (int i = 0; i < numRanges; i++) { + scanner.acceptRange(i*2 + 1, i*2 + 2, array); + } + }, 1024); + + // deserialize PIUtils -> lightproto + final int idx = result.readerIndex(); + LightMLDataFormats.PositionInfo lighPositionInfoParsed = new LightMLDataFormats.PositionInfo(); + lighPositionInfoParsed.parseFrom(result, result.readableBytes()); + result.readerIndex(idx); + + validateLightproto(lighPositionInfoParsed, numRanges); + + // serialize lightproto + int serializedSz = lighPositionInfoParsed.getSerializedSize(); + ByteBuf lightResult = PulsarByteBufAllocator.DEFAULT.buffer(serializedSz); + lighPositionInfoParsed.writeTo(lightResult); + + byte[] light = ByteBufUtil.getBytes(lightResult); + byte[] util = ByteBufUtil.getBytes(result); + + assertEquals(light.length, util.length); + + for (int i = 0; i < light.length; i++) { + if (light[i] != util[i]) { + log.error("Mismatch at index {} light={} util={}", i, light[i], util[i]); + } + } + + assertEquals(light, util); + + // deserialize lightproto -> protobuf + parseProtobufAndValidate(light, numRanges); + + // deserialize PIUtils -> protobuf + parseProtobufAndValidate(util, numRanges); + + result.release(); + lightResult.release(); + } + + private static void validateLightproto(LightMLDataFormats.PositionInfo lighPositionInfoParsed, int numRanges) { + assertEquals(1, lighPositionInfoParsed.getLedgerId()); + assertEquals(2, lighPositionInfoParsed.getEntryId()); + + assertEquals(1, lighPositionInfoParsed.getPropertiesCount()); + assertEquals("foo", lighPositionInfoParsed.getPropertyAt(0).getName()); + assertEquals(1, lighPositionInfoParsed.getPropertyAt(0).getValue()); + + assertEquals(numRanges, lighPositionInfoParsed.getIndividualDeletedMessagesCount()); + int curr = 0; + for (int i = 0; i < numRanges; i++) { + assertEquals(i*4 + 1, lighPositionInfoParsed.getIndividualDeletedMessageAt(curr).getLowerEndpoint().getLedgerId()); + assertEquals(i*4 + 2, lighPositionInfoParsed.getIndividualDeletedMessageAt(curr).getLowerEndpoint().getEntryId()); + assertEquals(i*4 + 3, lighPositionInfoParsed.getIndividualDeletedMessageAt(curr).getUpperEndpoint().getLedgerId()); + assertEquals(i*4 + 4, lighPositionInfoParsed.getIndividualDeletedMessageAt(curr).getUpperEndpoint().getEntryId()); + curr++; + } + + assertEquals(numRanges, lighPositionInfoParsed.getBatchedEntryDeletionIndexInfosCount()); + curr = 0; + for (int i = 0; i < numRanges; i++) { + assertEquals(i*2 + 1, lighPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getPosition().getLedgerId()); + assertEquals(i*2 + 2, lighPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getPosition().getEntryId()); + assertEquals(7L, lighPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getDeleteSetAt(0)); + assertEquals(8L, lighPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getDeleteSetAt(1)); + curr++; + } + } + + private static void parseProtobufAndValidate(byte[] data, int numRanges) throws InvalidProtocolBufferException { + MLDataFormats.PositionInfo positionInfoParsed = MLDataFormats.PositionInfo.parseFrom(data); + + assertEquals(1, positionInfoParsed.getLedgerId()); + assertEquals(2, positionInfoParsed.getEntryId()); + + assertEquals(1, positionInfoParsed.getPropertiesCount()); + assertEquals("foo", positionInfoParsed.getProperties(0).getName()); + assertEquals(1, positionInfoParsed.getProperties(0).getValue()); + + assertEquals(numRanges, positionInfoParsed.getIndividualDeletedMessagesCount()); + int curr = 0; + for (int i = 0; i < numRanges; i++) { + assertEquals(i*4 + 1, positionInfoParsed.getIndividualDeletedMessages(curr).getLowerEndpoint().getLedgerId()); + assertEquals(i*4 + 2, positionInfoParsed.getIndividualDeletedMessages(curr).getLowerEndpoint().getEntryId()); + assertEquals(i*4 + 3, positionInfoParsed.getIndividualDeletedMessages(curr).getUpperEndpoint().getLedgerId()); + assertEquals(i*4 + 4, positionInfoParsed.getIndividualDeletedMessages(curr).getUpperEndpoint().getEntryId()); + curr++; + } + + assertEquals(numRanges, positionInfoParsed.getBatchedEntryDeletionIndexInfoCount()); + curr = 0; + for (int i = 0; i < numRanges; i++) { + assertEquals(i*2 + 1, positionInfoParsed.getBatchedEntryDeletionIndexInfo(curr).getPosition().getLedgerId()); + assertEquals(i*2 + 2, positionInfoParsed.getBatchedEntryDeletionIndexInfo(curr).getPosition().getEntryId()); + assertEquals(List.of(7L, 8L), positionInfoParsed.getBatchedEntryDeletionIndexInfo(curr).getDeleteSetList()); + curr++; + } + } + } \ No newline at end of file From c2f09089f3c0b5d3af09a2740daef214c0a94154 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 17 May 2024 08:43:56 +0200 Subject: [PATCH 07/22] Remove auto reset of cursor in case of read error (cherry picked from commit d4b94ab218f6e01fd288dda99709da20c87d3387) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f1f1b106dbd00..ff3175d1d1e0f 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 @@ -682,6 +682,8 @@ private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byt try { data = decompressDataIfNeeded(data, lh); } catch (Throwable e) { + log.error("[{}] Failed to decompress position info from ledger {} for cursor {}: {}", ledger.getName(), + lh.getId(), name, e); callback.operationFailed(new ManagedLedgerException(e)); return; } @@ -692,12 +694,8 @@ private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byt } catch (InvalidProtocolBufferException e) { log.error("[{}] Failed to parse position info from ledger {} for cursor {}: {}", ledger.getName(), lh.getId(), name, e); - // Rewind to oldest entry available - positionInfo = PositionInfo - .newBuilder() - .setLedgerId(-1) - .setEntryId(-1) - .build(); + callback.operationFailed(new ManagedLedgerException(e)); + return; } Map recoveredProperties = Collections.emptyMap(); From 27152ffcb71448aeee9ac650717a221bfb8afd80 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 17 May 2024 10:04:24 +0200 Subject: [PATCH 08/22] Revert removal of 'containsKey' in ManagedCursorImpl (cherry picked from commit 5f07f0cde706f95e2495360dad6ac6f846dfa7c2) --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index ff3175d1d1e0f..75082545ec31d 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 @@ -485,7 +485,9 @@ public boolean removeProperty(String key) { if (lastMarkDeleteEntry != null) { LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> { Map properties = last.properties; - if (properties != null) { + // we can call remove only if the property is present + // some implementation of the map can throw exceptions + if (properties != null && properties.containsKey(key)) { properties.remove(key); } return last; From 564a668b27609935abe6825b50b4b6dfc0dbe07f Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Mon, 20 May 2024 11:37:35 +0200 Subject: [PATCH 09/22] Prevent ZK connection loss in case of huge cursor status (#273) (cherry picked from commit 6d2e4946b6ae48f3daf0f513821cbe6259b5f1b8) --- .../mledger/impl/ManagedCursorImpl.java | 75 +++++++++++-------- .../mledger/impl/MetaStoreImpl.java | 5 ++ 2 files changed, 50 insertions(+), 30 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 75082545ec31d..f643b9caf13ea 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 @@ -2793,6 +2793,8 @@ private void persistPositionMetaStore(long cursorsLedgerId, Position position, M new CursorAlreadyClosedException(name + " cursor already closed")))); return; } + log.info("[{}][{}] Persisting cursor metadata into metadata store (persistIndividualDeletedMessageRanges: {})", + ledger.getName(), name, persistIndividualDeletedMessageRanges); final Stat lastCursorLedgerStat = cursorLedgerStat; @@ -2807,7 +2809,7 @@ private void persistPositionMetaStore(long cursorsLedgerId, Position position, M info.addAllProperties(buildPropertiesMap(properties)); info.addAllCursorProperties(buildStringPropertiesMap(cursorProperties)); if (persistIndividualDeletedMessageRanges) { - info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); + info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges(true)); if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { info.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()); } @@ -3153,7 +3155,7 @@ private static List buildStringPropertiesMap(Map return stringProperties; } - private List buildIndividualDeletedMessageRanges() { + private List buildIndividualDeletedMessageRanges(boolean forMetastore) { lock.writeLock().lock(); try { log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {}", @@ -3188,19 +3190,28 @@ private List buildIndividualDeletedMessageRanges() { .setUpperEndpoint(upperPosition) .build(); - acksSerializedSize.addAndGet(messageRange.getSerializedSize()); + int currentSize = acksSerializedSize.addAndGet(messageRange.getSerializedSize()); rangeList.add(messageRange); + if (forMetastore && currentSize > (1024 * 1024 - 10 * 1024)) { + log.warn("[{}] [{}] buildIndividualDeletedMessageRanges, " + + "rangeListSize {} " + + "maxUnackedRangesToPersist {}, " + + "reached {} bytes that is too big for the metastore", + ledger.getName(), name, + rangeList.size(), + getConfig().getMaxUnackedRangesToPersist(), currentSize); + return false; + } + return rangeList.size() <= getConfig().getMaxUnackedRangesToPersist(); }); this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); - log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {} " - + "individualDeletedMessagesSerializedSize {} rangeListSize {} " + log.info("[{}] [{}] buildIndividualDeletedMessageRanges, rangeListSize {} " + "maxUnackedRangesToPersist {}", - ledger.getName(), name, individualDeletedMessages.size(), - individualDeletedMessagesSerializedSize, rangeList.size(), + ledger.getName(), name, rangeList.size(), getConfig().getMaxUnackedRangesToPersist()); return rangeList; @@ -3230,11 +3241,11 @@ private void scanIndividualDeletedMessageRanges( this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); - log.info("[{}] [{}] scanIndividualDeletedMessageRanges, numRanges {} " - + "individualDeletedMessagesSerializedSize {} rangeListSize {} " + log.info("[{}] [{}] scanIndividualDeletedMessageRanges, " + + "rangeListSize {} " + "maxUnackedRangesToPersist {}", - ledger.getName(), name, individualDeletedMessages.size(), - individualDeletedMessagesSerializedSize, rangeCount.get(), + ledger.getName(), name, + rangeCount.get(), getConfig().getMaxUnackedRangesToPersist()); } finally { lock.readLock().unlock(); @@ -3387,27 +3398,30 @@ private void writeToBookKeeperLastChunk(LedgerHandle lh, PositionImpl position, Runnable onFinished) { lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> { - if (rc == BKException.Code.OK) { - if (log.isDebugEnabled()) { - log.debug("[{}] Updated cursor {} position {} in meta-ledger {}", ledger.getName(), name, position, - lh1.getId()); - } + try { + if (rc == BKException.Code.OK) { + if (log.isDebugEnabled()) { + log.debug("[{}] Updated cursor {} position {} in meta-ledger {}", ledger.getName(), name, + position, + lh1.getId()); + } - rolloverLedgerIfNeeded(lh1); + rolloverLedgerIfNeeded(lh1); - mbean.persistToLedger(true); - mbean.addWriteCursorLedgerSize(data.readableBytes()); - callback.operationComplete(); - onFinished.run(); - } else { - log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}: {}", ledger.getName(), name, - position, lh1.getId(), BKException.getMessage(rc)); - // If we've had a write error, the ledger will be automatically closed, we need to create a new one, - // in the meantime the mark-delete will be queued. - STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); - - // Before giving up, try to persist the position in the metadata store. - persistPositionToMetaStore(mdEntry, callback); + mbean.persistToLedger(true); + mbean.addWriteCursorLedgerSize(data.readableBytes()); + callback.operationComplete(); + } else { + log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}: {}", ledger.getName(), name, + position, lh1.getId(), BKException.getMessage(rc)); + // If we've had a write error, the ledger will be automatically closed, we need to create a new one, + // in the meantime the mark-delete will be queued. + STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); + + // Before giving up, try to persist the position in the metadata store. + persistPositionToMetaStore(mdEntry, callback); + } + } finally { onFinished.run(); } }, null); @@ -3502,6 +3516,7 @@ boolean rolloverLedgerIfNeeded(LedgerHandle lh1) { } void persistPositionToMetaStore(MarkDeleteEntry mdEntry, final VoidCallback callback) { + log.info("[{}][{}] Persisting cursor metadata into metadata store", ledger.getName(), name); final Position newPosition = mdEntry.newPosition; STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); mbean.persistToLedger(false); 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 a9117848e58bc..28e60cbf5b95c 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 @@ -249,6 +249,8 @@ public void asyncUpdateCursorInfo(String ledgerName, String cursorName, ManagedC String path = PREFIX + ledgerName + "/" + cursorName; byte[] content = compressCursorInfo(info); + log.info("[{}] Persisting cursor={} info with content size {} bytes to metastore", + ledgerName, cursorName, content.length); long expectedVersion; @@ -267,6 +269,7 @@ public void asyncUpdateCursorInfo(String ledgerName, String cursorName, ManagedC .thenAcceptAsync(optStat -> callback.operationComplete(null, optStat), executor .chooseThread(ledgerName)) .exceptionally(ex -> { + log.error("[{}] [{}] Failed to update cursor info", ledgerName, cursorName, ex); executor.executeOrdered(ledgerName, () -> callback.operationFailed(getException(ex))); return null; @@ -525,6 +528,8 @@ private byte[] compressManagedInfo(byte[] info, byte[] metadata, int metadataSer compositeByteBuf.addComponent(true, encodeByteBuf); byte[] dataBytes = new byte[compositeByteBuf.readableBytes()]; compositeByteBuf.readBytes(dataBytes); + log.info("Compressed cursor info, info size {}, metadata size {}, compressed size: {}", + info.length, metadata.length, dataBytes.length); return dataBytes; } finally { if (metadataByteBuf != null) { From 0d23d5b06d427852e62f50759d5f6dfb310c59ce Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 21 May 2024 14:50:32 +0200 Subject: [PATCH 10/22] [managed-ledger] Compressed cursors: fix problem with little buffers (#275) (cherry picked from commit 6a2a010abb2ad4b5f15a2ebde7ce2da26d3457af) --- .../mledger/impl/ManagedCursorImpl.java | 19 ++++++++++++++----- .../mledger/impl/ManagedCursorTest.java | 9 ++++++++- .../test/MockedBookKeeperTestCase.java | 7 ++++++- .../client/PulsarMockBookKeeper.java | 4 ++-- .../client/PulsarMockLedgerHandle.java | 12 +++++++++--- 5 files changed, 39 insertions(+), 12 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f643b9caf13ea..eaf6cf51130ab 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 @@ -34,6 +34,7 @@ import com.google.common.collect.Range; import com.google.common.util.concurrent.RateLimiter; import com.google.protobuf.InvalidProtocolBufferException; +import io.airlift.compress.MalformedInputException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.CompositeByteBuf; @@ -3460,25 +3461,33 @@ private ByteBuf compressDataIfNeeded(ByteBuf data, LedgerHandle lh) { } } - private static byte[] decompressDataIfNeeded(byte[] data, LedgerHandle lh) { + static byte[] decompressDataIfNeeded(byte[] data, LedgerHandle lh) { byte[] pulsarCursorInfoCompression = lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE); if (pulsarCursorInfoCompression != null) { String pulsarCursorInfoCompressionString = new String(pulsarCursorInfoCompression); - CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( - CompressionType.valueOf(pulsarCursorInfoCompressionString)); + if (log.isDebugEnabled()) { + log.debug("Ledger {} compression {} decompressing {} bytes, full {}", + lh.getId(), pulsarCursorInfoCompressionString, data.length, + ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(data))); + } ByteArrayInputStream input = new ByteArrayInputStream(data); DataInputStream dataInputStream = new DataInputStream(input); try { int uncompressedSize = dataInputStream.readInt(); - byte[] compressedData = dataInputStream.readNBytes(uncompressedSize); + byte[] compressedData = dataInputStream.readAllBytes(); + CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( + CompressionType.valueOf(pulsarCursorInfoCompressionString)); ByteBuf decode = compressionCodec.decode(Unpooled.wrappedBuffer(compressedData), uncompressedSize); try { return ByteBufUtil.getBytes(decode); } finally { decode.release(); } - } catch (IOException error) { + } catch (IOException | MalformedInputException error) { + log.error("Cannot decompress cursor position using {}. Payload is {}", + pulsarCursorInfoCompressionString, + ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(data)), error); throw new RuntimeException(error); } } 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 b42e5fe283c73..350d9bacf7518 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 @@ -128,6 +128,11 @@ public static Object[][] useOpenRangeSet() { return new Object[][] { { Boolean.TRUE }, { Boolean.FALSE } }; } + @Override + protected void setupManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) { + super.setupManagedLedgerFactoryConfig(config); + config.setManagedCursorInfoCompressionType("LZ4"); + } @Test public void testCloseCursor() throws Exception { @@ -3269,7 +3274,9 @@ public void operationFailed(MetaStoreException e) { try { LedgerEntry entry = seq.nextElement(); PositionInfo positionInfo; - positionInfo = PositionInfo.parseFrom(entry.getEntry()); + byte[] data = entry.getEntry(); + data = ManagedCursorImpl.decompressDataIfNeeded(data, lh); + positionInfo = PositionInfo.parseFrom(data); individualDeletedMessagesCount.set(positionInfo.getIndividualDeletedMessagesCount()); } catch (Exception e) { } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java index c7685cfaa6594..bd1d42e54b6e1 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java @@ -67,6 +67,10 @@ public MockedBookKeeperTestCase(int numBookies) { this.numBookies = numBookies; } + protected void setupManagedLedgerFactoryConfig(ManagedLedgerFactoryConfig config) { + // No-op + } + @BeforeMethod(alwaysRun = true) public final void setUp(Method method) throws Exception { LOG.info(">>>>>> starting {}", method); @@ -84,7 +88,8 @@ public final void setUp(Method method) throws Exception { ManagedLedgerFactoryConfig managedLedgerFactoryConfig = new ManagedLedgerFactoryConfig(); initManagedLedgerFactoryConfig(managedLedgerFactoryConfig); - factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + setupManagedLedgerFactoryConfig(managedLedgerFactoryConfig); + factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, managedLedgerFactoryConfig); setUpTestCase(); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java index 4516cfea01f05..f1a9694b3d165 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -124,7 +124,7 @@ public void asyncCreateLedger(int ensSize, int writeQuorumSize, int ackQuorumSiz long id = sequence.getAndIncrement(); log.info("Creating ledger {}", id); PulsarMockLedgerHandle lh = - new PulsarMockLedgerHandle(PulsarMockBookKeeper.this, id, digestType, passwd); + new PulsarMockLedgerHandle(PulsarMockBookKeeper.this, id, digestType, passwd, properties); ledgers.put(id, lh); return FutureUtils.value(lh); } catch (Throwable t) { @@ -147,7 +147,7 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum try { long id = sequence.getAndIncrement(); log.info("Creating ledger {}", id); - PulsarMockLedgerHandle lh = new PulsarMockLedgerHandle(this, id, digestType, passwd); + PulsarMockLedgerHandle lh = new PulsarMockLedgerHandle(this, id, digestType, passwd, null); ledgers.put(id, lh); return lh; } catch (Throwable t) { diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java index aa61e541d0d6b..400065e41b7f0 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -26,8 +26,10 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Enumeration; import java.util.List; +import java.util.Map; import java.util.Queue; import java.util.concurrent.CompletableFuture; import lombok.Getter; @@ -65,8 +67,10 @@ public class PulsarMockLedgerHandle extends LedgerHandle { boolean fenced = false; public PulsarMockLedgerHandle(PulsarMockBookKeeper bk, long id, - DigestType digest, byte[] passwd) throws GeneralSecurityException { - super(bk.getClientCtx(), id, new Versioned<>(createMetadata(id, digest, passwd), new LongVersion(0L)), + DigestType digest, byte[] passwd, + Map properties) throws GeneralSecurityException { + super(bk.getClientCtx(), id, new Versioned<>(createMetadata(id, digest, passwd, properties), + new LongVersion(0L)), digest, passwd, WriteFlag.NONE); this.bk = bk; this.id = id; @@ -267,13 +271,15 @@ public CompletableFuture readLastAddConfirmedAndEntryAsyn return readHandle.readLastAddConfirmedAndEntryAsync(entryId, timeOutInMillis, parallel); } - private static LedgerMetadata createMetadata(long id, DigestType digest, byte[] passwd) { + private static LedgerMetadata createMetadata(long id, DigestType digest, byte[] passwd, + Map properties) { List ensemble = new ArrayList<>(PulsarMockBookKeeper.getMockEnsemble()); return LedgerMetadataBuilder.create() .withDigestType(digest.toApiDigestType()) .withPassword(passwd) .withId(id) .newEnsembleEntry(0L, ensemble) + .withCustomMetadata(properties != null ? properties : Collections.emptyMap()) .build(); } From 08af8fc1e9127fcb5c3586d2a34df85eefe215a6 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 21 May 2024 15:00:59 +0200 Subject: [PATCH 11/22] [tests] Fix build after merge conflict (cherry picked from commit 4c5387d783f4ea626070553d667c44c9078e45b9) --- .../apache/pulsar/client/impl/BrokerClientIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java index 1e8754a2d675c..5dfdfaa9802f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/BrokerClientIntegrationTest.java @@ -726,7 +726,7 @@ public void testAddEntryOperationTimeout() throws Exception { class MockLedgerHandle extends PulsarMockLedgerHandle { public MockLedgerHandle(PulsarMockBookKeeper bk, long id, DigestType digest, byte[] passwd) throws GeneralSecurityException { - super(bk, id, digest, passwd); + super(bk, id, digest, passwd, null); } @Override From e8d3930149cfbc2a6b502d4f7c01a250b00dd95a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Wed, 22 May 2024 14:04:10 +0200 Subject: [PATCH 12/22] Fix WriteCursorLedgerSize metric (cherry picked from commit c3fe80e4941183abde373e2126ca91f3ac51baf2) --- .../apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index eaf6cf51130ab..a681c20c966a8 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 @@ -3347,7 +3347,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin if (numParts == 1) { // no need for chunking // asyncAddEntry will release data ByteBuf - writeToBookKeeperLastChunk(lh, mdEntry, callback, data, position, () -> {}); + writeToBookKeeperLastChunk(lh, mdEntry, callback, data, len, position, () -> {}); } else { // chunking int part = 0; @@ -3383,7 +3383,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin } // need to explicitly release data ByteBuf writeToBookKeeperLastChunk(lh, mdEntry, callback, - Unpooled.wrappedBuffer(footerData), position, data::release); + Unpooled.wrappedBuffer(footerData), len, position, data::release); } offset += currentLen; part++; @@ -3396,6 +3396,7 @@ private void writeToBookKeeperLastChunk(LedgerHandle lh, MarkDeleteEntry mdEntry, VoidCallback callback, ByteBuf data, + int totalLength, PositionImpl position, Runnable onFinished) { lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> { @@ -3410,7 +3411,7 @@ private void writeToBookKeeperLastChunk(LedgerHandle lh, rolloverLedgerIfNeeded(lh1); mbean.persistToLedger(true); - mbean.addWriteCursorLedgerSize(data.readableBytes()); + mbean.addWriteCursorLedgerSize(totalLength); callback.operationComplete(); } else { log.warn("[{}] Error updating cursor {} position {} in meta-ledger {}: {}", ledger.getName(), name, From 89adf3882d930fd1cd63c4e081bc8244680a1abe Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Tue, 28 May 2024 09:04:31 -0700 Subject: [PATCH 13/22] try ledger recovery from previous entries in case of corrupt/missing footer of the chunked data (#282) (cherry picked from commit 6e72ecbba11325ba17f78a903ec7298bbe937b03) --- .../mledger/impl/ManagedCursorImpl.java | 180 +++++++++++------- .../mledger/impl/ManagedCursorTest.java | 78 ++++++++ .../mledger/impl/ManagedLedgerTest.java | 2 +- 3 files changed, 187 insertions(+), 73 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index a681c20c966a8..f8fc232314717 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 @@ -40,6 +40,7 @@ import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.IOException; import java.time.Clock; @@ -47,6 +48,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.Enumeration; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -75,6 +77,7 @@ import lombok.Getter; import lombok.NoArgsConstructor; import lombok.ToString; +import org.apache.bookkeeper.client.AsyncCallback; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; @@ -240,6 +243,8 @@ public class ManagedCursorImpl implements ManagedCursor { // active state cache in ManagedCursor. It should be in sync with the state in activeCursors in ManagedLedger. private volatile boolean isActive = false; + protected int maxPositionChunkSize = 1024 * 1024; + static class MarkDeleteEntry { final Position newPosition; final MarkDeleteCallback callback; @@ -585,79 +590,112 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac // Read the last entry in the ledger long lastEntryInLedger = lh.getLastAddConfirmed(); + recoverFromLedgerByEntryId(info, callback, lh, lastEntryInLedger); + }; - if (lastEntryInLedger < 0) { - log.warn("[{}] Error reading from metadata ledger {} for cursor {}: No entries in ledger", - ledger.getName(), ledgerId, name); - // Rewind to last cursor snapshot available - initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); - return; - } + try { + bookkeeper.asyncOpenLedger(ledgerId, digestType, getConfig().getPassword(), openCallback, + null); + } catch (Throwable t) { + log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", + ledger.getName(), ledgerId, name, t); + openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null); + } + } - lh.asyncReadEntries(lastEntryInLedger, lastEntryInLedger, (rc1, lh1, seq, ctx1) -> { - if (log.isDebugEnabled()) { - log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); - } - if (isBkErrorNotRecoverable(rc1)) { - log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), - ledgerId, name, BKException.getMessage(rc1)); - // Rewind to oldest entry available - initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); - return; - } else if (rc1 != BKException.Code.OK) { - log.warn("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), - ledgerId, name, BKException.getMessage(rc1)); + private void recoverFromLedgerByEntryId(ManagedCursorInfo info, + VoidCallback callback, + LedgerHandle lh, + long entryId) { + long ledgerId = lh.getId(); + + if (entryId < 0) { + log.warn("[{}] Error reading from metadata ledger {} for cursor {}: No valid entries in ledger", + ledger.getName(), ledgerId, name); + // Rewind to last cursor snapshot available + initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); + return; + } - callback.operationFailed(createManagedLedgerException(rc1)); - return; - } + lh.asyncReadEntries(entryId, entryId, (rc1, lh1, seq, ctx1) -> { + if (log.isDebugEnabled()) { + log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); + } + if (isBkErrorNotRecoverable(rc1)) { + log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), + ledgerId, name, BKException.getMessage(rc1)); + // Rewind to oldest entry available + initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); + return; + } else if (rc1 != BKException.Code.OK) { + log.warn("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), + ledgerId, name, BKException.getMessage(rc1)); - LedgerEntry entry = seq.nextElement(); - mbean.addReadCursorLedgerSize(entry.getLength()); - PositionInfo positionInfo; - try { - byte[] data = entry.getEntry(); - data = decompressDataIfNeeded(data, lh); - positionInfo = PositionInfo.parseFrom(data); - } catch (InvalidProtocolBufferException e) { - callback.operationFailed(new ManagedLedgerException(e)); - return; - } - log.info("[{}] Cursor {} recovered to position {}", ledger.getName(), name, positionInfo); + callback.operationFailed(createManagedLedgerException(rc1)); + return; + } - Map recoveredProperties = Collections.emptyMap(); - if (positionInfo.getPropertiesCount() > 0) { - // Recover properties map - recoveredProperties = new HashMap<>(); - for (int i = 0; i < positionInfo.getPropertiesCount(); i++) { - LongProperty property = positionInfo.getProperties(i); - recoveredProperties.put(property.getName(), property.getValue()); + LedgerEntry entry = seq.nextElement(); + byte[] data = entry.getEntry(); + try { + ChunkSequenceFooter chunkSequenceFooter = parseChunkSequenceFooter(data); + if (chunkSequenceFooter.numParts > 0) { + readChunkSequence(callback, lh, entryId, chunkSequenceFooter); + } else { + Throwable res = tryCompleteCursorRecovery(lh, data); + if (res == null) { + callback.operationComplete(); + } else { + log.warn("[{}] Error recovering from metadata ledger {} entry {} for cursor {}. " + + "Will try recovery from previous entry.", + ledger.getName(), ledgerId, entryId, name, res); + //try recovery from previous entry + recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); } } + } catch (IOException error) { + log.error("Cannot parse footer", error); + log.warn("[{}] Error recovering from metadata ledger {} entry {} for cursor {}, cannot parse footer. " + + "Will try recovery from previous entry.", + ledger.getName(), ledgerId, entryId, name, error); + recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); + } + }, null); + } - log.info("[{}] Cursor {} recovered with recoveredProperties {}, individualDeletedMessagesCount {}", - ledger.getName(), name, recoveredProperties, positionInfo.getIndividualDeletedMessagesCount()); - - Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); - if (positionInfo.getIndividualDeletedMessagesCount() > 0) { - recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); + private void readChunkSequence(VoidCallback callback, LedgerHandle lh, + long footerPosition, ChunkSequenceFooter chunkSequenceFooter) { + long startPos = footerPosition - chunkSequenceFooter.numParts; + long endPos = footerPosition - 1; + log.info("readChunkSequence from pos {}, num parts {}, startPos {}, endPos {}", + footerPosition, chunkSequenceFooter.numParts, startPos, endPos); + lh.asyncReadEntries(startPos, endPos, new AsyncCallback.ReadCallback() { + @Override + public void readComplete(int rc, LedgerHandle lh, Enumeration entries, Object ctx) { + ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + entries.asIterator().forEachRemaining(entry -> { + log.info("pos {} len {} bytes ", entry.getEntryId(), entry.getLength()); + try { + buffer.write(entry.getEntry()); + } catch (IOException err) { + throw new RuntimeException(err); + } + }); + byte[] result = buffer.toByteArray(); + log.info("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, + result.length, chunkSequenceFooter.length); + if (result.length != chunkSequenceFooter.length) { + callback.operationFailed(ManagedLedgerException.getManagedLedgerException(new IOException( + "Expected " + chunkSequenceFooter.length + " bytes but read " + result.length + " bytes"))); } - if (getConfig().isDeletionAtBatchIndexLevelEnabled() - && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { - recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); + Throwable res = tryCompleteCursorRecovery(lh, result); + if (res == null) { + callback.operationComplete(); + } else { + callback.operationFailed(new ManagedLedgerException(res)); } - recoveredCursor(position, recoveredProperties, cursorProperties, lh); - callback.operationComplete(); - }, null); - }; - try { - bookkeeper.asyncOpenLedger(ledgerId, digestType, getConfig().getPassword(), openCallback, - null); - } catch (Throwable t) { - log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", - ledger.getName(), ledgerId, name, t); - openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null); - } + } + }, null); } @AllArgsConstructor @@ -679,7 +717,7 @@ private ChunkSequenceFooter parseChunkSequenceFooter(byte[] data) throws IOExcep return ObjectMapperFactory.getMapper().getObjectMapper().readValue(data, ChunkSequenceFooter.class); } - private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byte[] data) { + private Throwable tryCompleteCursorRecovery(LedgerHandle lh, byte[] data) { mbean.addReadCursorLedgerSize(data.length); try { @@ -687,8 +725,7 @@ private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byt } catch (Throwable e) { log.error("[{}] Failed to decompress position info from ledger {} for cursor {}: {}", ledger.getName(), lh.getId(), name, e); - callback.operationFailed(new ManagedLedgerException(e)); - return; + return e; } PositionInfo positionInfo; @@ -697,8 +734,7 @@ private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byt } catch (InvalidProtocolBufferException e) { log.error("[{}] Failed to parse position info from ledger {} for cursor {}: {}", ledger.getName(), lh.getId(), name, e); - callback.operationFailed(new ManagedLedgerException(e)); - return; + return e; } Map recoveredProperties = Collections.emptyMap(); @@ -720,7 +756,7 @@ private void completeCursorRecovery(VoidCallback callback, LedgerHandle lh, byt recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } recoveredCursor(position, recoveredProperties, cursorProperties, lh); - callback.operationComplete(); + return null; } private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { @@ -3307,6 +3343,7 @@ private void buildBatchEntryDeletionIndexInfoList( } void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, final VoidCallback callback) { + checkArgument(maxPositionChunkSize > 0, "maxPositionChunkSize mus be greater than zero"); long now = System.nanoTime(); Position position = mdEntry.newPosition; @@ -3327,10 +3364,9 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin long endCompress = System.nanoTime(); - int maxSize = 1024 * 1024; int offset = 0; final int len = data.readableBytes(); - int numParts = 1 + (len / maxSize); + int numParts = 1 + (len / maxPositionChunkSize); if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {}", @@ -3353,7 +3389,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin int part = 0; while (part != numParts) { int remaining = len - offset; - int currentLen = Math.min(maxSize, remaining); + int currentLen = Math.min(maxPositionChunkSize, remaining); boolean isLast = part == numParts - 1; if (log.isDebugEnabled()) { 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 350d9bacf7518..ccb7ddb20f9b5 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 @@ -75,6 +75,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.client.api.ReadHandle; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -99,6 +100,7 @@ import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedCursorInfo; import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.apache.pulsar.common.api.proto.IntRange; import org.apache.pulsar.common.util.FutureUtil; @@ -3619,6 +3621,82 @@ public void operationFailed(ManagedLedgerException exception) { assertEquals(c.getReadPosition(), readPositionBeforeRecover); assertEquals(c.getNumberOfEntries(), 2L); } + + @Test(timeOut = 20000) + public void testRecoverCursorCorruptLastEntry() throws Exception { + ManagedLedger ml = factory.open("testRecoverCursorCorruptLastEntry"); + ManagedCursorImpl c = (ManagedCursorImpl) ml.openCursor("sub", CommandSubscribe.InitialPosition.Latest); + // force chunking + c.maxPositionChunkSize = 2; + + // A new cursor starts out with these values. The rest of the test assumes this, so we assert it here. + assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); + assertEquals(c.getReadPosition().getEntryId(), 0); + assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1); + + c.resetCursor(PositionImpl.LATEST); + + // A reset cursor starts out with these values. The rest of the test assumes this, so we assert it here. + assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); + assertEquals(c.getReadPosition().getEntryId(), 0); + assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1); + + // Trigger the lastConfirmedEntry to move forward + ml.addEntry(new byte[1]); + ml.addEntry(new byte[1]); + ml.addEntry(new byte[1]); + ml.addEntry(new byte[1]); + + c.resetCursor(PositionImpl.LATEST); + //corrupt last entry + LedgerHandle cursorLedger = (LedgerHandle)FieldUtils.readDeclaredField(c, "cursorLedger", true); + // can't parse json + cursorLedger.addEntry("{{".getBytes()); + // can't parse PositionInfo protobuf + cursorLedger.addEntry("aa".getBytes()); + + assertEquals(c.getMarkDeletedPosition().getEntryId(), 3); + assertEquals(c.getReadPosition().getEntryId(), 4); + assertEquals(ml.getLastConfirmedEntry().getEntryId(), 3); + + // Publish messages to move the lastConfirmedEntry field forward + ml.addEntry(new byte[1]); + ml.addEntry(new byte[1]); + + final Position markDeleteBeforeRecover = c.getMarkDeletedPosition(); + final Position readPositionBeforeRecover = c.getReadPosition(); + + ManagedCursorInfo info = ManagedCursorInfo.newBuilder() + .setCursorsLedgerId(c.getCursorLedger()) + .setMarkDeleteLedgerId(markDeleteBeforeRecover.getLedgerId()) + .setMarkDeleteEntryId(markDeleteBeforeRecover.getEntryId()) + .setLastActive(0L) + .build(); + + CountDownLatch latch = new CountDownLatch(1); + AtomicBoolean failed = new AtomicBoolean(false); + c.recoverFromLedger(info, new VoidCallback() { + @Override + public void operationComplete() { + latch.countDown(); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + failed.set(true); + latch.countDown(); + } + }); + + latch.await(); + if (failed.get()) { + fail("Cursor recovery should not fail"); + } + assertEquals(c.getMarkDeletedPosition(), markDeleteBeforeRecover); + assertEquals(c.getReadPosition(), readPositionBeforeRecover); + assertEquals(c.getNumberOfEntries(), 2L); + } + @Test void testAlwaysInactive() throws Exception { ManagedLedger ml = factory.open("testAlwaysInactive"); 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 83a6c771513a9..608d66fdc4111 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 @@ -3260,7 +3260,7 @@ public void testManagedLedgerWithAddEntryTimeOut() throws Exception { class MockLedgerHandle extends PulsarMockLedgerHandle { public MockLedgerHandle(PulsarMockBookKeeper bk, long id, DigestType digest, byte[] passwd) throws GeneralSecurityException { - super(bk, id, digest, passwd); + super(bk, id, digest, passwd, null); } @Override From 43a5b31d0605079aa2bbadf5d333f073e6fb967e Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Wed, 29 May 2024 14:52:21 -0700 Subject: [PATCH 14/22] fix boken test after merge/resolve --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 3 +++ 1 file changed, 3 insertions(+) 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 f8fc232314717..ea1b5591d884d 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 @@ -3303,6 +3303,9 @@ private List buildBatchEntryDeletio Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { Map.Entry entry = iterator.next(); + nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); + nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); + batchDeletedIndexInfoBuilder.setPosition(nestedPositionBuilder.build()); long[] array = entry.getValue().toLongArray(); List deleteSet = new ArrayList<>(array.length); for (long l : array) { From 8db72f4b56d3eb6d96184d67d697e25d7487c7f9 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Mon, 9 Sep 2024 14:18:28 -0700 Subject: [PATCH 15/22] post-rebase fixes --- .../apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 8 ++++---- .../apache/bookkeeper/mledger/impl/PositionInfoUtils.java | 4 +++- .../apache/bookkeeper/mledger/impl/ManagedCursorTest.java | 4 ++-- .../bookkeeper/mledger/impl/PositionInfoUtilsTest.java | 8 +++++--- 4 files changed, 14 insertions(+), 10 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index ea1b5591d884d..870a1ea224791 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 @@ -747,7 +747,7 @@ private Throwable tryCompleteCursorRecovery(LedgerHandle lh, byte[] data) { } } - PositionImpl position = new PositionImpl(positionInfo); + Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId()); if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } @@ -3333,9 +3333,9 @@ private void buildBatchEntryDeletionIndexInfoList( return; } int count = 0; - Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); + Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); while (iterator.hasNext() && count < maxBatchDeletedIndexToPersist) { - Map.Entry entry = iterator.next(); + Map.Entry entry = iterator.next(); long[] array = entry.getValue().toLongArray(); consumer.acceptRange(entry.getKey().getLedgerId(), entry.getKey().getEntryId(), array); count++; @@ -3436,7 +3436,7 @@ private void writeToBookKeeperLastChunk(LedgerHandle lh, VoidCallback callback, ByteBuf data, int totalLength, - PositionImpl position, + Position position, Runnable onFinished) { lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> { try { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java index 2cf68ac92b57d..96c363cb28990 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java @@ -23,6 +23,8 @@ import java.nio.charset.StandardCharsets; import java.util.Map; import java.util.function.Consumer; + +import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; final class PositionInfoUtils { @@ -35,7 +37,7 @@ interface BatchedEntryDeletionIndexInfoConsumer { void acceptRange(long ledgerId, long entryId, long[] array); } - static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, PositionImpl position, + static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, Position position, Consumer rangeScanner, Consumer batchDeletedIndexesScanner, int lastSerializedSize) { 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 ccb7ddb20f9b5..a46193e93292c 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 @@ -3634,7 +3634,7 @@ public void testRecoverCursorCorruptLastEntry() throws Exception { assertEquals(c.getReadPosition().getEntryId(), 0); assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1); - c.resetCursor(PositionImpl.LATEST); + c.resetCursor(PositionFactory.LATEST); // A reset cursor starts out with these values. The rest of the test assumes this, so we assert it here. assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); @@ -3647,7 +3647,7 @@ public void testRecoverCursorCorruptLastEntry() throws Exception { ml.addEntry(new byte[1]); ml.addEntry(new byte[1]); - c.resetCursor(PositionImpl.LATEST); + c.resetCursor(PositionFactory.LATEST); //corrupt last entry LedgerHandle cursorLedger = (LedgerHandle)FieldUtils.readDeclaredField(c, "cursorLedger", true); // can't parse json diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java index dc953676ca947..6332092d61a57 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.List; +import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.PositionFactory; import org.apache.bookkeeper.mledger.proto.LightMLDataFormats; import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -38,7 +40,7 @@ public class PositionInfoUtilsTest { @Test public void testSerializeDeserialize() throws Exception { - PositionImpl position = new PositionImpl(1, 2); + Position position = PositionFactory.create(1, 2); ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, Map.of("foo", 1L), null, null); @@ -79,7 +81,7 @@ public void testSerializeDeserialize() throws Exception { @Test public void testSerializeDeserializeEmpty() throws Exception { - PositionImpl position = new PositionImpl(1, 2); + Position position = PositionFactory.create(1, 2); ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, null, null, null); @@ -100,7 +102,7 @@ public void testSerializeDeserializeEmpty() throws Exception { @Test public void testSerializeDeserialize2() throws Exception { - PositionImpl position = new PositionImpl(1, 2); + Position position = PositionFactory.create(1, 2); ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position, Map.of("foo", 1L), null, null); From 1397fafb04429a4ce2509758622630b0aa0a775c Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Mon, 23 Sep 2024 15:50:40 -0700 Subject: [PATCH 16/22] removed usage of byte[] where possible --- .../mledger/impl/ManagedCursorImpl.java | 84 +++++++++++-------- .../mledger/impl/ManagedCursorTest.java | 6 +- 2 files changed, 50 insertions(+), 40 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 870a1ea224791..82d56f81ce3ae 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 @@ -39,10 +39,8 @@ import io.netty.buffer.ByteBufUtil; import io.netty.buffer.CompositeByteBuf; import io.netty.buffer.Unpooled; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.time.Clock; import java.util.ArrayDeque; import java.util.ArrayList; @@ -636,7 +634,7 @@ private void recoverFromLedgerByEntryId(ManagedCursorInfo info, } LedgerEntry entry = seq.nextElement(); - byte[] data = entry.getEntry(); + ByteBuf data = entry.getEntryBuffer(); try { ChunkSequenceFooter chunkSequenceFooter = parseChunkSequenceFooter(data); if (chunkSequenceFooter.numParts > 0) { @@ -672,23 +670,28 @@ private void readChunkSequence(VoidCallback callback, LedgerHandle lh, lh.asyncReadEntries(startPos, endPos, new AsyncCallback.ReadCallback() { @Override public void readComplete(int rc, LedgerHandle lh, Enumeration entries, Object ctx) { - ByteArrayOutputStream buffer = new ByteArrayOutputStream(); + CompositeByteBuf buffer = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); + + AtomicInteger readableBytes = new AtomicInteger(0); entries.asIterator().forEachRemaining(entry -> { - log.info("pos {} len {} bytes ", entry.getEntryId(), entry.getLength()); - try { - buffer.write(entry.getEntry()); - } catch (IOException err) { - throw new RuntimeException(err); + if (log.isInfoEnabled()) { + log.debug("pos {} len {} bytes ", entry.getEntryId(), entry.getLength()); } + ByteBuf part = entry.getEntryBuffer(); + buffer.addComponent(part); + readableBytes.addAndGet(part.readableBytes()); }); - byte[] result = buffer.toByteArray(); + buffer.writerIndex(readableBytes.get()) + .readerIndex(0); + log.info("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, - result.length, chunkSequenceFooter.length); - if (result.length != chunkSequenceFooter.length) { + buffer.readableBytes(), chunkSequenceFooter.length); + if (buffer.readableBytes() != chunkSequenceFooter.length) { callback.operationFailed(ManagedLedgerException.getManagedLedgerException(new IOException( - "Expected " + chunkSequenceFooter.length + " bytes but read " + result.length + " bytes"))); + "Expected " + chunkSequenceFooter.length + " bytes but read " + + buffer.readableBytes() + " bytes"))); } - Throwable res = tryCompleteCursorRecovery(lh, result); + Throwable res = tryCompleteCursorRecovery(lh, buffer); if (res == null) { callback.operationComplete(); } else { @@ -709,20 +712,28 @@ public static final class ChunkSequenceFooter { private int length; } - private ChunkSequenceFooter parseChunkSequenceFooter(byte[] data) throws IOException { - if (data.length == 0 || data[0] != '{') { + private ChunkSequenceFooter parseChunkSequenceFooter(ByteBuf data) throws IOException { + // getChar() doesn't move the reader index + if (data.readableBytes() == 0 || data.getByte(0) != '{') { // this is not JSON return ChunkSequenceFooter.NOT_CHUNKED; } - return ObjectMapperFactory.getMapper().getObjectMapper().readValue(data, ChunkSequenceFooter.class); + + try { + return ObjectMapperFactory.getMapper().getObjectMapper() + .readValue(data.toString(StandardCharsets.UTF_8), ChunkSequenceFooter.class); + } catch (JsonProcessingException e) { + return ChunkSequenceFooter.NOT_CHUNKED; + } } - private Throwable tryCompleteCursorRecovery(LedgerHandle lh, byte[] data) { - mbean.addReadCursorLedgerSize(data.length); + private Throwable tryCompleteCursorRecovery(LedgerHandle lh, ByteBuf data) { + mbean.addReadCursorLedgerSize(data.readableBytes()); try { data = decompressDataIfNeeded(data, lh); } catch (Throwable e) { + data.release(); log.error("[{}] Failed to decompress position info from ledger {} for cursor {}: {}", ledger.getName(), lh.getId(), name, e); return e; @@ -730,11 +741,13 @@ private Throwable tryCompleteCursorRecovery(LedgerHandle lh, byte[] data) { PositionInfo positionInfo; try { - positionInfo = PositionInfo.parseFrom(data); + positionInfo = PositionInfo.parseFrom(data.nioBuffer()); } catch (InvalidProtocolBufferException e) { log.error("[{}] Failed to parse position info from ledger {} for cursor {}: {}", ledger.getName(), lh.getId(), name, e); return e; + } finally { + data.release(); } Map recoveredProperties = Collections.emptyMap(); @@ -3492,42 +3505,39 @@ private ByteBuf compressDataIfNeeded(ByteBuf data, LedgerHandle lh) { result.readerIndex(0) .writerIndex(4 + compressedSize); - int ratio = (int) (compressedSize * 100.0 / uncompressedSize); - log.info("[{}] Cursor {} Compressed data size {} bytes (with {}, original size {} bytes, ratio {}%)", - ledger.getName(), name, compressedSize, pulsarCursorInfoCompressionString, uncompressedSize, ratio); + if (log.isInfoEnabled()) { + int ratio = (int) (compressedSize * 100.0 / uncompressedSize); + log.info("[{}] Cursor {} Compressed data size {} bytes (with {}, original size {} bytes, ratio {}%)", + ledger.getName(), name, compressedSize, pulsarCursorInfoCompressionString, + uncompressedSize, ratio); + } return result; } finally { data.release(); } } - static byte[] decompressDataIfNeeded(byte[] data, LedgerHandle lh) { + static ByteBuf decompressDataIfNeeded(ByteBuf data, LedgerHandle lh) { byte[] pulsarCursorInfoCompression = lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE); if (pulsarCursorInfoCompression != null) { String pulsarCursorInfoCompressionString = new String(pulsarCursorInfoCompression); if (log.isDebugEnabled()) { log.debug("Ledger {} compression {} decompressing {} bytes, full {}", - lh.getId(), pulsarCursorInfoCompressionString, data.length, + lh.getId(), pulsarCursorInfoCompressionString, data.readableBytes(), ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(data))); } - ByteArrayInputStream input = new ByteArrayInputStream(data); - DataInputStream dataInputStream = new DataInputStream(input); try { - int uncompressedSize = dataInputStream.readInt(); - byte[] compressedData = dataInputStream.readAllBytes(); + // this moves readerIndex + int uncompressedSize = data.readInt(); CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec( CompressionType.valueOf(pulsarCursorInfoCompressionString)); - ByteBuf decode = compressionCodec.decode(Unpooled.wrappedBuffer(compressedData), uncompressedSize); - try { - return ByteBufUtil.getBytes(decode); - } finally { - decode.release(); - } + ByteBuf decode = compressionCodec.decode(data, uncompressedSize); + return decode; } catch (IOException | MalformedInputException error) { log.error("Cannot decompress cursor position using {}. Payload is {}", pulsarCursorInfoCompressionString, - ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(data)), error); + ByteBufUtil.prettyHexDump(data), error); throw new RuntimeException(error); } } 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 a46193e93292c..96c6a3e2cd42f 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 @@ -149,7 +149,7 @@ public void testCloseCursor() throws Exception { ledger.addEntry(new byte[]{3}); ledger.addEntry(new byte[]{4}); ledger.addEntry(new byte[]{5}); - // Persistent cursor info to ledger. + // Persist cursor info to ledger. c1.delete(PositionFactory.create(c1.getReadPosition().getLedgerId(), c1.getReadPosition().getEntryId())); Awaitility.await().until(() ->c1.getStats().getPersistLedgerSucceed() > 0); // Make cursor ledger can not work. @@ -3276,9 +3276,9 @@ public void operationFailed(MetaStoreException e) { try { LedgerEntry entry = seq.nextElement(); PositionInfo positionInfo; - byte[] data = entry.getEntry(); + ByteBuf data = entry.getEntryBuffer(); data = ManagedCursorImpl.decompressDataIfNeeded(data, lh); - positionInfo = PositionInfo.parseFrom(data); + positionInfo = PositionInfo.parseFrom(data.nioBuffer()); individualDeletedMessagesCount.set(positionInfo.getIndividualDeletedMessagesCount()); } catch (Exception e) { } From d4b419502b3a10d9e6dac229b92d0ca9a9e44d42 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Tue, 24 Sep 2024 14:44:05 -0700 Subject: [PATCH 17/22] added config parameters for teh chunk size and to enable/disable chinking --- .../mledger/ManagedLedgerConfig.java | 37 +++++++++++++++++++ .../mledger/ManagedLedgerFactoryConfig.java | 10 +++++ .../mledger/impl/ManagedCursorImpl.java | 9 +++-- .../mledger/impl/ManagedCursorTest.java | 10 +++-- .../pulsar/broker/ServiceConfiguration.java | 11 +++++- .../broker/ManagedLedgerClientFactory.java | 4 ++ .../pulsar/broker/service/BrokerService.java | 4 ++ 7 files changed, 77 insertions(+), 8 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 03439f93ccad8..356302a463c59 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -45,7 +45,9 @@ public class ManagedLedgerConfig { private boolean createIfMissing = true; private int maxUnackedRangesToPersist = 10000; private int maxBatchDeletedIndexToPersist = 10000; + private String cursorInfoCompressionType = "NONE"; private boolean persistentUnackedRangesWithMultipleEntriesEnabled = false; + private int persistentUnackedRangesMaxEntrySize = 1024 * 1024; private boolean deletionAtBatchIndexLevelEnabled = true; private int maxUnackedRangesToPersistInMetadataStore = 1000; private int maxEntriesPerLedger = 50000; @@ -480,14 +482,49 @@ public int getMaxBatchDeletedIndexToPersist() { return maxBatchDeletedIndexToPersist; } + /** + * @return true if persistent unacked ranges with multiple entries enabled. + */ public boolean isPersistentUnackedRangesWithMultipleEntriesEnabled() { return persistentUnackedRangesWithMultipleEntriesEnabled; } + /** + * If enabled, the maximum "acknowledgment holes" will be stored in multiple entries, allowing the higher limits. + * @param multipleEntriesEnabled + */ public void setPersistentUnackedRangesWithMultipleEntriesEnabled(boolean multipleEntriesEnabled) { this.persistentUnackedRangesWithMultipleEntriesEnabled = multipleEntriesEnabled; } + /** + * @return max entry size for persistent unacked ranges. + */ + public int getPersistentUnackedRangesMaxEntrySize() { + return persistentUnackedRangesMaxEntrySize; + } + + /** + * If persistentUnackedRangesWithMultipleEntriesEnabled, this sets maximum entry size for storage in bytes. + */ + public void setPersistentUnackedRangesMaxEntrySize(int persistentUnackedRangesMaxEntrySize) { + this.persistentUnackedRangesMaxEntrySize = persistentUnackedRangesMaxEntrySize; + } + + /** + * @return compression type to use for cursor info. + */ + public String getCursorInfoCompressionType() { + return cursorInfoCompressionType; + } + + /** + * Set the compression type to use for cursor info. + */ + public void setCursorInfoCompressionType(String cursorInfoCompressionType) { + this.cursorInfoCompressionType = cursorInfoCompressionType; + } + /** * @param maxUnackedRangesToPersist * max unacked message ranges that will be persisted and receverd. 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 386310b3ccbae..e422ab6032331 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 @@ -108,6 +108,16 @@ public class ManagedLedgerFactoryConfig { */ private long managedCursorInfoCompressionThresholdInBytes = 0; + /** + * If enabled, the maximum "acknowledgment holes" will be stored in multiple entries, allowing the higher limits. + */ + private boolean persistentUnackedRangesWithMultipleEntriesEnabled = false; + + /** + * If persistentUnackedRangesWithMultipleEntriesEnabled, this sets maximum entry size for storage in bytes. + */ + private int persistentUnackedRangesMaxEntrySize = 1024 * 1024; + public MetadataCompressionConfig getCompressionConfigForManagedLedgerInfo() { return new MetadataCompressionConfig(managedLedgerInfoCompressionType, managedLedgerInfoCompressionThresholdInBytes); 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 82d56f81ce3ae..10e746b0fef00 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 @@ -149,6 +149,7 @@ public class ManagedCursorImpl implements ManagedCursor { protected final ManagedLedgerImpl ledger; private final String name; private final String cursorInfoCompressionType; + private final boolean isChunkingEnabled; private volatile Map cursorProperties; private final BookKeeper.DigestType digestType; @@ -241,7 +242,7 @@ public class ManagedCursorImpl implements ManagedCursor { // active state cache in ManagedCursor. It should be in sync with the state in activeCursors in ManagedLedger. private volatile boolean isActive = false; - protected int maxPositionChunkSize = 1024 * 1024; + private final int maxPositionChunkSize; static class MarkDeleteEntry { final Position newPosition; @@ -355,7 +356,9 @@ public interface VoidCallback { markDeleteLimiter = null; } this.mbean = new ManagedCursorMXBeanImpl(this); - this.cursorInfoCompressionType = ledger.getFactory().getConfig().getManagedCursorInfoCompressionType(); + this.cursorInfoCompressionType = getConfig().getCursorInfoCompressionType(); + this.isChunkingEnabled = getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled(); + this.maxPositionChunkSize = getConfig().getPersistentUnackedRangesMaxEntrySize(); } private void updateCursorLedgerStat(ManagedCursorInfo cursorInfo, Stat stat) { @@ -3382,7 +3385,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin int offset = 0; final int len = data.readableBytes(); - int numParts = 1 + (len / maxPositionChunkSize); + int numParts = isChunkingEnabled ? 1 + (len / maxPositionChunkSize) : 1; if (log.isDebugEnabled()) { log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {}", 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 96c6a3e2cd42f..fd7209b8e252e 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 @@ -3624,10 +3624,14 @@ public void operationFailed(ManagedLedgerException exception) { @Test(timeOut = 20000) public void testRecoverCursorCorruptLastEntry() throws Exception { - ManagedLedger ml = factory.open("testRecoverCursorCorruptLastEntry"); - ManagedCursorImpl c = (ManagedCursorImpl) ml.openCursor("sub", CommandSubscribe.InitialPosition.Latest); // force chunking - c.maxPositionChunkSize = 2; + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setPersistentUnackedRangesWithMultipleEntriesEnabled(true); + config.setPersistentUnackedRangesMaxEntrySize(2); + config.setCursorInfoCompressionType("LZ4"); + + ManagedLedger ml = factory.open("testRecoverCursorCorruptLastEntry", config); + ManagedCursorImpl c = (ManagedCursorImpl) ml.openCursor("sub", CommandSubscribe.InitialPosition.Latest); // A new cursor starts out with these values. The rest of the test assumes this, so we assert it here. assertEquals(c.getMarkDeletedPosition().getEntryId(), -1); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 33b4fbff5f5bb..632fcfac6b62f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2212,9 +2212,16 @@ The max allowed delay for delayed delivery (in milliseconds). If the broker rece private int managedLedgerMaxUnackedRangesToPersist = 10000; @FieldContext( category = CATEGORY_STORAGE_ML, - doc = "If enabled, the maximum \"acknowledgment holes\" will not be limited and \"acknowledgment holes\" " - + "are stored in multiple entries.") + doc = "If enabled, the maximum \"acknowledgment holes\" will be stored in multiple entries, " + + "allowing the higher limits.") private boolean persistentUnackedRangesWithMultipleEntriesEnabled = false; + + @FieldContext( + category = CATEGORY_STORAGE_ML, + doc = "If persistentUnackedRangesWithMultipleEntriesEnabled, " + + "this sets maximum entry size for storage in bytes.") + private int persistentUnackedRangesMaxEntrySize = 1024 * 1024; + @Deprecated @FieldContext( category = CATEGORY_STORAGE_ML, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java index 9bbc2857863ff..55d86506f4428 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/ManagedLedgerClientFactory.java @@ -81,6 +81,10 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata managedLedgerFactoryConfig.setManagedCursorInfoCompressionType(conf.getManagedCursorInfoCompressionType()); managedLedgerFactoryConfig.setManagedCursorInfoCompressionThresholdInBytes( conf.getManagedCursorInfoCompressionThresholdInBytes()); + managedLedgerFactoryConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( + conf.isPersistentUnackedRangesWithMultipleEntriesEnabled()); + managedLedgerFactoryConfig.setPersistentUnackedRangesMaxEntrySize( + conf.getPersistentUnackedRangesMaxEntrySize()); Configuration configuration = new ClientConfiguration(); if (conf.isBookkeeperClientExposeStatsToPrometheus()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 09f04d878c4e5..05fdc36f089b8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1934,6 +1934,10 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); + managedLedgerConfig.setPersistentUnackedRangesMaxEntrySize( + serviceConfig.getPersistentUnackedRangesMaxEntrySize()); + managedLedgerConfig.setCursorInfoCompressionType( + serviceConfig.getManagedCursorInfoCompressionType()); managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInMetadataStore()); managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); From 2fdbe63815d163bd458744c90961daafd7149121 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Fri, 27 Sep 2024 15:10:52 -0700 Subject: [PATCH 18/22] CR feedback, addComponent(true, ..) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 10e746b0fef00..203b87fbf2418 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 @@ -675,17 +675,13 @@ private void readChunkSequence(VoidCallback callback, LedgerHandle lh, public void readComplete(int rc, LedgerHandle lh, Enumeration entries, Object ctx) { CompositeByteBuf buffer = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); - AtomicInteger readableBytes = new AtomicInteger(0); entries.asIterator().forEachRemaining(entry -> { if (log.isInfoEnabled()) { log.debug("pos {} len {} bytes ", entry.getEntryId(), entry.getLength()); } ByteBuf part = entry.getEntryBuffer(); - buffer.addComponent(part); - readableBytes.addAndGet(part.readableBytes()); + buffer.addComponent(true, part); }); - buffer.writerIndex(readableBytes.get()) - .readerIndex(0); log.info("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, buffer.readableBytes(), chunkSequenceFooter.length); @@ -3503,10 +3499,8 @@ private ByteBuf compressDataIfNeeded(ByteBuf data, LedgerHandle lh) { ByteBuf szBuf = PulsarByteBufAllocator.DEFAULT.buffer(4).writeInt(uncompressedSize); CompositeByteBuf result = PulsarByteBufAllocator.DEFAULT.compositeBuffer(2); - result.addComponent(szBuf) - .addComponent(encode); - result.readerIndex(0) - .writerIndex(4 + compressedSize); + result.addComponent(true, szBuf) + .addComponent(true, encode); if (log.isInfoEnabled()) { int ratio = (int) (compressedSize * 100.0 / uncompressedSize); From 9b8880150856e7278c198e8626ef47dd0e7d7a9d Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Mon, 30 Sep 2024 12:12:52 -0700 Subject: [PATCH 19/22] Updated broker.conf with new entries --- conf/broker.conf | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 617e202e5ec65..433dce8759277 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1941,10 +1941,17 @@ managedLedgerMaxUnackedRangesToPersistInZooKeeper=-1 # persist, it will help to reduce the duplicates caused by the ack state that can not be fully persistent. dispatcherPauseOnAckStatePersistentEnabled=false -# If enabled, the maximum "acknowledgment holes" will not be limited and "acknowledgment holes" are stored in -# multiple entries. +# If enabled, the maximum "acknowledgment holes" (as defined by managedLedgerMaxUnackedRangesToPersist) +# can be stored in multiple entries, allowing the higher limits. persistentUnackedRangesWithMultipleEntriesEnabled=false +# If persistentUnackedRangesWithMultipleEntriesEnabled, this sets maximum entry size for storage in bytes. +#persistentUnackedRangesMaxEntrySize=1048576 + +# Set the compression type to use for cursor info. +# Possible options are NONE, LZ4, ZLIB, ZSTD, SNAPPY +#cursorInfoCompressionType=NONE + # Deprecated - Use managedLedgerCacheEvictionIntervalMs instead managedLedgerCacheEvictionFrequency=0 From 7918f212e710449d702bb29d260494303b3244df Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Tue, 1 Oct 2024 16:16:43 -0700 Subject: [PATCH 20/22] updated configs with docs and new config values, including the standalone.conf --- conf/broker.conf | 4 ++-- conf/standalone.conf | 11 +++++++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 433dce8759277..b9f32f3e6afce 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1946,11 +1946,11 @@ dispatcherPauseOnAckStatePersistentEnabled=false persistentUnackedRangesWithMultipleEntriesEnabled=false # If persistentUnackedRangesWithMultipleEntriesEnabled, this sets maximum entry size for storage in bytes. -#persistentUnackedRangesMaxEntrySize=1048576 +persistentUnackedRangesMaxEntrySize=1048576 # Set the compression type to use for cursor info. # Possible options are NONE, LZ4, ZLIB, ZSTD, SNAPPY -#cursorInfoCompressionType=NONE +cursorInfoCompressionType=NONE # Deprecated - Use managedLedgerCacheEvictionIntervalMs instead managedLedgerCacheEvictionFrequency=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index 535800a43f3e0..d117d1072c207 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -826,6 +826,17 @@ managedLedgerMaxUnackedRangesToPersist=10000 # MetadataStore. managedLedgerMaxUnackedRangesToPersistInMetadataStore=1000 +# If enabled, the maximum "acknowledgment holes" (as defined by managedLedgerMaxUnackedRangesToPersist) +# can be stored in multiple entries, allowing the higher limits. +persistentUnackedRangesWithMultipleEntriesEnabled=false + +# If persistentUnackedRangesWithMultipleEntriesEnabled, this sets maximum entry size for storage in bytes. +persistentUnackedRangesMaxEntrySize=1048576 + +# Set the compression type to use for cursor info. +# Possible options are NONE, LZ4, ZLIB, ZSTD, SNAPPY +cursorInfoCompressionType=NONE + # Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets # corrupted at bookkeeper and managed-cursor is stuck at that ledger. autoSkipNonRecoverableData=false From 590c5ac5c0173a12a82d0ba3ef6618d7e3293ccf Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Thu, 3 Oct 2024 09:59:01 -0700 Subject: [PATCH 21/22] info logging to debug --- .../mledger/impl/ManagedCursorImpl.java | 283 +++++++++--------- .../mledger/impl/MetaStoreImpl.java | 12 +- 2 files changed, 157 insertions(+), 138 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index 1c4523ac79424..f140bd930f745 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 @@ -179,18 +179,18 @@ public class ManagedCursorImpl implements ManagedCursor { protected volatile MarkDeleteEntry lastMarkDeleteEntry; protected static final AtomicReferenceFieldUpdater WAITING_READ_OP_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, OpReadEntry.class, "waitingReadOp"); + AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, OpReadEntry.class, "waitingReadOp"); @SuppressWarnings("unused") private volatile OpReadEntry waitingReadOp = null; public static final int FALSE = 0; public static final int TRUE = 1; private static final AtomicIntegerFieldUpdater RESET_CURSOR_IN_PROGRESS_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "resetCursorInProgress"); + AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "resetCursorInProgress"); @SuppressWarnings("unused") private volatile int resetCursorInProgress = FALSE; private static final AtomicIntegerFieldUpdater PENDING_READ_OPS_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingReadOps"); + AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingReadOps"); @SuppressWarnings("unused") private volatile int pendingReadOps = 0; @@ -258,7 +258,7 @@ static class MarkDeleteEntry { List callbackGroup; public MarkDeleteEntry(Position newPosition, Map properties, - MarkDeleteCallback callback, Object ctx) { + MarkDeleteCallback callback, Object ctx) { this.newPosition = newPosition; this.properties = properties; this.callback = callback; @@ -293,7 +293,7 @@ public void triggerFailed(ManagedLedgerException exception) { protected final ArrayDeque pendingMarkDeleteOps = new ArrayDeque<>(); private static final AtomicIntegerFieldUpdater PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER = - AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingMarkDeletedSubmittedCount"); + AtomicIntegerFieldUpdater.newUpdater(ManagedCursorImpl.class, "pendingMarkDeletedSubmittedCount"); @SuppressWarnings("unused") private volatile int pendingMarkDeletedSubmittedCount = 0; private volatile long lastLedgerSwitchTimestamp; @@ -312,7 +312,7 @@ public enum State { } protected static final AtomicReferenceFieldUpdater STATE_UPDATER = - AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, State.class, "state"); + AtomicReferenceFieldUpdater.newUpdater(ManagedCursorImpl.class, State.class, "state"); protected volatile State state = null; protected final ManagedCursorMXBean mbean; @@ -601,7 +601,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac null); } catch (Throwable t) { log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", - ledger.getName(), ledgerId, name, t); + ledger.getName(), ledgerId, name, t); openCallback.openComplete(BKException.Code.UnexpectedConditionException, null, null); } } @@ -670,28 +670,34 @@ private void readChunkSequence(VoidCallback callback, LedgerHandle lh, long footerPosition, ChunkSequenceFooter chunkSequenceFooter) { long startPos = footerPosition - chunkSequenceFooter.numParts; long endPos = footerPosition - 1; - log.info("readChunkSequence from pos {}, num parts {}, startPos {}, endPos {}", - footerPosition, chunkSequenceFooter.numParts, startPos, endPos); + if (log.isDebugEnabled()) { + log.debug("readChunkSequence from pos {}, num parts {}, startPos {}, endPos {}", + footerPosition, chunkSequenceFooter.numParts, startPos, endPos); + } lh.asyncReadEntries(startPos, endPos, new AsyncCallback.ReadCallback() { @Override public void readComplete(int rc, LedgerHandle lh, Enumeration entries, Object ctx) { CompositeByteBuf buffer = PulsarByteBufAllocator.DEFAULT.compositeBuffer(); entries.asIterator().forEachRemaining(entry -> { - if (log.isInfoEnabled()) { + if (log.isDebugEnabled()) { log.debug("pos {} len {} bytes ", entry.getEntryId(), entry.getLength()); } ByteBuf part = entry.getEntryBuffer(); buffer.addComponent(true, part); }); - log.info("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, - buffer.readableBytes(), chunkSequenceFooter.length); if (buffer.readableBytes() != chunkSequenceFooter.length) { + log.warn("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, + buffer.readableBytes(), chunkSequenceFooter.length); callback.operationFailed(ManagedLedgerException.getManagedLedgerException(new IOException( "Expected " + chunkSequenceFooter.length + " bytes but read " + buffer.readableBytes() + " bytes"))); } + if (log.isDebugEnabled()) { + log.debug("Read {} chunks, total of {} bytes, expected {} bytes", chunkSequenceFooter.numParts, + buffer.readableBytes(), chunkSequenceFooter.length); + } Throwable res = tryCompleteCursorRecovery(lh, buffer); if (res == null) { callback.operationComplete(); @@ -744,7 +750,7 @@ public void recoverIndividualDeletedMessages(PositionInfo positionInfo) { } } - private Throwable tryCompleteCursorRecovery(LedgerHandle lh, ByteBuf data) { + private Throwable tryCompleteCursorRecovery(LedgerHandle lh, ByteBuf data) { mbean.addReadCursorLedgerSize(data.readableBytes()); try { @@ -788,8 +794,10 @@ private Throwable tryCompleteCursorRecovery(LedgerHandle lh, ByteBuf data) { } private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) { - log.info("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}", - ledger.getName(), name, individualDeletedMessagesList.size()); + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}", + ledger.getName(), name, individualDeletedMessagesList.size()); + } lock.writeLock().lock(); try { individualDeletedMessages.clear(); @@ -826,7 +834,7 @@ private void recoverIndividualDeletedMessages(List i } } - private void recoverBatchDeletedIndexes ( + private void recoverBatchDeletedIndexes( List batchDeletedIndexInfoList) { lock.writeLock().lock(); try { @@ -895,11 +903,12 @@ public void operationComplete(Void result, Stat stat) { STATE_UPDATER.set(ManagedCursorImpl.this, State.NoLedger); callback.operationComplete(); } + @Override public void operationFailed(MetaStoreException e) { callback.operationFailed(e); } - }, false); + }, false); } @Override @@ -952,7 +961,7 @@ public void asyncReadEntries(int numberOfEntriesToRead, long maxSizeBytes, ReadE @Override public void asyncReadEntriesWithSkip(int numberOfEntriesToRead, long maxSizeBytes, ReadEntriesCallback callback, - Object ctx, Position maxPosition, Predicate skipCondition) { + Object ctx, Position maxPosition, Predicate skipCondition) { checkArgument(numberOfEntriesToRead > 0); if (isClosed()) { callback.readEntriesFailed(new ManagedLedgerException @@ -1013,7 +1022,7 @@ public String toString() { @Override public void asyncGetNthEntry(int n, IndividualDeletedEntries deletedEntries, ReadEntryCallback callback, - Object ctx) { + Object ctx) { checkArgument(n > 0); if (isClosed()) { callback.readEntryFailed(new ManagedLedgerException @@ -1368,7 +1377,7 @@ public void findEntryComplete(Position position, Object ctx) { @Override public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, - Object ctx) { + Object ctx) { result.exception = exception; counter.countDown(); } @@ -1384,28 +1393,29 @@ public void findEntryFailed(ManagedLedgerException exception, Optional @Override public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, - FindEntryCallback callback, Object ctx) { + FindEntryCallback callback, Object ctx) { asyncFindNewestMatching(constraint, condition, callback, ctx, false); } @Override public void asyncFindNewestMatching(FindPositionConstraint constraint, Predicate condition, - FindEntryCallback callback, Object ctx, boolean isFindFromLedger) { + FindEntryCallback callback, Object ctx, boolean isFindFromLedger) { OpFindNewest op; Position startPosition = null; long max = 0; switch (constraint) { - case SearchAllAvailableEntries: - startPosition = getFirstPosition(); - max = ledger.getNumberOfEntries() - 1; - break; - case SearchActiveEntries: - startPosition = ledger.getNextValidPosition(markDeletePosition); - max = getNumberOfEntriesInStorage(); - break; - default: - callback.findEntryFailed(new ManagedLedgerException("Unknown position constraint"), Optional.empty(), ctx); - return; + case SearchAllAvailableEntries: + startPosition = getFirstPosition(); + max = ledger.getNumberOfEntries() - 1; + break; + case SearchActiveEntries: + startPosition = ledger.getNextValidPosition(markDeletePosition); + max = getNumberOfEntriesInStorage(); + break; + default: + callback.findEntryFailed(new ManagedLedgerException("Unknown position constraint"), + Optional.empty(), ctx); + return; } if (startPosition == null) { callback.findEntryFailed(new ManagedLedgerException("Couldn't find start position"), @@ -1518,7 +1528,7 @@ public void operationComplete() { ledger.getName(), newReadPosition, oldReadPosition, name); } else { log.info("[{}] reset readPosition to {} skipping from current read readPosition {} on " - + "cursor {}", ledger.getName(), newReadPosition, oldReadPosition, name); + + "cursor {}", ledger.getName(), newReadPosition, oldReadPosition, name); } readPosition = newReadPosition; ledger.onCursorReadPositionUpdated(ManagedCursorImpl.this, newReadPosition); @@ -1555,16 +1565,16 @@ public void operationFailed(ManagedLedgerException exception) { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, getProperties(), null, null); internalAsyncMarkDelete(newMarkDeletePosition, isCompactionCursor() ? getProperties() : Collections.emptyMap(), new MarkDeleteCallback() { - @Override - public void markDeleteComplete(Object ctx) { - finalCallback.operationComplete(); - } + @Override + public void markDeleteComplete(Object ctx) { + finalCallback.operationComplete(); + } - @Override - public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { - finalCallback.operationFailed(exception); - } - }, null); + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + finalCallback.operationFailed(exception); + } + }, null); } @Override @@ -1667,17 +1677,16 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { * Async replays given positions: a. before reading it filters out already-acked messages b. reads remaining entries * async and gives it to given ReadEntriesCallback c. returns all already-acked messages which are not replayed so, * those messages can be removed by caller(Dispatcher)'s replay-list and it won't try to replay it again - * */ @Override public Set asyncReplayEntries(final Set positions, - ReadEntriesCallback callback, Object ctx) { + ReadEntriesCallback callback, Object ctx) { return asyncReplayEntries(positions, callback, ctx, false); } @Override public Set asyncReplayEntries(Set positions, - ReadEntriesCallback callback, Object ctx, boolean sortEntries) { + ReadEntriesCallback callback, Object ctx, boolean sortEntries) { List entries = Lists.newArrayListWithExpectedSize(positions.size()); if (positions.isEmpty()) { callback.readEntriesComplete(entries, ctx); @@ -1737,7 +1746,7 @@ public String toString() { }; positions.stream().filter(position -> !alreadyAcknowledgedPositions.contains(position)) - .forEach(p ->{ + .forEach(p -> { if (p.compareTo(this.readPosition) == 0) { this.setReadPosition(this.readPosition.getNext()); log.warn("[{}][{}] replayPosition{} equals readPosition{}," + " need set next readPosition", @@ -1923,7 +1932,7 @@ public void skipEntriesFailed(ManagedLedgerException exception, Object ctx) { @Override public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries deletedEntries, - final SkipEntriesCallback callback, Object ctx) { + final SkipEntriesCallback callback, Object ctx) { log.info("[{}] Skipping {} entries on cursor {}", ledger.getName(), numEntriesToSkip, name); long numDeletedMessages = 0; if (deletedEntries == IndividualDeletedEntries.Exclude) { @@ -1932,25 +1941,25 @@ public void asyncSkipEntries(int numEntriesToSkip, IndividualDeletedEntries dele asyncMarkDelete(ledger.getPositionAfterN(markDeletePosition, numEntriesToSkip + numDeletedMessages, PositionBound.startExcluded), new MarkDeleteCallback() { - @Override - public void markDeleteComplete(Object ctx) { - callback.skipEntriesComplete(ctx); - } + @Override + public void markDeleteComplete(Object ctx) { + callback.skipEntriesComplete(ctx); + } - @Override - public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { - if (exception.getCause() instanceof IllegalArgumentException) { - // There could be a race condition between calling clear backlog and other mark delete - // operations. - // If we get an exception it means the backlog was already cleared in the meantime. - callback.skipEntriesComplete(ctx); - } else { - log.error("[{}] Skip {} entries failed for cursor {}", ledger.getName(), numEntriesToSkip, - name, exception); - callback.skipEntriesFailed(exception, ctx); - } - } - }, ctx); + @Override + public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { + if (exception.getCause() instanceof IllegalArgumentException) { + // There could be a race condition between calling clear backlog and other mark delete + // operations. + // If we get an exception it means the backlog was already cleared in the meantime. + callback.skipEntriesComplete(ctx); + } else { + log.error("[{}] Skip {} entries failed for cursor {}", ledger.getName(), numEntriesToSkip, + name, exception); + callback.skipEntriesFailed(exception, ctx); + } + } + }, ctx); } // required in getNumIndividualDeletedEntriesToSkip method @@ -2025,9 +2034,7 @@ void initializeCursorPosition(Pair lastPositionCounter) { } /** - * - * @param newMarkDeletePosition - * the new acknowledged position + * @param newMarkDeletePosition the new acknowledged position * @return the previous acknowledged position */ Position setAcknowledgedPosition(Position newMarkDeletePosition) { @@ -2111,7 +2118,7 @@ public MarkDeletingMarkedPosition(String s) { @Override public void asyncMarkDelete(final Position position, Map properties, - final MarkDeleteCallback callback, final Object ctx) { + final MarkDeleteCallback callback, final Object ctx) { requireNonNull(position); if (isClosed()) { @@ -2195,7 +2202,7 @@ public void asyncMarkDelete(final Position position, Map propertie } else { if (log.isDebugEnabled()) { log.debug("[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {}" - + " for cursor [{}]", ledger.getName(), position, ledger.getLastConfirmedEntry(), name); + + " for cursor [{}]", ledger.getName(), position, ledger.getLastConfirmedEntry(), name); } callback.markDeleteFailed(new ManagedLedgerException("Invalid mark deleted position"), ctx); return; @@ -2223,7 +2230,7 @@ public void asyncMarkDelete(final Position position, Map propertie } protected void internalAsyncMarkDelete(final Position newPosition, Map properties, - final MarkDeleteCallback callback, final Object ctx) { + final MarkDeleteCallback callback, final Object ctx) { ledger.mbean.addMarkDeleteOp(); MarkDeleteEntry mdEntry = new MarkDeleteEntry(newPosition, properties, callback, ctx); @@ -2232,35 +2239,35 @@ protected void internalAsyncMarkDelete(final Position newPosition, Map 0) { - // Wait until no read operation are pending + case SwitchingLedger: pendingMarkDeleteOps.add(mdEntry); - } else { - // Execute the mark delete immediately - internalMarkDelete(mdEntry); - } - break; + break; - default: - log.error("[{}][{}] Invalid cursor state: {}", ledger.getName(), name, state); - callback.markDeleteFailed(new ManagedLedgerException("Cursor was in invalid state: " + state), ctx); - break; + case Open: + if (PENDING_READ_OPS_UPDATER.get(this) > 0) { + // Wait until no read operation are pending + pendingMarkDeleteOps.add(mdEntry); + } else { + // Execute the mark delete immediately + internalMarkDelete(mdEntry); + } + break; + + default: + log.error("[{}][{}] Invalid cursor state: {}", ledger.getName(), name, state); + callback.markDeleteFailed(new ManagedLedgerException("Cursor was in invalid state: " + state), ctx); + break; } } } @@ -2330,7 +2337,7 @@ public void operationComplete() { if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { Map subMap = batchDeletedIndexes.subMap(PositionFactory.EARLIEST, false, PositionFactory.create(mdEntry.newPosition.getLedgerId(), - mdEntry.newPosition.getEntryId()), true); + mdEntry.newPosition.getEntryId()), true); subMap.values().forEach(BitSetRecyclable::recycle); subMap.clear(); } @@ -2453,12 +2460,13 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } for (Position pos : positions) { - Position position = requireNonNull(pos); + Position position = requireNonNull(pos); if (ledger.getLastConfirmedEntry().compareTo(position) < 0) { if (log.isDebugEnabled()) { log.debug( - "[{}] Failed mark delete due to invalid markDelete {} is ahead of last-confirmed-entry {} " - + "for cursor [{}]", ledger.getName(), position, ledger.getLastConfirmedEntry(), name); + "[{}] Failed mark delete due to invalid markDelete {} is ahead of " + + "last-confirmed-entry {} for cursor [{}]", + ledger.getName(), position, ledger.getLastConfirmedEntry(), name); } callback.deleteFailed(new ManagedLedgerException("Invalid mark deleted position"), ctx); return; @@ -2488,12 +2496,12 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb // make the RangeSet recognize the "continuity" between adjacent Positions. Position previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), - previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); + previousPosition.getEntryId(), position.getLedgerId(), position.getEntryId()); MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); if (log.isDebugEnabled()) { log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, - individualDeletedMessages); + individualDeletedMessages); } } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(ackSet); @@ -2505,8 +2513,8 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb if (bitSet.isEmpty()) { Position previousPosition = ledger.getPreviousPosition(position); individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(), - previousPosition.getEntryId(), - position.getLedgerId(), position.getEntryId()); + previousPosition.getEntryId(), + position.getLedgerId(), position.getEntryId()); MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this); BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { @@ -2622,8 +2630,7 @@ private void updateLastMarkDeleteEntryToLatest(final Position newPosition, /** * Given a list of entries, filter out the entries that have already been individually deleted. * - * @param entries - * a list of entries + * @param entries a list of entries * @return a list of entries not containing deleted messages */ List filterReadEntries(List entries) { @@ -2800,7 +2807,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { * @param ctx */ void persistPositionWhenClosing(Position position, Map properties, - final AsyncCallbacks.CloseCallback callback, final Object ctx) { + final AsyncCallbacks.CloseCallback callback, final Object ctx) { if (shouldPersistUnackRangesToLedger()) { persistPositionToLedger(cursorLedger, new MarkDeleteEntry(position, properties, null, null), @@ -2852,14 +2859,18 @@ && getConfig().getMaxUnackedRangesToPersist() > 0 } private void persistPositionMetaStore(long cursorsLedgerId, Position position, Map properties, - MetaStoreCallback callback, boolean persistIndividualDeletedMessageRanges) { + MetaStoreCallback callback, + boolean persistIndividualDeletedMessageRanges) { if (state == State.Closed) { ledger.getExecutor().execute(() -> callback.operationFailed(new MetaStoreException( new CursorAlreadyClosedException(name + " cursor already closed")))); return; } - log.info("[{}][{}] Persisting cursor metadata into metadata store (persistIndividualDeletedMessageRanges: {})", - ledger.getName(), name, persistIndividualDeletedMessageRanges); + if (log.isDebugEnabled()) { + log.debug("[{}][{}] Persisting cursor metadata into metadata store " + + "(persistIndividualDeletedMessageRanges: {})", + ledger.getName(), name, persistIndividualDeletedMessageRanges); + } final Stat lastCursorLedgerStat = cursorLedgerStat; @@ -3223,8 +3234,10 @@ private static List buildStringPropertiesMap(Map private List buildIndividualDeletedMessageRanges(boolean forMetastore) { lock.writeLock().lock(); try { - log.info("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {}", - ledger.getName(), name, individualDeletedMessages.size()); + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {}", + ledger.getName(), name, individualDeletedMessages.size()); + } if (individualDeletedMessages.isEmpty()) { this.individualDeletedMessagesSerializedSize = 0; return Collections.emptyList(); @@ -3274,11 +3287,12 @@ private List buildIndividualDeletedMessageRanges(boo this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); - log.info("[{}] [{}] buildIndividualDeletedMessageRanges, rangeListSize {} " - + "maxUnackedRangesToPersist {}", - ledger.getName(), name, rangeList.size(), - getConfig().getMaxUnackedRangesToPersist()); - + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] buildIndividualDeletedMessageRanges, rangeListSize {} " + + "maxUnackedRangesToPersist {}", + ledger.getName(), name, rangeList.size(), + getConfig().getMaxUnackedRangesToPersist()); + } return rangeList; } finally { lock.writeLock().unlock(); @@ -3306,12 +3320,14 @@ private void scanIndividualDeletedMessageRanges( this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); individualDeletedMessages.resetDirtyKeys(); - log.info("[{}] [{}] scanIndividualDeletedMessageRanges, " - + "rangeListSize {} " - + "maxUnackedRangesToPersist {}", - ledger.getName(), name, - rangeCount.get(), - getConfig().getMaxUnackedRangesToPersist()); + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] scanIndividualDeletedMessageRanges, " + + "rangeListSize {} " + + "maxUnackedRangesToPersist {}", + ledger.getName(), name, + rangeCount.get(), + getConfig().getMaxUnackedRangesToPersist()); + } } finally { lock.readLock().unlock(); } @@ -3441,16 +3457,13 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin int numParts = isChunkingEnabled ? 1 + (len / maxPositionChunkSize) : 1; if (log.isDebugEnabled()) { - log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, numParts {}", - ledger.getName(), name, lh.getId(), - position, len, numParts); + log.debug("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, " + + "numParts {}, serializeTime {} ms" + + " compressTime {} ms, total {} ms", ledger.getName(), name, lh.getId(), + position, len, numParts, + (endSer - now) / 1000000, + (endCompress - endSer) / 1000000, (endCompress - now) / 1000000); } - log.info("[{}] Cursor {} Appending to ledger={} position={} data size {} bytes, " - + "numParts {}, serializeTime {} ms" - + " compressTime {} ms, total {} ms", ledger.getName(), name, lh.getId(), - position, len, numParts, - (endSer - now) / 1000000, - (endCompress - endSer) / 1000000, (endCompress - now) / 1000000); if (numParts == 1) { // no need for chunking @@ -3628,7 +3641,9 @@ boolean rolloverLedgerIfNeeded(LedgerHandle lh1) { } void persistPositionToMetaStore(MarkDeleteEntry mdEntry, final VoidCallback callback) { - log.info("[{}][{}] Persisting cursor metadata into metadata store", ledger.getName(), name); + if (log.isDebugEnabled()) { + log.debug("[{}][{}] Persisting cursor metadata into metadata store", ledger.getName(), name); + } final Position newPosition = mdEntry.newPosition; STATE_UPDATER.compareAndSet(ManagedCursorImpl.this, State.Open, State.NoLedger); mbean.persistToLedger(false); 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 28e60cbf5b95c..16234eeb52ce9 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 @@ -249,8 +249,10 @@ public void asyncUpdateCursorInfo(String ledgerName, String cursorName, ManagedC String path = PREFIX + ledgerName + "/" + cursorName; byte[] content = compressCursorInfo(info); - log.info("[{}] Persisting cursor={} info with content size {} bytes to metastore", - ledgerName, cursorName, content.length); + if (log.isDebugEnabled()) { + log.debug("[{}] Persisting cursor={} info with content size {} bytes to metastore", + ledgerName, cursorName, content.length); + } long expectedVersion; @@ -528,8 +530,10 @@ private byte[] compressManagedInfo(byte[] info, byte[] metadata, int metadataSer compositeByteBuf.addComponent(true, encodeByteBuf); byte[] dataBytes = new byte[compositeByteBuf.readableBytes()]; compositeByteBuf.readBytes(dataBytes); - log.info("Compressed cursor info, info size {}, metadata size {}, compressed size: {}", - info.length, metadata.length, dataBytes.length); + if (log.isDebugEnabled()) { + log.debug("Compressed cursor info, info size {}, metadata size {}, compressed size: {}", + info.length, metadata.length, dataBytes.length); + } return dataBytes; } finally { if (metadataByteBuf != null) { From 54157d8ee57d3011b19f193ac31a6dc40bf903a3 Mon Sep 17 00:00:00 2001 From: Andrey Yegorov Date: Thu, 3 Oct 2024 14:27:43 -0700 Subject: [PATCH 22/22] CR feedback --- .../mledger/impl/ManagedCursorImpl.java | 55 +++++++++++-------- .../mledger/impl/PositionInfoUtils.java | 8 ++- .../mledger/impl/PositionInfoUtilsTest.java | 9 ++- 3 files changed, 43 insertions(+), 29 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index f140bd930f745..0922e6d63763d 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 @@ -639,31 +639,37 @@ private void recoverFromLedgerByEntryId(ManagedCursorInfo info, } LedgerEntry entry = seq.nextElement(); - ByteBuf data = entry.getEntryBuffer(); - try { - ChunkSequenceFooter chunkSequenceFooter = parseChunkSequenceFooter(data); - if (chunkSequenceFooter.numParts > 0) { - readChunkSequence(callback, lh, entryId, chunkSequenceFooter); + recoverFromEntry(info, callback, lh, entryId, entry); + }, null); + } + + private void recoverFromEntry(ManagedCursorInfo info, VoidCallback callback, LedgerHandle lh, + long entryId, LedgerEntry entry) { + ByteBuf data = entry.getEntryBuffer(); + try { + ChunkSequenceFooter chunkSequenceFooter = parseChunkSequenceFooter(data); + if (chunkSequenceFooter.numParts > 0) { + data.release(); + readChunkSequence(callback, lh, entryId, chunkSequenceFooter); + } else { + // data is released in tryCompleteCursorRecovery + Throwable res = tryCompleteCursorRecovery(lh, data); + if (res == null) { + callback.operationComplete(); } else { - Throwable res = tryCompleteCursorRecovery(lh, data); - if (res == null) { - callback.operationComplete(); - } else { - log.warn("[{}] Error recovering from metadata ledger {} entry {} for cursor {}. " - + "Will try recovery from previous entry.", - ledger.getName(), ledgerId, entryId, name, res); - //try recovery from previous entry - recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); - } + log.warn("[{}] Error recovering from metadata ledger {} entry {} for cursor {}. " + + "Will try recovery from previous entry.", + ledger.getName(), lh.getId(), entryId, name, res); + //try recovery from previous entry + recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); } - } catch (IOException error) { - log.error("Cannot parse footer", error); - log.warn("[{}] Error recovering from metadata ledger {} entry {} for cursor {}, cannot parse footer. " - + "Will try recovery from previous entry.", - ledger.getName(), ledgerId, entryId, name, error); - recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); } - }, null); + } catch (IOException error) { + log.error("[{}] Error recovering from metadata ledger {} entry {} for cursor {}, cannot parse footer. " + + "Will try recovery from previous entry.", + ledger.getName(), lh.getId(), entryId, name, error); + recoverFromLedgerByEntryId(info, callback, lh, entryId - 1); + } } private void readChunkSequence(VoidCallback callback, LedgerHandle lh, @@ -702,6 +708,8 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration entri if (res == null) { callback.operationComplete(); } else { + log.error("[{}] Error recovering from metadata ledger {} entry {} for cursor {}", + ledger.getName(), lh.getId(), footerPosition, name, res); callback.operationFailed(new ManagedLedgerException(res)); } } @@ -3313,8 +3321,7 @@ private void scanIndividualDeletedMessageRanges( } individualDeletedMessages.forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> { - acksSerializedSize.addAndGet(16 * 4); - consumer.acceptRange(lowerKey, lowerValue, upperKey, upperValue); + consumer.acceptRange(lowerKey, lowerValue, upperKey, upperValue, acksSerializedSize); return rangeCount.incrementAndGet() <= maxUnackedRangesToPersist; }); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java index dead4e71bb0a9..c8b5d8381dd2e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java @@ -22,6 +22,7 @@ import io.netty.buffer.ByteBufUtil; import java.nio.charset.StandardCharsets; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -29,7 +30,8 @@ final class PositionInfoUtils { interface IndividuallyDeletedMessagesRangeConsumer { - void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId, long upperEntryId); + void acceptRange(long lowerLegerId, long lowerEntryId, + long upperLedgerId, long upperEntryId, AtomicInteger acksSerializedSize); } interface IndividuallyDeletedRangesConsumer { @@ -56,7 +58,8 @@ static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, MessageRange _item = new MessageRange(); rangeScanner.accept(new IndividuallyDeletedMessagesRangeConsumer() { @Override - public void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId, long upperEntryId) { + public void acceptRange(long lowerLegerId, long lowerEntryId, + long upperLedgerId, long upperEntryId, AtomicInteger acksSerializedSize) { _item.clear(); NestedPositionInfo lower = _item.setLowerEndpoint(); NestedPositionInfo upper = _item.setUpperEndpoint(); @@ -65,6 +68,7 @@ public void acceptRange(long lowerLegerId, long lowerEntryId, long upperLedgerId upper.setLedgerId(upperLedgerId); upper.setEntryId(upperEntryId); LightProtoCodec.writeVarInt(_b, PositionInfo._INDIVIDUAL_DELETED_MESSAGES_TAG); + acksSerializedSize.addAndGet(_item.getSerializedSize()); LightProtoCodec.writeVarInt(_b, _item.getSerializedSize()); _item.writeTo(_b); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java index efd029027d16e..8c1e504d484fd 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java @@ -25,6 +25,7 @@ import io.netty.buffer.ByteBufUtil; import java.util.Map; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.PositionFactory; @@ -38,6 +39,7 @@ public class PositionInfoUtilsTest { private static final Logger log = LoggerFactory.getLogger(PositionInfoUtilsTest.class); + final AtomicInteger counter = new AtomicInteger(0); @Test public void testSerializeDeserialize() throws Exception { Position position = PositionFactory.create(1, 2); @@ -45,8 +47,8 @@ public void testSerializeDeserialize() throws Exception { Map.of("foo", 1L), null, null); ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { - scanner.acceptRange(1, 2, 3, 4); - scanner.acceptRange(5, 6, 7, 8); + scanner.acceptRange(1, 2, 3, 4, counter); + scanner.acceptRange(5, 6, 7, 8, counter); }, (scanner) -> { long[] array = {7L, 8L}; scanner.acceptRange(1, 2, array); @@ -76,6 +78,7 @@ public void testSerializeDeserialize() throws Exception { assertEquals(1, positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getPosition().getLedgerId()); assertEquals(2, positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getPosition().getEntryId()); assertEquals(List.of(7L, 8L), positionInfoParsed.getBatchedEntryDeletionIndexInfo(0).getDeleteSetList()); + result.release(); } @@ -111,7 +114,7 @@ public void testSerializeDeserialize2() throws Exception { final int numRanges = 10000; ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> { for (int i = 0; i < numRanges; i++) { - scanner.acceptRange(i*4 + 1, i*4 + 2, i*4 + 3, i*4 + 4); + scanner.acceptRange(i*4 + 1, i*4 + 2, i*4 + 3, i*4 + 4, counter); } }, (scanner) -> { long[] array = {7L, 8L};