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 asyncReplayEntries(final Set positions, - ReadEntriesCallback callback, Object ctx) { + ReadEntriesCallback callback, Object ctx) { return asyncReplayEntries(positions, callback, ctx, false); } @Override public Set asyncReplayEntries(Set positions, - ReadEntriesCallback callback, Object ctx, boolean sortEntries) { + ReadEntriesCallback callback, Object ctx, boolean sortEntries) { List entries = Lists.newArrayListWithExpectedSize(positions.size()); if (positions.isEmpty()) { callback.readEntriesComplete(entries, ctx); @@ -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(); }