Class Engine

  • All Implemented Interfaces:
    java.io.Closeable, java.lang.AutoCloseable
    Direct Known Subclasses:
    InternalEngine, ReadOnlyEngine

    public abstract class Engine
    extends java.lang.Object
    implements java.io.Closeable
    • Field Detail

      • HISTORY_UUID_KEY

        public static final java.lang.String HISTORY_UUID_KEY
        See Also:
        Constant Field Values
      • MIN_RETAINED_SEQNO

        public static final java.lang.String MIN_RETAINED_SEQNO
        See Also:
        Constant Field Values
      • MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID

        public static final java.lang.String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID
        See Also:
        Constant Field Values
      • CAN_MATCH_SEARCH_SOURCE

        public static final java.lang.String CAN_MATCH_SEARCH_SOURCE
        See Also:
        Constant Field Values
      • shardId

        protected final ShardId shardId
      • allocationId

        protected final java.lang.String allocationId
      • logger

        protected final org.apache.logging.log4j.Logger logger
      • store

        protected final Store store
      • isClosed

        protected final java.util.concurrent.atomic.AtomicBoolean isClosed
      • failEngineLock

        protected final java.util.concurrent.locks.ReentrantLock failEngineLock
      • rwl

        protected final java.util.concurrent.locks.ReentrantReadWriteLock rwl
      • failedEngine

        protected final org.apache.lucene.util.SetOnce<java.lang.Exception> failedEngine
      • lastWriteNanos

        protected volatile long lastWriteNanos
    • Constructor Detail

    • Method Detail

      • guardedRamBytesUsed

        protected static long guardedRamBytesUsed​(org.apache.lucene.util.Accountable a)
        Returns 0 in the case where accountable is null, otherwise returns ramBytesUsed()
      • isMergedSegment

        protected static boolean isMergedSegment​(org.apache.lucene.index.LeafReader reader)
        Returns whether a leaf reader comes from a merge (versus flush or addIndexes).
      • getLastCommittedSegmentInfos

        protected abstract org.apache.lucene.index.SegmentInfos getLastCommittedSegmentInfos()
      • getMergeStats

        public MergeStats getMergeStats()
      • getHistoryUUID

        public abstract java.lang.String getHistoryUUID()
        returns the history uuid for the engine
      • getWritingBytes

        public abstract long getWritingBytes()
        Returns how many bytes we are currently moving from heap to disk
      • completionStats

        public CompletionStats completionStats​(java.lang.String... fieldNamePatterns)
                                        throws java.io.IOException
        Returns the CompletionStats for this engine
        Throws:
        java.io.IOException
      • docsStats

        protected final DocsStats docsStats​(org.apache.lucene.index.IndexReader indexReader)
      • verifyEngineBeforeIndexClosing

        public void verifyEngineBeforeIndexClosing()
                                            throws java.lang.IllegalStateException
        Performs the pre-closing checks on the Engine.
        Throws:
        java.lang.IllegalStateException - if the sanity checks failed
      • getIndexThrottleTimeInMillis

        public abstract long getIndexThrottleTimeInMillis()
        Returns the number of milliseconds this engine was under index throttling.
      • isThrottled

        public abstract boolean isThrottled()
        Returns the true iff this engine is currently under index throttling.
        See Also:
        getIndexThrottleTimeInMillis()
      • index

        public abstract Engine.IndexResult index​(Engine.Index index)
                                          throws java.io.IOException
        Perform document index operation on the engine
        Parameters:
        index - operation to perform
        Returns:
        Engine.IndexResult containing updated translog location, version and document specific failures Note: engine level failures (i.e. persistent engine failures) are thrown
        Throws:
        java.io.IOException
      • delete

        public abstract Engine.DeleteResult delete​(Engine.Delete delete)
                                            throws java.io.IOException
        Perform document delete operation on the engine
        Parameters:
        delete - operation to perform
        Returns:
        Engine.DeleteResult containing updated translog location, version and document specific failures Note: engine level failures (i.e. persistent engine failures) are thrown
        Throws:
        java.io.IOException
      • syncFlush

        public abstract Engine.SyncedFlushResult syncFlush​(java.lang.String syncId,
                                                           Engine.CommitId expectedCommitId)
                                                    throws EngineException
        Attempts to do a special commit where the given syncID is put into the commit data. The attempt succeeds if there are not pending writes in lucene and the current point is equal to the expected one.
        Parameters:
        syncId - id of this sync
        expectedCommitId - the expected value of
        Returns:
        true if the sync commit was made, false o.w.
        Throws:
        EngineException
      • acquireSearcher

        public final Engine.Searcher acquireSearcher​(java.lang.String source)
                                              throws EngineException
        Returns a new searcher instance. The consumer of this API is responsible for releasing the returned searcher in a safe manner, preferably in a try/finally block.
        Parameters:
        source - the source API or routing that triggers this searcher acquire
        Throws:
        EngineException
        See Also:
        Engine.Searcher.close()
      • acquireSearcher

        public Engine.Searcher acquireSearcher​(java.lang.String source,
                                               Engine.SearcherScope scope)
                                        throws EngineException
        Returns a new searcher instance. The consumer of this API is responsible for releasing the returned searcher in a safe manner, preferably in a try/finally block.
        Parameters:
        source - the source API or routing that triggers this searcher acquire
        scope - the scope of this searcher ie. if the searcher will be used for get or search purposes
        Throws:
        EngineException
        See Also:
        Engine.Searcher.close()
      • getReferenceManager

        protected abstract org.apache.lucene.search.ReferenceManager<org.apache.lucene.search.IndexSearcher> getReferenceManager​(Engine.SearcherScope scope)
      • isTranslogSyncNeeded

        public abstract boolean isTranslogSyncNeeded()
        Checks if the underlying storage sync is required.
      • ensureTranslogSynced

        public abstract boolean ensureTranslogSynced​(java.util.stream.Stream<Translog.Location> locations)
                                              throws java.io.IOException
        Ensures that all locations in the given stream have been written to the underlying storage.
        Throws:
        java.io.IOException
      • syncTranslog

        public abstract void syncTranslog()
                                   throws java.io.IOException
        Throws:
        java.io.IOException
      • acquireRetentionLock

        public abstract java.io.Closeable acquireRetentionLock()
        Acquires a lock on the translog files and Lucene soft-deleted documents to prevent them from being trimmed
      • newChangesSnapshot

        public abstract Translog.Snapshot newChangesSnapshot​(java.lang.String source,
                                                             MapperService mapperService,
                                                             long fromSeqNo,
                                                             long toSeqNo,
                                                             boolean requiredFullRange)
                                                      throws java.io.IOException
        Creates a new history snapshot from Lucene for reading operations whose seqno in the requesting seqno range (both inclusive). This feature requires soft-deletes enabled. If soft-deletes are disabled, this method will throw an IllegalStateException.
        Throws:
        java.io.IOException
      • readHistoryOperations

        public abstract Translog.Snapshot readHistoryOperations​(java.lang.String source,
                                                                MapperService mapperService,
                                                                long startingSeqNo)
                                                         throws java.io.IOException
        Creates a new history snapshot for reading operations since startingSeqNo (inclusive). The returned snapshot can be retrieved from either Lucene index or translog files.
        Throws:
        java.io.IOException
      • estimateNumberOfHistoryOperations

        public abstract int estimateNumberOfHistoryOperations​(java.lang.String source,
                                                              MapperService mapperService,
                                                              long startingSeqNo)
                                                       throws java.io.IOException
        Returns the estimated number of history operations whose seq# at least startingSeqNo(inclusive) in this engine.
        Throws:
        java.io.IOException
      • hasCompleteOperationHistory

        public abstract boolean hasCompleteOperationHistory​(java.lang.String source,
                                                            MapperService mapperService,
                                                            long startingSeqNo)
                                                     throws java.io.IOException
        Checks if this engine has every operations since startingSeqNo(inclusive) in its history (either Lucene or translog)
        Throws:
        java.io.IOException
      • getMinRetainedSeqNo

        public abstract long getMinRetainedSeqNo()
        Gets the minimum retained sequence number for this engine.
        Returns:
        the minimum retained sequence number
      • getTranslogStats

        public abstract TranslogStats getTranslogStats()
      • getTranslogLastWriteLocation

        public abstract Translog.Location getTranslogLastWriteLocation()
        Returns the last location that the translog of this engine has written into.
      • ensureOpen

        protected final void ensureOpen​(java.lang.Exception suppressed)
      • ensureOpen

        protected final void ensureOpen()
      • commitStats

        public final CommitStats commitStats()
        get commits stats for the last commit
      • getLocalCheckpoint

        public abstract long getLocalCheckpoint()
        Returns:
        the local checkpoint for this Engine
      • getSeqNoStats

        public abstract SeqNoStats getSeqNoStats​(long globalCheckpoint)
        Returns:
        a SeqNoStats object, using local state and the supplied global checkpoint
      • getLastSyncedGlobalCheckpoint

        public abstract long getLastSyncedGlobalCheckpoint()
        Returns the latest global checkpoint value that has been persisted in the underlying storage (i.e. translog's checkpoint)
      • segmentsStats

        public SegmentsStats segmentsStats​(boolean includeSegmentFileSizes)
        Global stats on segments.
      • writerSegmentStats

        protected void writerSegmentStats​(SegmentsStats stats)
      • getIndexBufferRAMBytesUsed

        public abstract long getIndexBufferRAMBytesUsed()
        How much heap is used that would be freed by a refresh. Note that this may throw AlreadyClosedException.
      • segments

        public abstract java.util.List<Segment> segments​(boolean verbose)
        The list of segments in the engine.
      • refreshNeeded

        public boolean refreshNeeded()
      • refresh

        @Nullable
        public abstract void refresh​(java.lang.String source)
                              throws EngineException
        Synchronously refreshes the engine for new search operations to reflect the latest changes.
        Throws:
        EngineException
      • writeIndexingBuffer

        public abstract void writeIndexingBuffer()
                                          throws EngineException
        Called when our engine is using too much heap and should move buffered indexed/deleted documents to disk.
        Throws:
        EngineException
      • shouldPeriodicallyFlush

        public abstract boolean shouldPeriodicallyFlush()
        Checks if this engine should be flushed periodically. This check is mainly based on the uncommitted translog size and the translog flush threshold setting.
      • flush

        public abstract Engine.CommitId flush​(boolean force,
                                              boolean waitIfOngoing)
                                       throws EngineException
        Flushes the state of the engine including the transaction log, clearing memory.
        Parameters:
        force - if true a lucene commit is executed even if no changes need to be committed.
        waitIfOngoing - if true this call will block until all currently running flushes have finished. Otherwise this call will return without blocking.
        Returns:
        the commit Id for the resulting commit
        Throws:
        EngineException
      • flush

        public final Engine.CommitId flush()
                                    throws EngineException
        Flushes the state of the engine including the transaction log, clearing memory and persisting documents in the lucene index to disk including a potentially heavy and durable fsync operation. This operation is not going to block if another flush operation is currently running and won't write a lucene commit if nothing needs to be committed.
        Returns:
        the commit Id for the resulting commit
        Throws:
        EngineException
      • shouldRollTranslogGeneration

        public abstract boolean shouldRollTranslogGeneration()
        Tests whether or not the translog generation should be rolled to a new generation. This test is based on the size of the current generation compared to the configured generation threshold size.
        Returns:
        true if the current generation should be rolled to a new generation
      • rollTranslogGeneration

        public abstract void rollTranslogGeneration()
                                             throws EngineException
        Rolls the translog generation and cleans unneeded.
        Throws:
        EngineException
      • forceMerge

        public void forceMerge​(boolean flush)
                        throws java.io.IOException
        Force merges to 1 segment
        Throws:
        java.io.IOException
      • forceMerge

        public abstract void forceMerge​(boolean flush,
                                        int maxNumSegments,
                                        boolean onlyExpungeDeletes,
                                        boolean upgrade,
                                        boolean upgradeOnlyAncientSegments)
                                 throws EngineException,
                                        java.io.IOException
        Triggers a forced merge on this engine
        Throws:
        EngineException
        java.io.IOException
      • acquireLastIndexCommit

        public abstract Engine.IndexCommitRef acquireLastIndexCommit​(boolean flushFirst)
                                                              throws EngineException
        Snapshots the most recent index and returns a handle to it. If needed will try and "commit" the lucene index to make sure we have a "fresh" copy of the files to snapshot.
        Parameters:
        flushFirst - indicates whether the engine should flush before returning the snapshot
        Throws:
        EngineException
      • failEngine

        public void failEngine​(java.lang.String reason,
                               @Nullable
                               java.lang.Exception failure)
        fail engine due to some error. the engine will also be closed. The underlying store is marked corrupted iff failure is caused by index corruption
      • maybeFailEngine

        protected boolean maybeFailEngine​(java.lang.String source,
                                          java.lang.Exception e)
        Check whether the engine should be failed
      • closeNoLock

        protected abstract void closeNoLock​(java.lang.String reason,
                                            java.util.concurrent.CountDownLatch closedLatch)
        Method to close the engine while the write lock is held. Must decrement the supplied when closing work is done and resources are freed.
      • flushAndClose

        public void flushAndClose()
                           throws java.io.IOException
        Flush the engine (committing segments to disk and truncating the translog) and close it.
        Throws:
        java.io.IOException
      • close

        public void close()
                   throws java.io.IOException
        Specified by:
        close in interface java.lang.AutoCloseable
        Specified by:
        close in interface java.io.Closeable
        Throws:
        java.io.IOException
      • onSettingsChanged

        public void onSettingsChanged()
      • getLastWriteNanos

        public long getLastWriteNanos()
        Returns the timestamp of the last write in nanoseconds. Note: this time might not be absolutely accurate since the Engine.Operation.startTime() is used which might be slightly inaccurate.
        See Also:
        System.nanoTime(), Engine.Operation.startTime()
      • activateThrottling

        public abstract void activateThrottling()
        Request that this engine throttle incoming indexing requests to one thread. Must be matched by a later call to deactivateThrottling().
      • deactivateThrottling

        public abstract void deactivateThrottling()
        Reverses a previous activateThrottling() call.
      • restoreLocalHistoryFromTranslog

        public abstract int restoreLocalHistoryFromTranslog​(Engine.TranslogRecoveryRunner translogRecoveryRunner)
                                                     throws java.io.IOException
        This method replays translog to restore the Lucene index which might be reverted previously. This ensures that all acknowledged writes are restored correctly when this engine is promoted.
        Returns:
        the number of translog operations have been recovered
        Throws:
        java.io.IOException
      • fillSeqNoGaps

        public abstract int fillSeqNoGaps​(long primaryTerm)
                                   throws java.io.IOException
        Fills up the local checkpoints history with no-ops until the local checkpoint and the max seen sequence ID are identical.
        Parameters:
        primaryTerm - the shards primary term this engine was created for
        Returns:
        the number of no-ops added
        Throws:
        java.io.IOException
      • recoverFromTranslog

        public abstract Engine recoverFromTranslog​(Engine.TranslogRecoveryRunner translogRecoveryRunner,
                                                   long recoverUpToSeqNo)
                                            throws java.io.IOException
        Performs recovery from the transaction log up to recoverUpToSeqNo (inclusive). This operation will close the engine if the recovery fails.
        Parameters:
        translogRecoveryRunner - the translog recovery runner
        recoverUpToSeqNo - the upper bound, inclusive, of sequence number to be recovered
        Throws:
        java.io.IOException
      • skipTranslogRecovery

        public abstract void skipTranslogRecovery()
        Do not replay translog operations, but make the engine be ready.
      • isRecovering

        public boolean isRecovering()
        Returns true iff this engine is currently recovering from translog.
      • maybePruneDeletes

        public abstract void maybePruneDeletes()
        Tries to prune buffered deletes from the version map.
      • getMaxSeenAutoIdTimestamp

        public long getMaxSeenAutoIdTimestamp()
        Returns the maximum auto_id_timestamp of all append-only index requests have been processed by this engine or the auto_id_timestamp received from its primary shard via updateMaxUnsafeAutoIdTimestamp(long). Notes this method returns the auto_id_timestamp of all append-only requests, not max_unsafe_auto_id_timestamp.
      • updateMaxUnsafeAutoIdTimestamp

        public abstract void updateMaxUnsafeAutoIdTimestamp​(long newTimestamp)
        Forces this engine to advance its max_unsafe_auto_id_timestamp marker to at least the given timestamp. The engine will disable optimization for all append-only whose timestamp at most newTimestamp.
      • getMaxSeqNoOfUpdatesOrDeletes

        public final long getMaxSeqNoOfUpdatesOrDeletes()
        Returns the maximum sequence number of either update or delete operations have been processed in this engine or the sequence number from advanceMaxSeqNoOfUpdatesOrDeletes(long). An index request is considered as an update operation if it overwrites the existing documents in Lucene index with the same document id.

        A note on the optimization using max_seq_no_of_updates_or_deletes: For each operation O, the key invariants are:

        1. I1: There is no operation on docID(O) with seqno that is > MSU(O) and < seqno(O)
        2. I2: If MSU(O) < seqno(O) then docID(O) did not exist when O was applied; more precisely, if there is any O' with seqno(O') < seqno(O) and docID(O') = docID(O) then the one with the greatest seqno is a delete.

        When a receiving shard (either a replica or a follower) receives an operation O, it must first ensure its own MSU at least MSU(O), and then compares its MSU to its local checkpoint (LCP). If LCP < MSU then there's a gap: there may be some operations that act on docID(O) about which we do not yet know, so we cannot perform an add. Note this also covers the case where a future operation O' with seqNo(O') > seqNo(O) and docId(O') = docID(O) is processed before O. In that case MSU(O') is at least seqno(O') and this means MSU >= seqNo(O') > seqNo(O) > LCP (because O wasn't processed yet).

        However, if MSU <= LCP then there is no gap: we have processed every operation <= LCP, and no operation O' with seqno(O') > LCP and seqno(O') < seqno(O) also has docID(O') = docID(O), because such an operation would have seqno(O') > LCP >= MSU >= MSU(O) which contradicts the first invariant. Furthermore in this case we immediately know that docID(O) has been deleted (or never existed) without needing to check Lucene for the following reason. If there's no earlier operation on docID(O) then this is clear, so suppose instead that the preceding operation on docID(O) is O': 1. The first invariant above tells us that seqno(O') <= MSU(O) <= LCP so we have already applied O' to Lucene. 2. Also MSU(O) <= MSU <= LCP < seqno(O) (we discard O if seqno(O) <= LCP) so the second invariant applies, meaning that the O' was a delete.

        Therefore, if MSU <= LCP < seqno(O) we know that O can safely be optimized with and added to lucene with addDocument. Moreover, operations that are optimized using the MSU optimization must not be processed twice as this will create duplicates in Lucene. To avoid this we check the local checkpoint tracker to see if an operation was already processed.

        See Also:
        reinitializeMaxSeqNoOfUpdatesOrDeletes(), advanceMaxSeqNoOfUpdatesOrDeletes(long)
      • reinitializeMaxSeqNoOfUpdatesOrDeletes

        public abstract void reinitializeMaxSeqNoOfUpdatesOrDeletes()
        A primary shard calls this method to re-initialize the max_seq_no_of_updates marker using the max_seq_no from Lucene index and translog before replaying the local translog in its local recovery.
      • advanceMaxSeqNoOfUpdatesOrDeletes

        public final void advanceMaxSeqNoOfUpdatesOrDeletes​(long seqNo)
        A replica shard receives a new max_seq_no_of_updates from its primary shard, then calls this method to advance this marker to at least the given sequence number.