Class LedgerHandle

    • Field Detail

      • LOG

        static final org.slf4j.Logger LOG
      • ledgerKey

        final byte[] ledgerKey
      • ledgerId

        final long ledgerId
      • executor

        final java.util.concurrent.ExecutorService executor
      • lastAddPushed

        long lastAddPushed
      • notSupportBatch

        boolean notSupportBatch
      • lastAddConfirmed

        volatile long lastAddConfirmed
        Last entryId which has been confirmed to be written durably to the bookies. This value is used by readers, the LAC protocol
      • pendingAddsSequenceHead

        volatile long pendingAddsSequenceHead
        Next entryId which is expected to move forward during sendAddSuccessCallbacks(). This is important in order to have an ordered sequence of addEntry acknowledged to the writer
      • length

        long length
      • throttler

        final com.google.common.util.concurrent.RateLimiter throttler
      • bookieFailureHistory

        final com.google.common.cache.LoadingCache<BookieId,​java.lang.Long> bookieFailureHistory
      • writeFlags

        final java.util.EnumSet<WriteFlag> writeFlags
      • timeoutFuture

        java.util.concurrent.ScheduledFuture<?> timeoutFuture
      • delayedWriteFailedBookies

        final java.util.Map<java.lang.Integer,​BookieId> delayedWriteFailedBookies
      • INVALID_ENTRY_ID

        public static final long INVALID_ENTRY_ID
        Invalid entry id. This value is returned from methods which should return an entry id but there is no valid entry available.
        See Also:
        Constant Field Values
      • INVALID_LEDGER_ID

        public static final long INVALID_LEDGER_ID
        Invalid ledger id. Ledger IDs must be greater than or equal to 0. Large negative used to make it easy to spot in logs if erroneously used.
        See Also:
        Constant Field Values
      • metadataLock

        final java.lang.Object metadataLock
      • changingEnsemble

        boolean changingEnsemble
      • numEnsembleChanges

        final java.util.concurrent.atomic.AtomicInteger numEnsembleChanges
      • ensembleChangeCounter

        final Counter ensembleChangeCounter
      • lacUpdateHitsCounter

        final Counter lacUpdateHitsCounter
      • lacUpdateMissesCounter

        final Counter lacUpdateMissesCounter
    • Constructor Detail

      • LedgerHandle

        LedgerHandle​(ClientContext clientCtx,
                     long ledgerId,
                     Versioned<LedgerMetadata> versionedMetadata,
                     BookKeeper.DigestType digestType,
                     byte[] password,
                     java.util.EnumSet<WriteFlag> writeFlags)
              throws java.security.GeneralSecurityException,
                     java.lang.NumberFormatException
        Throws:
        java.security.GeneralSecurityException
        java.lang.NumberFormatException
    • Method Detail

      • recordReadErrorOnBookie

        void recordReadErrorOnBookie​(int bookieIndex)
        Notify the LedgerHandle that a read operation was failed on a particular bookie.
      • initializeWriteHandleState

        protected void initializeWriteHandleState()
      • getId

        public long getId()
        Get the id of the current ledger.
        Specified by:
        getId in interface Handle
        Returns:
        the id of the ledger
      • getWriteFlags

        public java.util.EnumSet<WriteFlag> getWriteFlags()
      • setLastAddConfirmed

        void setLastAddConfirmed​(long lac)
      • getLastAddPushed

        public long getLastAddPushed()
        Get the entry id of the last entry that has been enqueued for addition (but may not have possibly been persisted to the ledger).
        Specified by:
        getLastAddPushed in interface WriteHandle
        Returns:
        the entry id of the last entry pushed or -1 if no entry has been pushed
      • getLedgerKey

        public byte[] getLedgerKey()
        Get the Ledger's key/password.
        Returns:
        byte array for the ledger's key/password.
      • getLedgerMetadata

        public LedgerMetadata getLedgerMetadata()
        Returns the metadata of this ledger.

        This call only retrieves the metadata cached locally. If there is any metadata updated, the read handle will receive the metadata updates and update the metadata locally. The metadata notification can be deplayed, so it is possible you can receive a stale copy of ledger metadata from this call.

        Specified by:
        getLedgerMetadata in interface Handle
        Returns:
        the metadata of this ledger.
      • getCustomMetadata

        public java.util.Map<java.lang.String,​byte[]> getCustomMetadata()
        Get this ledger's customMetadata map.
        Returns:
        map containing user provided customMetadata.
      • getNumFragments

        public long getNumFragments()
        Get the number of fragments that makeup this ledger.
        Returns:
        the count of fragments
      • getNumBookies

        public long getNumBookies()
        Get the count of unique bookies that own part of this ledger by going over all the fragments of the ledger.
        Returns:
        count of unique bookies
      • getDigestManager

        DigestManager getDigestManager()
        Get the DigestManager.
        Returns:
        DigestManager for the LedgerHandle
      • addToLength

        long addToLength​(long delta)
        Add to the length of the ledger in bytes.
        Parameters:
        delta -
        Returns:
        the length of the ledger after the addition
      • getLength

        public long getLength()
        Returns the length of the ledger in bytes.
        Specified by:
        getLength in interface ReadHandle
        Returns:
        the length of the ledger in bytes
      • getCtime

        public long getCtime()
        Returns the ledger creation time.
        Returns:
        the ledger creation time
      • getDistributionSchedule

        DistributionSchedule getDistributionSchedule()
        Get the Distribution Schedule.
        Returns:
        DistributionSchedule for the LedgerHandle
      • getBookiesHealthInfo

        BookiesHealthInfo getBookiesHealthInfo()
        Get the health info for bookies for this ledger.
        Returns:
        BookiesHealthInfo for every bookie in the write set.
      • close

        public void close()
                   throws java.lang.InterruptedException,
                          BKException
        Synchronous close the write handle, any adds in flight will return errors.

        Closing a ledger will ensure that all clients agree on what the last entry of the ledger is. Once the ledger has been closed, all reads from the ledger will return the same set of entries.

        The close operation can error if it finds conflicting metadata when it tries to write to the metadata store. On close, the metadata state is set to closed and lastEntry and length of the ledger are fixed in the metadata. A conflict occurs if the metadata in the metadata store has a different value for the lastEntry or length. If another process has updated the metadata, setting it to closed, but have fixed the lastEntry and length to the same values as this process is trying to write, the operation completes successfully.

        Specified by:
        close in interface java.lang.AutoCloseable
        Specified by:
        close in interface Handle
        Specified by:
        close in interface WriteHandle
        Throws:
        java.lang.InterruptedException
        BKException
        See Also:
        Handle.closeAsync()
      • closeAsync

        public java.util.concurrent.CompletableFuture<java.lang.Void> closeAsync()
        Asynchronous close the write handle, any adds in flight will return errors.

        Closing a ledger will ensure that all clients agree on what the last entry of the ledger is. Once the ledger has been closed, all reads from the ledger will return the same set of entries.

        The close operation can error if it finds conflicting metadata when it tries to write to the metadata store. On close, the metadata state is set to closed and lastEntry and length of the ledger are fixed in the metadata. A conflict occurs if the metadata in the metadata store has a different value for the lastEntry or length. If another process has updated the metadata, setting it to closed, but have fixed the lastEntry and length to the same values as this process is trying to write, the operation completes successfully.

        Specified by:
        closeAsync in interface Handle
        Specified by:
        closeAsync in interface WriteHandle
        Returns:
        an handle to access the result of the operation
      • asyncClose

        public void asyncClose​(AsyncCallback.CloseCallback cb,
                               java.lang.Object ctx)
        Asynchronous close, any adds in flight will return errors.

        Closing a ledger will ensure that all clients agree on what the last entry of the ledger is. This ensures that, once the ledger has been closed, all reads from the ledger will return the same set of entries.

        Parameters:
        cb - callback implementation
        ctx - control object
      • isClosed

        public boolean isClosed()
        Returns whether the ledger is sealed or not.

        A ledger is sealed when either the client explicitly closes it (WriteHandle.close() or Handle.close()) or another client explicitly open and recovery it OpenBuilder.withRecovery(boolean).

        This method only checks the metadata cached locally. The metadata can be not update-to-date because the metadata notification is delayed.

        Specified by:
        isClosed in interface ReadHandle
        Returns:
        true if the ledger is sealed, otherwise false.
      • isHandleWritable

        boolean isHandleWritable()
      • doAsyncCloseInternal

        void doAsyncCloseInternal​(AsyncCallback.CloseCallback cb,
                                  java.lang.Object ctx,
                                  int rc)
        Same as public version of asyncClose except that this one takes an additional parameter which is the return code to hand to all the pending add ops.
        Parameters:
        cb -
        ctx -
        rc -
      • readEntries

        public java.util.Enumeration<LedgerEntry> readEntries​(long firstEntry,
                                                              long lastEntry)
                                                       throws java.lang.InterruptedException,
                                                              BKException
        Read a sequence of entries synchronously.
        Parameters:
        firstEntry - id of first entry of sequence (included)
        lastEntry - id of last entry of sequence (included)
        Throws:
        java.lang.InterruptedException
        BKException
        See Also:
        asyncReadEntries(long, long, ReadCallback, Object)
      • batchReadUnconfirmedEntries

        public java.util.Enumeration<LedgerEntry> batchReadUnconfirmedEntries​(long firstEntry,
                                                                              int maxCount,
                                                                              long maxSize)
                                                                       throws java.lang.InterruptedException,
                                                                              BKException
        Read a sequence of entries synchronously, allowing to read after the LastAddConfirmed range.
        This is the same of asyncBatchReadUnconfirmedEntries(long, int, long, ReadCallback, Object)
        Parameters:
        firstEntry - id of first entry of sequence (included)
        maxCount - id of last entry of sequence (included)
        maxSize - the total entries size
        Throws:
        java.lang.InterruptedException
        BKException
      • asyncReadEntries

        public void asyncReadEntries​(long firstEntry,
                                     long lastEntry,
                                     AsyncCallback.ReadCallback cb,
                                     java.lang.Object ctx)
        Read a sequence of entries asynchronously.
        Parameters:
        firstEntry - id of first entry of sequence
        lastEntry - id of last entry of sequence
        cb - object implementing read callback interface
        ctx - control object
      • asyncBatchReadEntries

        public void asyncBatchReadEntries​(long startEntry,
                                          int maxCount,
                                          long maxSize,
                                          AsyncCallback.ReadCallback cb,
                                          java.lang.Object ctx)
        Read a sequence of entries in asynchronously. It send an RPC to get all entries instead of send multi RPC to get all entries.
        Parameters:
        startEntry - id of first entry of sequence
        maxCount - the entries count
        maxSize - the total entries size
        cb - object implementing read callback interface
        ctx - control object
      • asyncReadUnconfirmedEntries

        public void asyncReadUnconfirmedEntries​(long firstEntry,
                                                long lastEntry,
                                                AsyncCallback.ReadCallback cb,
                                                java.lang.Object ctx)
        Read a sequence of entries asynchronously, allowing to read after the LastAddConfirmed range.
        This is the same of asyncReadEntries(long, long, ReadCallback, Object) but it lets the client read without checking the local value of LastAddConfirmed, so that it is possibile to read entries for which the writer has not received the acknowledge yet.
        For entries which are within the range 0..LastAddConfirmed BookKeeper guarantees that the writer has successfully received the acknowledge.
        For entries outside that range it is possible that the writer never received the acknowledge and so there is the risk that the reader is seeing entries before the writer and this could result in a consistency issue in some cases.
        With this method you can even read entries before the LastAddConfirmed and entries after it with one call, the expected consistency will be as described above for each subrange of ids.
        Parameters:
        firstEntry - id of first entry of sequence
        lastEntry - id of last entry of sequence
        cb - object implementing read callback interface
        ctx - control object
        See Also:
        asyncReadEntries(long, long, ReadCallback, Object), asyncReadLastConfirmed(ReadLastConfirmedCallback, Object), readUnconfirmedEntries(long, long)
      • asyncBatchReadUnconfirmedEntries

        public void asyncBatchReadUnconfirmedEntries​(long startEntry,
                                                     int maxCount,
                                                     long maxSize,
                                                     AsyncCallback.ReadCallback cb,
                                                     java.lang.Object ctx)
        Read a sequence of entries asynchronously, allowing to read after the LastAddConfirmed range. It sends an RPC to get all entries instead of send multi RPC to get all entries.
        Parameters:
        startEntry - id of first entry of sequence
        maxCount - the entries count
        maxSize - the total entries size
        cb - object implementing read callback interface
        ctx - control object
      • readAsync

        public java.util.concurrent.CompletableFuture<LedgerEntries> readAsync​(long firstEntry,
                                                                               long lastEntry)
        Read a sequence of entries asynchronously.
        Specified by:
        readAsync in interface ReadHandle
        Parameters:
        firstEntry - id of first entry of sequence
        lastEntry - id of last entry of sequence
        Returns:
        an handle to the result of the operation
      • batchReadAsync

        public java.util.concurrent.CompletableFuture<LedgerEntries> batchReadAsync​(long startEntry,
                                                                                    int maxCount,
                                                                                    long maxSize)
        Read a sequence of entries in asynchronously. It sends an RPC to get all entries instead of send multi RPC to get all entries.
        Specified by:
        batchReadAsync in interface ReadHandle
        Parameters:
        startEntry - id of first entry of sequence
        maxCount - the entries count
        maxSize - the total entries size
        Returns:
        an handle to the result of the operation
      • readUnconfirmedAsync

        public java.util.concurrent.CompletableFuture<LedgerEntries> readUnconfirmedAsync​(long firstEntry,
                                                                                          long lastEntry)
        Read a sequence of entries asynchronously, allowing to read after the LastAddConfirmed range.
        This is the same of asyncReadEntries(long, long, ReadCallback, Object) but it lets the client read without checking the local value of LastAddConfirmed, so that it is possibile to read entries for which the writer has not received the acknowledge yet.
        For entries which are within the range 0..LastAddConfirmed BookKeeper guarantees that the writer has successfully received the acknowledge.
        For entries outside that range it is possible that the writer never received the acknowledge and so there is the risk that the reader is seeing entries before the writer and this could result in a consistency issue in some cases.
        With this method you can even read entries before the LastAddConfirmed and entries after it with one call, the expected consistency will be as described above for each subrange of ids.
        Specified by:
        readUnconfirmedAsync in interface ReadHandle
        Parameters:
        firstEntry - id of first entry of sequence
        lastEntry - id of last entry of sequence
        Returns:
        an handle to the result of the operation
        See Also:
        asyncReadEntries(long, long, ReadCallback, Object), asyncReadLastConfirmed(ReadLastConfirmedCallback, Object), readUnconfirmedEntries(long, long)
      • asyncReadEntriesInternal

        void asyncReadEntriesInternal​(long firstEntry,
                                      long lastEntry,
                                      AsyncCallback.ReadCallback cb,
                                      java.lang.Object ctx,
                                      boolean isRecoveryRead)
      • asyncBatchReadEntriesInternal

        void asyncBatchReadEntriesInternal​(long startEntry,
                                           int maxCount,
                                           long maxSize,
                                           AsyncCallback.ReadCallback cb,
                                           java.lang.Object ctx,
                                           boolean isRecoveryRead)
      • readEntriesInternalAsync

        java.util.concurrent.CompletableFuture<LedgerEntries> readEntriesInternalAsync​(long firstEntry,
                                                                                       long lastEntry,
                                                                                       boolean isRecoveryRead)
      • addEntry

        public long addEntry​(byte[] data)
                      throws java.lang.InterruptedException,
                             BKException
        Add entry synchronously to an open ledger.
        Parameters:
        data - array of bytes to be written to the ledger do not reuse the buffer, bk-client will release it appropriately
        Returns:
        the entryId of the new inserted entry
        Throws:
        java.lang.InterruptedException
        BKException
      • appendAsync

        public java.util.concurrent.CompletableFuture<java.lang.Long> appendAsync​(io.netty.buffer.ByteBuf data)
        Add entry asynchronously to an open ledger.
        Specified by:
        appendAsync in interface WriteHandle
        Parameters:
        data - a bytebuf to be written. The bytebuf's reference count will be decremented by 1 after the completable future is returned do not reuse the buffer, bk-client will release it appropriately.
        Returns:
        an handle to the result, in case of success it will return the id of the newly appended entry
      • addEntry

        public long addEntry​(long entryId,
                             byte[] data)
                      throws java.lang.InterruptedException,
                             BKException
        Add entry synchronously to an open ledger. This can be used only with LedgerHandleAdv returned through ledgers created with BookKeeper.createLedgerAdv(int, int, int, BookKeeper.DigestType, byte[]).
        Parameters:
        entryId - entryId to be added
        data - array of bytes to be written to the ledger do not reuse the buffer, bk-client will release it appropriately
        Returns:
        the entryId of the new inserted entry
        Throws:
        java.lang.InterruptedException
        BKException
      • addEntry

        public long addEntry​(byte[] data,
                             int offset,
                             int length)
                      throws java.lang.InterruptedException,
                             BKException
        Add entry synchronously to an open ledger.
        Parameters:
        data - array of bytes to be written to the ledger do not reuse the buffer, bk-client will release it appropriately
        offset - offset from which to take bytes from data
        length - number of bytes to take from data
        Returns:
        the entryId of the new inserted entry
        Throws:
        java.lang.InterruptedException
        BKException
      • addEntry

        public long addEntry​(long entryId,
                             byte[] data,
                             int offset,
                             int length)
                      throws java.lang.InterruptedException,
                             BKException
        Add entry synchronously to an open ledger. This can be used only with LedgerHandleAdv returned through ledgers created with BookKeeper.createLedgerAdv(int, int, int, BookKeeper.DigestType, byte[]).
        Parameters:
        entryId - entryId to be added.
        data - array of bytes to be written to the ledger do not reuse the buffer, bk-client will release it appropriately
        offset - offset from which to take bytes from data
        length - number of bytes to take from data
        Returns:
        entryId
        Throws:
        java.lang.InterruptedException
        BKException
      • asyncAddEntry

        public void asyncAddEntry​(byte[] data,
                                  AsyncCallback.AddCallback cb,
                                  java.lang.Object ctx)
        Add entry asynchronously to an open ledger.
        Parameters:
        data - array of bytes to be written do not reuse the buffer, bk-client will release it appropriately
        cb - object implementing callbackinterface
        ctx - some control object
      • asyncAddEntry

        public void asyncAddEntry​(byte[] data,
                                  int offset,
                                  int length,
                                  AsyncCallback.AddCallback cb,
                                  java.lang.Object ctx)
        Add entry asynchronously to an open ledger, using an offset and range.
        Parameters:
        data - array of bytes to be written do not reuse the buffer, bk-client will release it appropriately
        offset - offset from which to take bytes from data
        length - number of bytes to take from data
        cb - object implementing callbackinterface
        ctx - some control object
        Throws:
        java.lang.ArrayIndexOutOfBoundsException - if offset or length is negative or offset and length sum to a value higher than the length of data.
      • asyncAddEntry

        public void asyncAddEntry​(io.netty.buffer.ByteBuf data,
                                  AsyncCallback.AddCallback cb,
                                  java.lang.Object ctx)
      • asyncAddEntry

        public void asyncAddEntry​(long entryId,
                                  byte[] data,
                                  int offset,
                                  int length,
                                  AsyncCallback.AddCallback cb,
                                  java.lang.Object ctx)
        Add entry asynchronously to an open ledger, using an offset and range. This can be used only with LedgerHandleAdv returned through ledgers created with BookKeeper.createLedgerAdv(int, int, int, BookKeeper.DigestType, byte[]).
        Parameters:
        entryId - entryId of the entry to add.
        data - array of bytes to be written do not reuse the buffer, bk-client will release it appropriately
        offset - offset from which to take bytes from data
        length - number of bytes to take from data
        cb - object implementing callbackinterface
        ctx - some control object
        Throws:
        java.lang.ArrayIndexOutOfBoundsException - if offset or length is negative or offset and length sum to a value higher than the length of data.
      • asyncAddEntry

        public void asyncAddEntry​(long entryId,
                                  byte[] data,
                                  int offset,
                                  int length,
                                  AsyncCallback.AddCallbackWithLatency cb,
                                  java.lang.Object ctx)
        Add entry asynchronously to an open ledger, using an offset and range.
        Parameters:
        entryId - entryId of the entry to add
        data - array of bytes to be written do not reuse the buffer, bk-client will release it appropriately
        offset - offset from which to take bytes from data
        length - number of bytes to take from data
        cb - object implementing callbackinterface
        ctx - some control object
        Throws:
        java.lang.ArrayIndexOutOfBoundsException - if offset or length is negative or offset and length sum to a value higher than the length of data.
      • asyncAddEntry

        public void asyncAddEntry​(long entryId,
                                  io.netty.buffer.ByteBuf data,
                                  AsyncCallback.AddCallbackWithLatency cb,
                                  java.lang.Object ctx)
        Add entry asynchronously to an open ledger, using an offset and range. This can be used only with LedgerHandleAdv returned through ledgers created with BookKeeper.createLedgerAdv(int, int, int, BookKeeper.DigestType, byte[]).
        Parameters:
        entryId - entryId of the entry to add.
        data - io.netty.buffer.ByteBuf of bytes to be written do not reuse the buffer, bk-client will release it appropriately
        cb - object implementing callbackinterface
        ctx - some control object
      • force

        public java.util.concurrent.CompletableFuture<java.lang.Void> force()
        Enforce durability to the entries written by this handle.

        This API is useful with WriteFlag.DEFERRED_SYNC, because with that flag writes are acknowledged by the bookie without waiting for a durable write

        Specified by:
        force in interface ForceableHandle
        Returns:
        an handle to the result
      • asyncRecoveryAddEntry

        void asyncRecoveryAddEntry​(byte[] data,
                                   int offset,
                                   int length,
                                   AsyncCallback.AddCallback cb,
                                   java.lang.Object ctx)
        Make a recovery add entry request. Recovery adds can add to a ledger even if it has been fenced.

        This is only valid for bookie and ledger recovery, which may need to replicate entries to a quorum of bookies to ensure data safety.

        Normal client should never call this method.

      • waitForWritable

        protected boolean waitForWritable​(DistributionSchedule.WriteSet writeSet,
                                          int allowedNonWritableCount,
                                          long durationMs)
      • doAsyncAddEntry

        protected void doAsyncAddEntry​(PendingAddOp op)
      • updateLastConfirmed

        void updateLastConfirmed​(long lac,
                                 long len)
      • asyncReadLastConfirmed

        public void asyncReadLastConfirmed​(AsyncCallback.ReadLastConfirmedCallback cb,
                                           java.lang.Object ctx)
        Obtains asynchronously the last confirmed write from a quorum of bookies. This call obtains the last add confirmed each bookie has received for this ledger and returns the maximum. If the ledger has been closed, the value returned by this call may not correspond to the id of the last entry of the ledger, since it reads the hint of bookies. Consequently, in the case the ledger has been closed, it may return a different value than getLastAddConfirmed, which returns the local value of the ledger handle.
        Parameters:
        cb -
        ctx -
        See Also:
        getLastAddConfirmed()
      • tryReadLastAddConfirmedAsync

        public java.util.concurrent.CompletableFuture<java.lang.Long> tryReadLastAddConfirmedAsync()
        Obtains asynchronously the last confirmed write from a quorum of bookies but it doesn't wait all the responses from the quorum. It would callback immediately if it received a LAC which is larger than current LAC.
        Specified by:
        tryReadLastAddConfirmedAsync in interface ReadHandle
        Returns:
        an handle to the result of the operation
      • readLastAddConfirmedAsync

        public java.util.concurrent.CompletableFuture<java.lang.Long> readLastAddConfirmedAsync()
        Obtains asynchronously the last confirmed write from a quorum of bookies. This call obtains the last add confirmed each bookie has received for this ledger and returns the maximum. If the ledger has been closed, the value returned by this call may not correspond to the id of the last entry of the ledger, since it reads the hint of bookies. Consequently, in the case the ledger has been closed, it may return a different value than getLastAddConfirmed, which returns the local value of the ledger handle.
        Specified by:
        readLastAddConfirmedAsync in interface ReadHandle
        Returns:
        an handle to the result of the operation
        See Also:
        ReadHandle.getLastAddConfirmed()
      • readLastAddConfirmedAndEntryAsync

        public java.util.concurrent.CompletableFuture<LastConfirmedAndEntry> readLastAddConfirmedAndEntryAsync​(long entryId,
                                                                                                               long timeOutInMillis,
                                                                                                               boolean parallel)
        Asynchronous read specific entry and the latest last add confirmed. If the next entryId is less than known last add confirmed, the call will read next entry directly. If the next entryId is ahead of known last add confirmed, the call will issue a long poll read to wait for the next entry entryId.
        Specified by:
        readLastAddConfirmedAndEntryAsync in interface ReadHandle
        Parameters:
        entryId - next entry id to read
        timeOutInMillis - timeout period to wait for the entry id to be available (for long poll only) if timeout for get the entry, it will return null entry.
        parallel - whether to issue the long poll reads in parallel
        Returns:
        an handle to the result of the operation
      • asyncReadLastConfirmedAndEntry

        public void asyncReadLastConfirmedAndEntry​(long entryId,
                                                   long timeOutInMillis,
                                                   boolean parallel,
                                                   AsyncCallback.ReadLastConfirmedAndEntryCallback cb,
                                                   java.lang.Object ctx)
        Asynchronous read next entry and the latest last add confirmed. If the next entryId is less than known last add confirmed, the call will read next entry directly. If the next entryId is ahead of known last add confirmed, the call will issue a long poll read to wait for the next entry entryId.

        The callback will return the latest last add confirmed and next entry if it is available within timeout period timeOutInMillis.

        Parameters:
        entryId - next entry id to read
        timeOutInMillis - timeout period to wait for the entry id to be available (for long poll only)
        parallel - whether to issue the long poll reads in parallel
        cb - callback to return the result
        ctx - callback context
      • readLastConfirmed

        public long readLastConfirmed()
                               throws java.lang.InterruptedException,
                                      BKException
        Obtains synchronously the last confirmed write from a quorum of bookies. This call obtains the last add confirmed each bookie has received for this ledger and returns the maximum. If the ledger has been closed, the value returned by this call may not correspond to the id of the last entry of the ledger, since it reads the hint of bookies. Consequently, in the case the ledger has been closed, it may return a different value than getLastAddConfirmed, which returns the local value of the ledger handle.
        Returns:
        The entry id of the last confirmed write or INVALID_ENTRY_ID if no entry has been confirmed
        Throws:
        java.lang.InterruptedException
        BKException
        See Also:
        getLastAddConfirmed()
      • tryReadLastConfirmed

        public long tryReadLastConfirmed()
                                  throws java.lang.InterruptedException,
                                         BKException
        Obtains synchronously the last confirmed write from a quorum of bookies. It is similar as readLastConfirmed(), but it doesn't wait all the responses from the quorum. It would callback immediately if it received a LAC which is larger than current LAC.
        Returns:
        The entry id of the last confirmed write or INVALID_ENTRY_ID if no entry has been confirmed
        Throws:
        java.lang.InterruptedException
        BKException
        See Also:
        readLastConfirmed()
      • asyncReadExplicitLastConfirmed

        public void asyncReadExplicitLastConfirmed​(AsyncCallback.ReadLastConfirmedCallback cb,
                                                   java.lang.Object ctx)
        Obtains asynchronously the explicit last add confirmed from a quorum of bookies. This call obtains Explicit LAC value and piggy-backed LAC value (just like asyncReadLastConfirmed(ReadLastConfirmedCallback, Object)) from each bookie in the ensemble and returns the maximum. If in the write LedgerHandle, explicitLAC feature is not enabled then this call behavior will be similar to asyncReadLastConfirmed(ReadLastConfirmedCallback, Object). If the read explicit lastaddconfirmed is greater than getLastAddConfirmed, then it updates the lastAddConfirmed of this ledgerhandle. If the ledger has been closed, it returns the value of the last add confirmed from the metadata.
        Parameters:
        cb - callback to return read explicit last confirmed
        ctx - callback context
        See Also:
        getLastAddConfirmed()
      • readExplicitLastConfirmed

        public long readExplicitLastConfirmed()
                                       throws java.lang.InterruptedException,
                                              BKException
        Throws:
        java.lang.InterruptedException
        BKException
      • handleUnrecoverableErrorDuringAdd

        void handleUnrecoverableErrorDuringAdd​(int rc)
      • errorOutPendingAdds

        void errorOutPendingAdds​(int rc)
      • drainPendingAddsAndAdjustLength

        java.util.List<PendingAddOp> drainPendingAddsAndAdjustLength()
      • errorOutPendingAdds

        void errorOutPendingAdds​(int rc,
                                 java.util.List<PendingAddOp> ops)
      • sendAddSuccessCallbacks

        void sendAddSuccessCallbacks()
      • hasDelayedWriteFailedBookies

        boolean hasDelayedWriteFailedBookies()
      • notifyWriteFailed

        void notifyWriteFailed​(int index,
                               BookieId addr)
      • maybeHandleDelayedWriteBookieFailure

        void maybeHandleDelayedWriteBookieFailure()
      • handleBookieFailure

        void handleBookieFailure​(java.util.Map<java.lang.Integer,​BookieId> failedBookies)
      • ensembleChangeLoop

        void ensembleChangeLoop​(java.util.List<BookieId> origEnsemble,
                                java.util.Map<java.lang.Integer,​BookieId> failedBookies)
      • unsetSuccessAndSendWriteRequest

        void unsetSuccessAndSendWriteRequest​(java.util.List<BookieId> ensemble,
                                             java.util.Set<java.lang.Integer> bookies)
      • registerOperationFailureOnBookie

        void registerOperationFailureOnBookie​(BookieId bookie,
                                              long entryId)
      • getCurrentEnsemble

        java.util.List<BookieId> getCurrentEnsemble()
        Get the current ensemble from the ensemble list. The current ensemble is the last ensemble in the list. The ledger handle uses this ensemble when triggering operations which work on the end of the ledger, such as adding new entries or reading the last add confirmed.

        This method is also used by ReadOnlyLedgerHandle during recovery, and when tailing a ledger.

        Generally, this method should only be called by LedgerHandle and not by the operations themselves, to avoid adding more dependencies between the classes. There are too many already.

      • executeOrdered

        void executeOrdered​(java.lang.Runnable runnable)
                     throws java.util.concurrent.RejectedExecutionException
        Execute the callback in the thread pinned to the ledger.
        Parameters:
        runnable -
        Throws:
        java.util.concurrent.RejectedExecutionException
      • getPendingAddOps

        public java.util.Queue<PendingAddOp> getPendingAddOps()