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/conf/broker.conf b/conf/broker.conf
index 617e202e5ec65..b9f32f3e6afce 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
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
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/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java
index a24251450b4f4..0c53afde0cac2 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/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 b39fd231cdc06..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
@@ -21,10 +21,12 @@
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;
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;
@@ -32,11 +34,19 @@
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;
+import io.netty.buffer.Unpooled;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
import java.time.Clock;
import java.util.ArrayDeque;
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;
@@ -61,6 +71,12 @@
import java.util.function.Predicate;
import java.util.stream.Collectors;
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;
import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
import org.apache.bookkeeper.client.AsyncCallback.OpenCallback;
import org.apache.bookkeeper.client.BKException;
@@ -91,6 +107,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.LongListMap;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.LongProperty;
@@ -98,13 +115,16 @@
import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.MessageRange;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo;
-import org.apache.bookkeeper.mledger.proto.MLDataFormats.PositionInfo.Builder;
import org.apache.bookkeeper.mledger.proto.MLDataFormats.StringProperty;
-import org.apache.commons.lang3.mutable.MutableInt;
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;
@@ -115,6 +135,7 @@
@SuppressWarnings("checkstyle:javadoctype")
public class ManagedCursorImpl implements ManagedCursor {
+
private static final Comparator ENTRY_COMPARATOR = (e1, e2) -> {
if (e1.getLedgerId() != e2.getLedgerId()) {
return e1.getLedgerId() < e2.getLedgerId() ? -1 : 1;
@@ -129,11 +150,14 @@ public class ManagedCursorImpl implements ManagedCursor {
protected final BookKeeper bookkeeper;
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;
protected volatile Position markDeletePosition;
+ private int lastSerializedSize;
// this position is have persistent mark delete position
protected volatile Position persistentMarkDeletePosition;
@@ -155,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;
@@ -220,7 +244,9 @@ 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;
- class MarkDeleteEntry {
+ private final int maxPositionChunkSize;
+
+ static class MarkDeleteEntry {
final Position newPosition;
final MarkDeleteCallback callback;
final Object ctx;
@@ -232,7 +258,7 @@ 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;
@@ -267,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;
@@ -286,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;
@@ -332,6 +358,9 @@ public interface VoidCallback {
markDeleteLimiter = null;
}
this.mbean = new ManagedCursorMXBeanImpl(this);
+ this.cursorInfoCompressionType = getConfig().getCursorInfoCompressionType();
+ this.isChunkingEnabled = getConfig().isPersistentUnackedRangesWithMultipleEntriesEnabled();
+ this.maxPositionChunkSize = getConfig().getPersistentUnackedRangesMaxEntrySize();
}
private void updateCursorLedgerStat(ManagedCursorInfo cursorInfo, Stat stat) {
@@ -465,6 +494,8 @@ public boolean removeProperty(String key) {
if (lastMarkDeleteEntry != null) {
LAST_MARK_DELETE_ENTRY_UPDATER.updateAndGet(this, last -> {
Map properties = last.properties;
+ // 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);
}
@@ -562,70 +593,152 @@ 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
+ 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);
+ }
+ }
+
+ 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;
+ }
+
+ 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));
+
+ callback.operationFailed(createManagedLedgerException(rc1));
+ return;
}
- 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));
+ LedgerEntry entry = seq.nextElement();
+ recoverFromEntry(info, callback, lh, entryId, entry);
+ }, null);
+ }
- callback.operationFailed(createManagedLedgerException(rc1));
- return;
+ 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 {
+ 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("[{}] 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);
+ }
+ }
- LedgerEntry entry = seq.nextElement();
- mbean.addReadCursorLedgerSize(entry.getLength());
- PositionInfo positionInfo;
- try {
- positionInfo = PositionInfo.parseFrom(entry.getEntry());
- } catch (InvalidProtocolBufferException e) {
- callback.operationFailed(new ManagedLedgerException(e));
- return;
- }
+ private void readChunkSequence(VoidCallback callback, LedgerHandle lh,
+ long footerPosition, ChunkSequenceFooter chunkSequenceFooter) {
+ long startPos = footerPosition - chunkSequenceFooter.numParts;
+ long endPos = footerPosition - 1;
+ 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();
- 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());
+ entries.asIterator().forEachRemaining(entry -> {
+ if (log.isDebugEnabled()) {
+ log.debug("pos {} len {} bytes ", entry.getEntryId(), entry.getLength());
}
- }
+ ByteBuf part = entry.getEntryBuffer();
+ buffer.addComponent(true, part);
+ });
- Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId());
- recoverIndividualDeletedMessages(positionInfo);
- if (getConfig().isDeletionAtBatchIndexLevelEnabled()
- && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) {
- recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList());
+ 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")));
}
- recoveredCursor(position, recoveredProperties, cursorProperties, lh);
- callback.operationComplete();
- }, null);
- };
+ 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();
+ } else {
+ log.error("[{}] Error recovering from metadata ledger {} entry {} for cursor {}",
+ ledger.getName(), lh.getId(), footerPosition, name, res);
+ callback.operationFailed(new ManagedLedgerException(res));
+ }
+ }
+ }, null);
+ }
+
+ @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(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;
+ }
+
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);
+ return ObjectMapperFactory.getMapper().getObjectMapper()
+ .readValue(data.toString(StandardCharsets.UTF_8), ChunkSequenceFooter.class);
+ } catch (JsonProcessingException e) {
+ return ChunkSequenceFooter.NOT_CHUNKED;
}
}
@@ -645,30 +758,54 @@ public void recoverIndividualDeletedMessages(PositionInfo positionInfo) {
}
}
- private List buildLongPropertiesMap(Map properties) {
- if (properties.isEmpty()) {
- return Collections.emptyList();
+ 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;
}
- List longListMap = new ArrayList<>();
- MutableInt serializedSize = new MutableInt();
- properties.forEach((id, ranges) -> {
- if (ranges == null || ranges.length <= 0) {
- return;
- }
- org.apache.bookkeeper.mledger.proto.MLDataFormats.LongListMap.Builder lmBuilder = LongListMap.newBuilder()
- .setKey(id);
- for (long range : ranges) {
- lmBuilder.addValues(range);
+
+ PositionInfo positionInfo;
+ try {
+ 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();
+ 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());
}
- LongListMap lm = lmBuilder.build();
- longListMap.add(lm);
- serializedSize.add(lm.getSerializedSize());
- });
- individualDeletedMessagesSerializedSize = serializedSize.toInteger();
- return longListMap;
+ }
+
+ Position position = PositionFactory.create(positionInfo.getLedgerId(), positionInfo.getEntryId());
+ recoverIndividualDeletedMessages(positionInfo);
+ if (getConfig().isDeletionAtBatchIndexLevelEnabled()
+ && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) {
+ recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList());
+ }
+ recoveredCursor(position, recoveredProperties, cursorProperties, lh);
+ return null;
}
private void recoverIndividualDeletedMessages(List individualDeletedMessagesList) {
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] [{}] Recovering individual deleted messages. Number of ranges: {}",
+ ledger.getName(), name, individualDeletedMessagesList.size());
+ }
lock.writeLock().lock();
try {
individualDeletedMessages.clear();
@@ -705,7 +842,7 @@ private void recoverIndividualDeletedMessages(List i
}
}
- private void recoverBatchDeletedIndexes (
+ private void recoverBatchDeletedIndexes(
List batchDeletedIndexInfoList) {
lock.writeLock().lock();
try {
@@ -774,11 +911,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
@@ -831,7 +969,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
@@ -892,7 +1030,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
@@ -1247,7 +1385,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();
}
@@ -1263,28 +1401,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"),
@@ -1397,7 +1536,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);
@@ -1434,16 +1573,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
@@ -1546,17 +1685,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 extends Position> asyncReplayEntries(final Set extends Position> positions,
- ReadEntriesCallback callback, Object ctx) {
+ ReadEntriesCallback callback, Object ctx) {
return asyncReplayEntries(positions, callback, ctx, false);
}
@Override
public Set extends Position> asyncReplayEntries(Set extends Position> 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);
@@ -1616,7 +1754,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",
@@ -1802,7 +1940,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) {
@@ -1811,25 +1949,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
@@ -1904,9 +2042,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) {
@@ -1990,7 +2126,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()) {
@@ -2006,7 +2142,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;
}
@@ -2074,7 +2210,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;
@@ -2102,7 +2238,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);
@@ -2111,35 +2247,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;
}
}
}
@@ -2209,7 +2345,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();
}
@@ -2332,12 +2468,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;
@@ -2367,12 +2504,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);
@@ -2384,8 +2521,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) {
@@ -2501,8 +2638,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) {
@@ -2679,7 +2815,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),
@@ -2731,12 +2867,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;
}
+ if (log.isDebugEnabled()) {
+ log.debug("[{}][{}] Persisting cursor metadata into metadata store "
+ + "(persistIndividualDeletedMessageRanges: {})",
+ ledger.getName(), name, persistIndividualDeletedMessageRanges);
+ }
final Stat lastCursorLedgerStat = cursorLedgerStat;
@@ -2751,7 +2893,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());
}
@@ -3037,7 +3179,7 @@ private CompletableFuture doCreateNewMetadataLedger() {
}
future.complete(lh);
});
- }, LedgerMetadataUtils.buildAdditionalMetadataForCursor(name));
+ }, LedgerMetadataUtils.buildAdditionalMetadataForCursor(name, cursorInfoCompressionType));
return future;
}
@@ -3057,6 +3199,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();
@@ -3085,9 +3239,13 @@ private static List buildStringPropertiesMap(Map
return stringProperties;
}
- private List buildIndividualDeletedMessageRanges() {
+ private List buildIndividualDeletedMessageRanges(boolean forMetastore) {
lock.writeLock().lock();
try {
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] [{}] buildIndividualDeletedMessageRanges, numRanges {}",
+ ledger.getName(), name, individualDeletedMessages.size());
+ }
if (individualDeletedMessages.isEmpty()) {
this.individualDeletedMessagesSerializedSize = 0;
return Collections.emptyList();
@@ -3118,20 +3276,70 @@ 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();
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] [{}] buildIndividualDeletedMessageRanges, rangeListSize {} "
+ + "maxUnackedRangesToPersist {}",
+ ledger.getName(), name, rangeList.size(),
+ getConfig().getMaxUnackedRangesToPersist());
+ }
return rangeList;
} finally {
lock.writeLock().unlock();
}
}
+ 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()) {
+ this.individualDeletedMessagesSerializedSize = 0;
+ return;
+ }
+
+ individualDeletedMessages.forEachRawRange((lowerKey, lowerValue, upperKey, upperValue) -> {
+ consumer.acceptRange(lowerKey, lowerValue, upperKey, upperValue, acksSerializedSize);
+ return rangeCount.incrementAndGet() <= maxUnackedRangesToPersist;
+ });
+
+ this.individualDeletedMessagesSerializedSize = acksSerializedSize.get();
+ individualDeletedMessages.resetDirtyKeys();
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] [{}] scanIndividualDeletedMessageRanges, "
+ + "rangeListSize {} "
+ + "maxUnackedRangesToPersist {}",
+ ledger.getName(), name,
+ rangeCount.get(),
+ getConfig().getMaxUnackedRangesToPersist());
+ }
+ } finally {
+ lock.readLock().unlock();
+ }
+ }
+
private List buildBatchEntryDeletionIndexInfoList() {
lock.readLock().lock();
try {
@@ -3164,58 +3372,252 @@ private List buildBatchEntryDeletio
}
}
+ private void buildBatchEntryDeletionIndexInfoList(
+ PositionInfoUtils.BatchedEntryDeletionIndexInfoConsumer consumer) {
+ if (!getConfig().isDeletionAtBatchIndexLevelEnabled()) {
+ return;
+ }
+ int maxBatchDeletedIndexToPersist = getConfig().getMaxBatchDeletedIndexToPersist();
+ lock.readLock().lock();
+ try {
+ if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) {
+ return;
+ }
+ int count = 0;
+ Iterator> iterator = batchDeletedIndexes.entrySet().iterator();
+ while (iterator.hasNext() && count < maxBatchDeletedIndexToPersist) {
+ Map.Entry entry = iterator.next();
+ long[] array = entry.getValue().toLongArray();
+ consumer.acceptRange(entry.getKey().getLedgerId(), entry.getKey().getEntryId(), array);
+ count++;
+ }
+ } finally {
+ lock.readLock().unlock();
+ }
+ }
+
+ private void scanIndividuallyDeletedRanges(Map internalRanges,
+ PositionInfoUtils.IndividuallyDeletedRangesConsumer
+ individuallyDeletedRangesConsumer) {
+ if (internalRanges == null || internalRanges.isEmpty()) {
+ return;
+ }
+
+ AtomicInteger serializedSize = new AtomicInteger(0);
+ internalRanges.forEach((ledgerId, ranges) -> {
+ serializedSize.addAndGet(16 * 4 + 8 * ranges.length);
+ individuallyDeletedRangesConsumer.acceptRange(ledgerId, ranges);
+ });
+ this.individualDeletedMessagesSerializedSize = serializedSize.get();
+ }
+
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;
- Builder piBuilder = PositionInfo.newBuilder().setLedgerId(position.getLedgerId())
- .setEntryId(position.getEntryId())
- .addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList())
- .addAllProperties(buildPropertiesMap(mdEntry.properties));
+
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(),
+ position);
+ }
+
+ requireNonNull(lh);
Map internalRanges = null;
try {
- internalRanges = individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist());
+ // to support downgrade this is hidden behind the feature flag
+ internalRanges = isChunkingEnabled
+ ? individualDeletedMessages.toRanges(getConfig().getMaxUnackedRangesToPersist())
+ : null;
} catch (Exception e) {
log.warn("[{}]-{} Failed to serialize individualDeletedMessages", ledger.getName(), name, e);
}
- if (internalRanges != null && !internalRanges.isEmpty()) {
- piBuilder.addAllIndividualDeletedMessageRanges(buildLongPropertiesMap(internalRanges));
+
+ final ByteBuf rawData;
+ if (internalRanges == null || internalRanges.isEmpty()) {
+ rawData = PositionInfoUtils.serializePositionInfo(mdEntry,
+ position,
+ this::scanIndividualDeletedMessageRanges,
+ this::buildBatchEntryDeletionIndexInfoList,
+ x -> {},
+ lastSerializedSize);
} else {
- piBuilder.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges());
+ final Map internalRangesConst = internalRanges;
+ rawData = PositionInfoUtils.serializePositionInfo(mdEntry,
+ position,
+ x -> {},
+ this::buildBatchEntryDeletionIndexInfoList,
+ x -> this.scanIndividuallyDeletedRanges(internalRangesConst, x),
+ lastSerializedSize);
}
- PositionInfo pi = piBuilder.build();
+ long endSer = System.nanoTime();
+ this.lastSerializedSize = rawData.readableBytes();
+
+ // rawData is released by compressDataIfNeeded if needed
+ ByteBuf data = compressDataIfNeeded(rawData, lh);
+
+ long endCompress = System.nanoTime();
+
+ int offset = 0;
+ final int len = data.readableBytes();
+ // to support downgrade this is hidden behind the feature flag
+ int numParts = isChunkingEnabled ? 1 + (len / maxPositionChunkSize) : 1;
if (log.isDebugEnabled()) {
- log.debug("[{}] Cursor {} Appending to ledger={} position={}", ledger.getName(), name, lh.getId(),
- position);
- }
+ 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);
+ }
+
+ if (numParts == 1) {
+ // no need for chunking
+ // asyncAddEntry will release data ByteBuf
+ writeToBookKeeperLastChunk(lh, mdEntry, callback, data, len, position, () -> {});
+ } else {
+ // chunking
+ int part = 0;
+ while (part != numParts) {
+ int remaining = len - offset;
+ int currentLen = Math.min(maxPositionChunkSize, remaining);
+ boolean isLast = part == numParts - 1;
- requireNonNull(lh);
- byte[] data = pi.toByteArray();
- 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());
+ 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);
}
- rolloverLedgerIfNeeded(lh1);
+ // 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);
- mbean.persistToLedger(true);
- mbean.addWriteCursorLedgerSize(data.length);
- 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);
+ 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), len, position, data::release);
+ }
+ offset += currentLen;
+ part++;
+ }
+ }
+ }
- // Before giving up, try to persist the position in the metadata store.
- persistPositionToMetaStore(mdEntry, callback);
+ private void writeToBookKeeperLastChunk(LedgerHandle lh,
+ MarkDeleteEntry mdEntry,
+ VoidCallback callback,
+ ByteBuf data,
+ int totalLength,
+ Position position,
+ Runnable onFinished) {
+ lh.asyncAddEntry(data, (rc, lh1, entryId, ctx) -> {
+ 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);
+
+ mbean.persistToLedger(true);
+ mbean.addWriteCursorLedgerSize(totalLength);
+ 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);
}
+ private ByteBuf compressDataIfNeeded(ByteBuf data, LedgerHandle lh) {
+ byte[] pulsarCursorInfoCompression =
+ lh.getCustomMetadata().get(METADATA_PROPERTY_CURSOR_COMPRESSION_TYPE);
+ 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(data);
+
+ int compressedSize = encode.readableBytes();
+
+ ByteBuf szBuf = PulsarByteBufAllocator.DEFAULT.buffer(4).writeInt(uncompressedSize);
+
+ CompositeByteBuf result = PulsarByteBufAllocator.DEFAULT.compositeBuffer(2);
+ result.addComponent(true, szBuf)
+ .addComponent(true, encode);
+
+ 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 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.readableBytes(),
+ ByteBufUtil.prettyHexDump(Unpooled.wrappedBuffer(data)));
+ }
+ try {
+ // this moves readerIndex
+ int uncompressedSize = data.readInt();
+ CompressionCodec compressionCodec = CompressionCodecProvider.getCompressionCodec(
+ CompressionType.valueOf(pulsarCursorInfoCompressionString));
+ ByteBuf decode = compressionCodec.decode(data, uncompressedSize);
+ return decode;
+ } catch (IOException | MalformedInputException error) {
+ log.error("Cannot decompress cursor position using {}. Payload is {}",
+ pulsarCursorInfoCompressionString,
+ ByteBufUtil.prettyHexDump(data), error);
+ throw new RuntimeException(error);
+ }
+ } else {
+ return data;
+ }
+ }
+
public boolean periodicRollover() {
LedgerHandle lh = cursorLedger;
if (State.Open.equals(STATE_UPDATER.get(this))
@@ -3246,6 +3648,9 @@ boolean rolloverLedgerIfNeeded(LedgerHandle lh1) {
}
void persistPositionToMetaStore(MarkDeleteEntry mdEntry, final VoidCallback callback) {
+ 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);
@@ -3276,7 +3681,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
@@ -3415,7 +3820,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());
@@ -3823,4 +4227,5 @@ public ManagedLedgerInternalStats.CursorStats getCursorStats() {
cs.properties = getProperties();
return cs;
}
+
}
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..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,6 +249,10 @@ public void asyncUpdateCursorInfo(String ledgerName, String cursorName, ManagedC
String path = PREFIX + ledgerName + "/" + cursorName;
byte[] content = compressCursorInfo(info);
+ if (log.isDebugEnabled()) {
+ log.debug("[{}] Persisting cursor={} info with content size {} bytes to metastore",
+ ledgerName, cursorName, content.length);
+ }
long expectedVersion;
@@ -267,6 +271,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;
@@ -453,8 +458,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 +485,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);
@@ -515,6 +530,10 @@ private byte[] compressManagedInfo(byte[] info, byte[] metadata, int metadataSer
compositeByteBuf.addComponent(true, encodeByteBuf);
byte[] dataBytes = new byte[compositeByteBuf.readableBytes()];
compositeByteBuf.readBytes(dataBytes);
+ 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) {
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..c8b5d8381dd2e
--- /dev/null
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtils.java
@@ -0,0 +1,1766 @@
+/*
+ * 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.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;
+
+final class PositionInfoUtils {
+
+ interface IndividuallyDeletedMessagesRangeConsumer {
+ void acceptRange(long lowerLegerId, long lowerEntryId,
+ long upperLedgerId, long upperEntryId, AtomicInteger acksSerializedSize);
+ }
+
+ interface IndividuallyDeletedRangesConsumer {
+ void acceptRange(long id, long[] ranges);
+ }
+
+ interface BatchedEntryDeletionIndexInfoConsumer {
+ void acceptRange(long ledgerId, long entryId, long[] array);
+ }
+
+ static ByteBuf serializePositionInfo(ManagedCursorImpl.MarkDeleteEntry mdEntry, Position position,
+ Consumer rangeScanner,
+ Consumer batchDeletedIndexesScanner,
+ Consumer compactRangesScanner,
+ int lastSerializedSize) {
+ int size = Math.max(lastSerializedSize, 64 * 1024);
+ ByteBuf _b = PulsarByteBufAllocator.DEFAULT.buffer(size);
+
+ 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();
+ rangeScanner.accept(new IndividuallyDeletedMessagesRangeConsumer() {
+ @Override
+ public void acceptRange(long lowerLegerId, long lowerEntryId,
+ long upperLedgerId, long upperEntryId, AtomicInteger acksSerializedSize) {
+ _item.clear();
+ NestedPositionInfo lower = _item.setLowerEndpoint();
+ NestedPositionInfo upper = _item.setUpperEndpoint();
+ lower.setLedgerId(lowerLegerId);
+ lower.setEntryId(lowerEntryId);
+ 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);
+ }
+ });
+
+ final LongProperty longProperty = new LongProperty();
+ Map properties = mdEntry.properties;
+ if (properties != null) {
+ properties.forEach((k, v) -> {
+ longProperty.clear();
+ 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();
+
+ 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);
+ 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);
+ }
+ });
+
+ LongListMap llMap = new LongListMap();
+ compactRangesScanner.accept(new IndividuallyDeletedRangesConsumer() {
+ @Override
+ public void acceptRange(long id, long[] ranges) {
+ if (ranges == null || ranges.length == 0) {
+ return;
+ }
+
+ llMap.clear();
+ llMap.setKey(id);
+ for (long l : ranges) {
+ llMap.addValue(l);
+ }
+
+ LightProtoCodec.writeVarInt(_b, PositionInfo._INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG);
+ LightProtoCodec.writeVarInt(_b, llMap.getSerializedSize());
+ llMap.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 java.util.List individualDeletedMessageRanges = null;
+ private int _individualDeletedMessageRangesCount = 0;
+ private static final int _INDIVIDUAL_DELETED_MESSAGE_RANGES_FIELD_NUMBER = 6;
+ private static final int _INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG = (_INDIVIDUAL_DELETED_MESSAGE_RANGES_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS)
+ | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED;
+ private static final int _INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG_SIZE = LightProtoCodec
+ .computeVarIntSize(_INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG);
+ public int getIndividualDeletedMessageRangesCount() {
+ return _individualDeletedMessageRangesCount;
+ }
+ public LongListMap getIndividualDeletedMessageRangeAt(int idx) {
+ if (idx < 0 || idx >= _individualDeletedMessageRangesCount) {
+ throw new IndexOutOfBoundsException("Index " + idx + " is out of the list size ("
+ + _individualDeletedMessageRangesCount + ") for field 'individualDeletedMessageRanges'");
+ }
+ return individualDeletedMessageRanges.get(idx);
+ }
+ public java.util.List getIndividualDeletedMessageRangesList() {
+ if (_individualDeletedMessageRangesCount == 0) {
+ return java.util.Collections.emptyList();
+ } else {
+ return individualDeletedMessageRanges.subList(0, _individualDeletedMessageRangesCount);
+ }
+ }
+ public LongListMap addIndividualDeletedMessageRange() {
+ if (individualDeletedMessageRanges == null) {
+ individualDeletedMessageRanges = new java.util.ArrayList();
+ }
+ if (individualDeletedMessageRanges.size() == _individualDeletedMessageRangesCount) {
+ individualDeletedMessageRanges.add(new LongListMap());
+ }
+ _cachedSize = -1;
+ return individualDeletedMessageRanges.get(_individualDeletedMessageRangesCount++);
+ }
+ public PositionInfo addAllIndividualDeletedMessageRanges(Iterable individualDeletedMessageRanges) {
+ for (LongListMap _o : individualDeletedMessageRanges) {
+ addIndividualDeletedMessageRange().copyFrom(_o);
+ }
+ return this;
+ }
+ public PositionInfo clearIndividualDeletedMessageRanges() {
+ for (int i = 0; i < _individualDeletedMessageRangesCount; i++) {
+ individualDeletedMessageRanges.get(i).clear();
+ }
+ _individualDeletedMessageRangesCount = 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);
+ }
+ for (int i = 0; i < _individualDeletedMessageRangesCount; i++) {
+ LongListMap _item = individualDeletedMessageRanges.get(i);
+ LightProtoCodec.writeVarInt(_b, _INDIVIDUAL_DELETED_MESSAGE_RANGES_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;
+ }
+ for (int i = 0; i < _individualDeletedMessageRangesCount; i++) {
+ LongListMap _item = individualDeletedMessageRanges.get(i);
+ _size += _INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG_SIZE;
+ int MsgsizeIndividualDeletedMessageRanges = _item.getSerializedSize();
+ _size += LightProtoCodec.computeVarIntSize(MsgsizeIndividualDeletedMessageRanges)
+ + MsgsizeIndividualDeletedMessageRanges;
+ }
+ _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;
+ case _INDIVIDUAL_DELETED_MESSAGE_RANGES_TAG :
+ int _individualDeletedMessageRangesSize = LightProtoCodec.readVarInt(_buffer);
+ addIndividualDeletedMessageRange().parseFrom(_buffer, _individualDeletedMessageRangesSize);
+ 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;
+ for (int i = 0; i < _individualDeletedMessageRangesCount; i++) {
+ individualDeletedMessageRanges.get(i).clear();
+ }
+ _individualDeletedMessageRangesCount = 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));
+ }
+ for (int i = 0; i < _other.getIndividualDeletedMessageRangesCount(); i++) {
+ addIndividualDeletedMessageRange().copyFrom(_other.getIndividualDeletedMessageRangeAt(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 LongListMap {
+ private long key;
+ private static final int _KEY_FIELD_NUMBER = 1;
+ private static final int _KEY_TAG = (_KEY_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS)
+ | LightProtoCodec.WIRETYPE_VARINT;
+ private static final int _KEY_TAG_SIZE = LightProtoCodec.computeVarIntSize(_KEY_TAG);
+ private static final int _KEY_MASK = 1 << (0 % 32);
+ public boolean hasKey() {
+ return (_bitField0 & _KEY_MASK) != 0;
+ }
+ public long getKey() {
+ if (!hasKey()) {
+ throw new IllegalStateException("Field 'key' is not set");
+ }
+ return key;
+ }
+ public LongListMap setKey(long key) {
+ this.key = key;
+ _bitField0 |= _KEY_MASK;
+ _cachedSize = -1;
+ return this;
+ }
+ public LongListMap clearKey() {
+ _bitField0 &= ~_KEY_MASK;
+ return this;
+ }
+
+ private long[] values = null;
+ private int _valuesCount = 0;
+ private static final int _VALUES_FIELD_NUMBER = 2;
+ private static final int _VALUES_TAG = (_VALUES_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS)
+ | LightProtoCodec.WIRETYPE_VARINT;
+ private static final int _VALUES_TAG_SIZE = LightProtoCodec.computeVarIntSize(_VALUES_TAG);
+ private static final int _VALUES_TAG_PACKED = (_VALUES_FIELD_NUMBER << LightProtoCodec.TAG_TYPE_BITS)
+ | LightProtoCodec.WIRETYPE_LENGTH_DELIMITED;
+ public int getValuesCount() {
+ return _valuesCount;
+ }
+ public long getValueAt(int idx) {
+ if (idx < 0 || idx >= _valuesCount) {
+ throw new IndexOutOfBoundsException(
+ "Index " + idx + " is out of the list size (" + _valuesCount + ") for field 'values'");
+ }
+ return values[idx];
+ }
+ public void addValue(long value) {
+ if (values == null) {
+ values = new long[4];
+ }
+ if (values.length == _valuesCount) {
+ values = java.util.Arrays.copyOf(values, _valuesCount * 2);
+ }
+ _cachedSize = -1;
+ values[_valuesCount++] = value;
+ }
+ public LongListMap clearValues() {
+ _valuesCount = 0;
+ return this;
+ }
+
+ private int _bitField0;
+ private static final int _REQUIRED_FIELDS_MASK0 = 0 | _KEY_MASK;
+ public int writeTo(io.netty.buffer.ByteBuf _b) {
+ checkRequiredFields();
+ int _writeIdx = _b.writerIndex();
+ LightProtoCodec.writeVarInt(_b, _KEY_TAG);
+ LightProtoCodec.writeVarInt64(_b, key);
+ for (int i = 0; i < _valuesCount; i++) {
+ long _item = values[i];
+ LightProtoCodec.writeVarInt(_b, _VALUES_TAG);
+ LightProtoCodec.writeVarInt64(_b, _item);
+ }
+ return (_b.writerIndex() - _writeIdx);
+ }
+ public int getSerializedSize() {
+ if (_cachedSize > -1) {
+ return _cachedSize;
+ }
+
+ int _size = 0;
+ _size += _KEY_TAG_SIZE;
+ _size += LightProtoCodec.computeVarInt64Size(key);
+ for (int i = 0; i < _valuesCount; i++) {
+ long _item = values[i];
+ _size += _VALUES_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 _KEY_TAG :
+ _bitField0 |= _KEY_MASK;
+ key = LightProtoCodec.readVarInt64(_buffer);
+ break;
+ case _VALUES_TAG :
+ addValue(LightProtoCodec.readVarInt64(_buffer));
+ break;
+ case _VALUES_TAG_PACKED :
+ int _valueSize = LightProtoCodec.readVarInt(_buffer);
+ int _valueEndIdx = _buffer.readerIndex() + _valueSize;
+ while (_buffer.readerIndex() < _valueEndIdx) {
+ addValue(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 LongListMap clear() {
+ _valuesCount = 0;
+ _parsedBuffer = null;
+ _cachedSize = -1;
+ _bitField0 = 0;
+ return this;
+ }
+ public LongListMap copyFrom(LongListMap _other) {
+ _cachedSize = -1;
+ if (_other.hasKey()) {
+ setKey(_other.key);
+ }
+ for (int i = 0; i < _other.getValuesCount(); i++) {
+ addValue(_other.getValueAt(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 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/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 1067cda441f6a..c92fce4b4dd07 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;
@@ -74,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;
@@ -98,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;
@@ -127,6 +130,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 {
@@ -141,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.
@@ -3268,7 +3276,9 @@ public void operationFailed(MetaStoreException e) {
try {
LedgerEntry entry = seq.nextElement();
PositionInfo positionInfo;
- positionInfo = PositionInfo.parseFrom(entry.getEntry());
+ ByteBuf data = entry.getEntryBuffer();
+ data = ManagedCursorImpl.decompressDataIfNeeded(data, lh);
+ positionInfo = PositionInfo.parseFrom(data.nioBuffer());
c1.recoverIndividualDeletedMessages(positionInfo);
individualDeletedMessagesCount.set(c1.getIndividuallyDeletedMessagesSet().asRanges().size());
} catch (Exception e) {
@@ -3469,6 +3479,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);
@@ -3607,6 +3622,86 @@ public void operationFailed(ManagedLedgerException exception) {
assertEquals(c.getReadPosition(), readPositionBeforeRecover);
assertEquals(c.getNumberOfEntries(), 2L);
}
+
+ @Test(timeOut = 20000)
+ public void testRecoverCursorCorruptLastEntry() throws Exception {
+ // force chunking
+ 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);
+ assertEquals(c.getReadPosition().getEntryId(), 0);
+ assertEquals(ml.getLastConfirmedEntry().getEntryId(), -1);
+
+ 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);
+ 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(PositionFactory.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
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..8c1e504d484fd
--- /dev/null
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/PositionInfoUtilsTest.java
@@ -0,0 +1,288 @@
+/*
+ * 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 com.google.protobuf.InvalidProtocolBufferException;
+import io.netty.buffer.ByteBuf;
+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;
+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);
+
+ final AtomicInteger counter = new AtomicInteger(0);
+ @Test
+ public void testSerializeDeserialize() throws Exception {
+ Position position = PositionFactory.create(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, counter);
+ scanner.acceptRange(5, 6, 7, 8, counter);
+ }, (scanner) -> {
+ long[] array = {7L, 8L};
+ scanner.acceptRange(1, 2, array);
+ }, scanner -> {}, 1024);
+
+ 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 {
+ Position position = PositionFactory.create(1, 2);
+ ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position,
+ null, null, null);
+
+ ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position,
+ scanner -> {},
+ scanner -> {},
+ scanner -> {},
+ 1024);
+
+ 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();
+ }
+
+ @Test
+ public void testSerializeDeserialize2() throws Exception {
+ Position position = PositionFactory.create(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, counter);
+ }
+ }, (scanner) -> {
+ long[] array = {7L, 8L};
+ for (int i = 0; i < numRanges; i++) {
+ scanner.acceptRange(i*2 + 1, i*2 + 2, array);
+ }
+ }, scanner -> {
+ long[] array = {7L, 8L};
+ for (int i = 0; i < numRanges; i++) {
+ scanner.acceptRange(i, 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();
+ }
+
+ @Test
+ public void testSerializeDeserialize3() throws Exception {
+ Position position = PositionFactory.create(1, 2);
+ ManagedCursorImpl.MarkDeleteEntry entry = new ManagedCursorImpl.MarkDeleteEntry(position,
+ Map.of("foo", 1L), null, null);
+
+ ByteBuf result = PositionInfoUtils.serializePositionInfo(entry, position, (scanner) -> {},
+ (scanner) -> {
+ long[] array = {7L, 8L};
+ scanner.acceptRange(1, 2, array);
+ }, scanner -> {
+ scanner.acceptRange(1L, new long[]{0, 1});
+ scanner.acceptRange(2L, new long[]{7, 8});
+ }, 1024);
+
+ 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(0, positionInfoParsed.getIndividualDeletedMessagesCount());
+
+ 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());
+
+ assertEquals(2, positionInfoParsed.getIndividualDeletedMessageRangesCount());
+ assertEquals(1L, positionInfoParsed.getIndividualDeletedMessageRanges(0).getKey());
+ assertEquals(2, positionInfoParsed.getIndividualDeletedMessageRanges(0).getValuesCount());
+ assertEquals(0L, positionInfoParsed.getIndividualDeletedMessageRanges(0).getValues(0));
+ assertEquals(1L, positionInfoParsed.getIndividualDeletedMessageRanges(0).getValues(1));
+
+ assertEquals(2L, positionInfoParsed.getIndividualDeletedMessageRanges(1).getKey());
+ assertEquals(7L, positionInfoParsed.getIndividualDeletedMessageRanges(1).getValues(0));
+ assertEquals(8L, positionInfoParsed.getIndividualDeletedMessageRanges(1).getValues(1));
+
+ result.release();
+ }
+
+ private static void validateLightproto(LightMLDataFormats.PositionInfo lightPositionInfoParsed, int numRanges) {
+ assertEquals(1, lightPositionInfoParsed.getLedgerId());
+ assertEquals(2, lightPositionInfoParsed.getEntryId());
+
+ assertEquals(1, lightPositionInfoParsed.getPropertiesCount());
+ assertEquals("foo", lightPositionInfoParsed.getPropertyAt(0).getName());
+ assertEquals(1, lightPositionInfoParsed.getPropertyAt(0).getValue());
+
+ assertEquals(numRanges, lightPositionInfoParsed.getIndividualDeletedMessagesCount());
+ int curr = 0;
+ for (int i = 0; i < numRanges; i++) {
+ assertEquals(i * 4 + 1, lightPositionInfoParsed.getIndividualDeletedMessageAt(curr).getLowerEndpoint().getLedgerId());
+ assertEquals(i * 4 + 2, lightPositionInfoParsed.getIndividualDeletedMessageAt(curr).getLowerEndpoint().getEntryId());
+ assertEquals(i * 4 + 3, lightPositionInfoParsed.getIndividualDeletedMessageAt(curr).getUpperEndpoint().getLedgerId());
+ assertEquals(i * 4 + 4, lightPositionInfoParsed.getIndividualDeletedMessageAt(curr).getUpperEndpoint().getEntryId());
+ curr++;
+ }
+
+ assertEquals(numRanges, lightPositionInfoParsed.getBatchedEntryDeletionIndexInfosCount());
+ curr = 0;
+ for (int i = 0; i < numRanges; i++) {
+ assertEquals(i * 2 + 1, lightPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getPosition().getLedgerId());
+ assertEquals(i * 2 + 2, lightPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getPosition().getEntryId());
+ assertEquals(7L, lightPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getDeleteSetAt(0));
+ assertEquals(8L, lightPositionInfoParsed.getBatchedEntryDeletionIndexInfoAt(curr).getDeleteSetAt(1));
+ curr++;
+ }
+
+ assertEquals(numRanges, lightPositionInfoParsed.getIndividualDeletedMessageRangesCount());
+ curr = 0;
+ for (LightMLDataFormats.LongListMap llmap : lightPositionInfoParsed.getIndividualDeletedMessageRangesList()) {
+ assertEquals(curr, llmap.getKey());
+ assertEquals(7L, llmap.getValueAt(0));
+ assertEquals(8L, llmap.getValueAt(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++;
+ }
+
+ assertEquals(numRanges, positionInfoParsed.getIndividualDeletedMessageRangesCount());
+ curr = 0;
+ for (MLDataFormats.LongListMap llmap: positionInfoParsed.getIndividualDeletedMessageRangesList()) {
+ assertEquals(curr, llmap.getKey());
+ assertEquals(7L, llmap.getValues(0));
+ assertEquals(8L, llmap.getValues(1));
+ curr++;
+ }
+ }
+
+}
\ No newline at end of file
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/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 bfa99eedcadce..d25ef9079d11f 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
@@ -1939,6 +1939,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());
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
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();
}