Interface Index

  • All Known Implementing Classes:
    CassandraIndex, ClusteringColumnIndex, CollectionEntryIndex, CollectionKeyIndex, CollectionKeyIndexBase, CollectionValueIndex, KeysIndex, PartitionKeyIndex, PaxosUncommittedIndex, RegularColumnIndex, SASIIndex, StorageAttachedIndex

    public interface Index
    Consisting of a top level Index interface and two sub-interfaces which handle read and write operations, Searcher and Indexer respectively, this defines a secondary index implementation. Instantiation is done via reflection and implementations must provide a constructor which takes the base table's ColumnFamilyStore and the IndexMetadata which defines the Index as arguments. e.g: MyCustomIndex( ColumnFamilyStore baseCfs, IndexMetadata indexDef ) The main interface defines methods for index management, index selection at both write and query time, as well as validation of values that will ultimately be indexed. Two sub-interfaces are also defined, which represent single use helpers for short lived tasks at read and write time. Indexer: an event listener which receives notifications at particular points during an update of a single partition in the base table. Searcher: performs queries against the index based on a predicate defined in a RowFilter. An instance is expected to be single use, being involved in the execution of a single ReadCommand. The main interface includes factory methods for obtaining instances of both of the sub-interfaces; The methods defined in the top level interface can be grouped into 3 categories: Management Tasks: This group of methods is primarily concerned with maintenance of secondary indexes are are mainly called from SecondaryIndexManager. It includes methods for registering and un-registering an index, performing maintenance tasks such as (re)building an index from SSTable data, flushing, invalidating and so forth, as well as some to retrieve general metadata about the index (index name, any internal tables used for persistence etc). Several of these maintenance functions have a return type of Callable<?>; the expectation for these methods is that any work required to be performed by the method be done inside the Callable so that the responsibility for scheduling its execution can rest with SecondaryIndexManager. For instance, a task like reloading index metadata following potential updates caused by modifications to the base table may be performed in a blocking way. In contrast, adding a new index may require it to be built from existing SSTable data, a potentially expensive task which should be performed asynchronously. Index Selection: There are two facets to index selection, write time and read time selection. The former is concerned with identifying whether an index should be informed about a particular write operation. The latter is about providing means to use the index for search during query execution. Validation: Values that may be written to an index are checked as part of input validation, prior to an update or insert operation being accepted. Sub-interfaces: Update processing: Indexes are subscribed to the stream of events generated by modifications to the base table. Subscription is done via first registering the Index with the base table's SecondaryIndexManager. For each partition update, the set of registered indexes are then filtered based on the properties of the update using the selection methods on the main interface described above. Each of the indexes in the filtered set then provides an event listener to receive notifications about the update as it is processed. As such then, a event handler instance is scoped to a single partition update; SecondaryIndexManager obtains a new handler for every update it processes (via a call to the factory method, indexerFor. That handler will then receive all events for the update, before being discarded by the SecondaryIndexManager. Indexer instances are never re-used by SecondaryIndexManager and the expectation is that each call to indexerFor should return a unique instance, or at least if instances can be recycled, that a given instance is only used to process a single partition update at a time. Search: Each query (i.e. a single ReadCommand) that uses indexes will use a single instance of Index.Searcher. As with processing of updates, an Index must be registered with the primary table's SecondaryIndexManager to be able to support queries. During the processing of a ReadCommand, the Expressions in its RowFilter are examined to determine whether any of them are supported by a registered Index. supportsExpression is used to filter out Indexes which cannot support a given Expression. After filtering, the set of candidate indexes are ranked according to the result of getEstimatedResultRows and the most selective (i.e. the one expected to return the smallest number of results) is chosen. A Searcher instance is then obtained from the searcherFor method and used to perform the actual Index lookup. Finally, Indexes can define a post processing step to be performed on the coordinator, after results (partitions from the primary table) have been received from replicas and reconciled. This post processing is defined as a java.util.functions.BiFunction<PartitionIterator, RowFilter, PartitionIterator>, that is a function which takes as arguments a PartitionIterator (containing the reconciled result rows) and a RowFilter (from the ReadCommand being executed) and returns another iterator of partitions, possibly having transformed the initial results in some way. The post processing function is obtained from the Index's postProcessorFor method; the built-in indexes which ship with Cassandra return a no-op function here. An optional static method may be provided to validate custom index options (two variants are supported):
    public static Map<String, String> validateOptions(Map<String, String> options);
    The input is the map of index options supplied in the WITH clause of a CREATE INDEX statement.
    public static Map<String, String> validateOptions(Map<String, String> options, TableMetadata metadata);
    In this version, the base table's metadata is also supplied as an argument. If both overloaded methods are provided, only the one including the base table's metadata will be invoked. The validation method should return a map containing any of the supplied options which are not valid for the implementation. If the returned map is not empty, validation is considered failed and an error is raised. Alternatively, the implementation may choose to throw an org.apache.cassandra.exceptions.ConfigurationException if invalid options are encountered.
    • Field Detail

      • INDEX_BUILDER_SUPPORT

        static final Index.CollatedViewIndexBuildingSupport INDEX_BUILDER_SUPPORT
        Singleton instance of CollatedViewIndexBuildingSupport, which may be used by any Index implementation.
    • Method Detail

      • getBuildTaskSupport

        default Index.IndexBuildingSupport getBuildTaskSupport()
        Get an instance of a helper to provide tasks for building the index from a set of SSTable data. When processing a number of indexes to be rebuilt, SecondaryIndexManager.buildIndexesBlocking groups those with the same IndexBuildingSupport instance, allowing multiple indexes to be built with a single pass through the data. The singleton instance returned from the default method implementation builds indexes using a ReducingKeyIterator to provide a collated view of the SSTable data.
        Returns:
        an instance of the index build task helper. Index implementations which return the same instance will be built using a single task.
      • getRecoveryTaskSupport

        default Index.IndexBuildingSupport getRecoveryTaskSupport()
        Same as getBuildTaskSupport but can be overloaded with a specific 'recover' logic different than the index building one
      • getSupportedLoadTypeOnFailure

        default Index.LoadType getSupportedLoadTypeOnFailure​(boolean isInitialBuild)
        Returns the type of operations supported by the index in case its building has failed and it's needing recovery.
        Parameters:
        isInitialBuild - true if the failure is for the initial build task on index creation, false if the failure is for a full rebuild or recovery.
      • getInitializationTask

        java.util.concurrent.Callable<?> getInitializationTask()
        Return a task to perform any initialization work when a new index instance is created. This may involve costly operations such as (re)building the index, and is performed asynchronously by SecondaryIndexManager
        Returns:
        a task to perform any necessary initialization work
      • getIndexMetadata

        IndexMetadata getIndexMetadata()
        Returns the IndexMetadata which configures and defines the index instance. This should be the same object passed as the argument to setIndexMetadata.
        Returns:
        the index's metadata
      • getMetadataReloadTask

        java.util.concurrent.Callable<?> getMetadataReloadTask​(IndexMetadata indexMetadata)
        Return a task to reload the internal metadata of an index. Called when the base table metadata is modified or when the configuration of the Index is updated Implementations should return a task which performs any necessary work to be done due to updating the configuration(s) such as (re)building etc. This task is performed asynchronously by SecondaryIndexManager
        Returns:
        task to be executed by the index manager during a reload
      • register

        void register​(IndexRegistry registry)
        An index must be registered in order to be able to either subscribe to update events on the base table and/or to provide Searcher functionality for reads. The double dispatch involved here, where the Index actually performs its own registration by calling back to the supplied IndexRegistry's own registerIndex method, is to make the decision as to whether or not to register an index belong to the implementation, not the manager.
        Parameters:
        registry - the index registry to register the instance with
      • unregister

        default void unregister​(IndexRegistry registry)
        Unregister current index when it's removed from system
        Parameters:
        registry - the index registry to register the instance with
      • getBackingTable

        java.util.Optional<ColumnFamilyStore> getBackingTable()
        If the index implementation uses a local table to store its index data, this method should return a handle to it. If not, an empty Optional should be returned. This exists to support legacy implementations, and should always be empty for indexes not belonging to a SingletonIndexGroup.
        Returns:
        an Optional referencing the Index's backing storage table if it has one, or Optional.empty() if not.
      • getBlockingFlushTask

        default java.util.concurrent.Callable<?> getBlockingFlushTask​(Memtable baseCfs)
        Return a task which performs a blocking flush of the index's data corresponding to the provided base table's Memtable. This may extract any necessary data from the base table's Memtable as part of the flush. This version of the method is invoked whenever we flush the base table. If the index stores no in-memory data of its own, it is safe to only implement this method.
        Returns:
        task to be executed by the index manager to perform the flush.
      • getBlockingFlushTask

        java.util.concurrent.Callable<?> getBlockingFlushTask()
        Return a task which performs a blocking flush of any in-memory index data to persistent storage, independent of any flush of the base table. Note that this method is only invoked outside of normal flushes: if there is no in-memory storage for this index, and it only extracts data on flush from the base table's Memtable, then it is safe to perform no work.
        Returns:
        task to be executed by the index manager to perform the flush.
      • getInvalidateTask

        java.util.concurrent.Callable<?> getInvalidateTask()
        Return a task which invalidates the index, indicating it should no longer be considered usable. This should include an clean up and releasing of resources required when dropping an index.
        Returns:
        task to be executed by the index manager to invalidate the index.
      • getTruncateTask

        java.util.concurrent.Callable<?> getTruncateTask​(long truncatedAt)
        Return a task to truncate the index with the specified truncation timestamp. Called when the base table is truncated.
        Parameters:
        truncatedAt - timestamp of the truncation operation. This will be the same timestamp used in the truncation of the base table.
        Returns:
        task to be executed by the index manager when the base table is truncated.
      • getPreJoinTask

        default java.util.concurrent.Callable<?> getPreJoinTask​(boolean hadBootstrap)
        Return a task to be executed before the node enters NORMAL state and finally joins the ring.
        Parameters:
        hadBootstrap - If the node had bootstrap before joining.
        Returns:
        task to be executed by the index manager before joining the ring.
      • shouldBuildBlocking

        boolean shouldBuildBlocking()
        Return true if this index can be built or rebuilt when the index manager determines it is necessary. Returning false enables the index implementation (or some other component) to control if and when SSTable data is incorporated into the index.

        This is called by SecondaryIndexManager in buildIndexBlocking, buildAllIndexesBlocking and rebuildIndexesBlocking where a return value of false causes the index to be exluded from the set of those which will process the SSTable data.

        Returns:
        if the index should be included in the set which processes SSTable data, false otherwise.
      • isSSTableAttached

        default boolean isSSTableAttached()
        For an index to qualify as SSTable-attached, it must do two things:

        1.) It must use SSTableFlushObserver to incrementally build indexes as SSTables are written. This ensures that non-entire file streaming builds them correctly before the streaming transaction finishes.

        2.) Its implementation of SecondaryIndexBuilder must support incremental building by SSTable.

        Returns:
        true if the index builds SSTable-attached on-disk components
      • getFlushObserver

        default SSTableFlushObserver getFlushObserver​(Descriptor descriptor,
                                                      LifecycleNewTracker tracker)
        Get flush observer to observe partition/cell events generated by flushing SSTable (memtable flush or compaction).
        Parameters:
        descriptor - The descriptor of the sstable observer is requested for.
        tracker - The LifecycleNewTracker associated with the SSTable being written
        Returns:
        SSTable flush observer.
      • dependsOn

        boolean dependsOn​(ColumnMetadata column)
        Called to determine whether this index targets a specific column. Used during schema operations such as when dropping or renaming a column, to check if the index will be affected by the change. Typically, if an index answers that it does depend upon a column, then schema operations on that column are not permitted until the index is dropped or altered.
        Parameters:
        column - the column definition to check
        Returns:
        true if the index depends on the supplied column being present; false if the column may be safely dropped or modified without adversely affecting the index
      • supportsExpression

        boolean supportsExpression​(ColumnMetadata column,
                                   Operator operator)
        Called to determine whether this index can provide a searcher to execute a query on the supplied column using the specified operator. This forms part of the query validation done before a CQL select statement is executed.
        Parameters:
        column - the target column of a search query predicate
        operator - the operator of a search query predicate
        Returns:
        true if this index is capable of supporting such expressions, false otherwise
      • filtersMultipleContains

        default boolean filtersMultipleContains()
        Returns whether this index does any kind of filtering when the query has multiple contains expressions, assuming that each of those expressions are supported as defined by supportsExpression(ColumnMetadata, Operator).
        Returns:
        true if this index uses filtering on multiple contains expressions, false otherwise
      • customExpressionValueType

        AbstractType<?> customExpressionValueType()
        If the index supports custom search expressions using the SELECT * FROM table WHERE expr(index_name, expression) syntax, this method should return the expected type of the expression argument. For example, if the index supports custom expressions as Strings, calls to this method should return UTF8Type.instance. If the index implementation does not support custom expressions, then it should return null.
        Returns:
        an the type of custom index expressions supported by this index, or an null if custom expressions are not supported.
      • getPostIndexQueryFilter

        RowFilter getPostIndexQueryFilter​(RowFilter filter)
        Transform an initial RowFilter into the filter that will still need to applied to a set of Rows after the index has performed it's initial scan. Used in ReadCommand#executeLocal to reduce the amount of filtering performed on the results of the index query.
        Parameters:
        filter - the intial filter belonging to a ReadCommand
        Returns:
        the (hopefully) reduced filter that would still need to be applied after the index was used to narrow the initial result set
      • getPostQueryOrdering

        default java.util.Comparator<java.nio.ByteBuffer> getPostQueryOrdering​(Restriction restriction,
                                                                               QueryOptions options)
        Return a comparator that reorders query result before sending to client
        Parameters:
        restriction - restriction that requires current index
        options - query options
        Returns:
        a comparator for post-query ordering; or null if not supported
      • getEstimatedResultRows

        long getEstimatedResultRows()
        Return an estimate of the number of results this index is expected to return for any given query that it can be used to answer. Used in conjunction with indexes() and supportsExpression() to determine the most selective index for a given ReadCommand. Additionally, this is also used by StorageProxy.estimateResultsPerRange to calculate the initial concurrency factor for range requests
        Returns:
        the estimated average number of results a Searcher may return for any given query
      • isQueryable

        default boolean isQueryable​(Index.Status status)
        Check if current index is queryable based on the index status.
        Parameters:
        status - current status of the index
        Returns:
        true if index should be queryable, false if index should be non-queryable
      • validate

        void validate​(PartitionUpdate update,
                      ClientState state)
               throws InvalidRequestException
        Called at write time to ensure that values present in the update are valid according to the rules of all registered indexes which will process it. The partition key as well as the clustering and cell values for each row in the update may be checked by index implementations
        Parameters:
        update - PartitionUpdate containing the values to be validated by registered Index implementations
        state - state related to the client connection
        Throws:
        InvalidRequestException
      • getComponents

        default java.util.Set<Component> getComponents()
        Returns the SSTable-attached Components created by this index.
        Returns:
        the SSTable components created by this index
      • indexerFor

        Index.Indexer indexerFor​(DecoratedKey key,
                                 RegularAndStaticColumns columns,
                                 long nowInSec,
                                 WriteContext ctx,
                                 IndexTransaction.Type transactionType,
                                 Memtable memtable)
        Creates an new Indexer object for updates to a given partition.
        Parameters:
        key - key of the partition being modified
        columns - the regular and static columns the created indexer will have to deal with. This can be empty as an update might only contain partition, range and row deletions, but the indexer is guaranteed to not get any cells for a column that is not part of columns.
        nowInSec - current time of the update operation
        ctx - WriteContext spanning the update operation
        transactionType - indicates what kind of update is being performed on the base data i.e. a write time insert/update/delete or the result of compaction
        memtable - current memtable that the write goes into. It's to make sure memtable and index memtable are in sync.
        Returns:
        the newly created indexer or null if the index is not interested by the update (this could be because the index doesn't care about that particular partition, doesn't care about that type of transaction, ...).
      • validate

        default void validate​(ReadCommand command)
                       throws InvalidRequestException
        Used to validate the various parameters of a supplied ReadCommand, this is called prior to execution. In theory, any command instance may be checked by any Index instance, but in practice the index will be the one returned by a call to the getIndex(ColumnFamilyStore cfs) method on the supplied command. Custom index implementations should perform any validation of query expressions here and throw a meaningful InvalidRequestException when any expression or other parameter is invalid.
        Parameters:
        command - a ReadCommand whose parameters are to be verified
        Throws:
        InvalidRequestException - if the details of the command fail to meet the index's validation rules
      • searcherFor

        Index.Searcher searcherFor​(ReadCommand command)
        Factory method for query time search helper.
        Parameters:
        command - the read command being executed
        Returns:
        an Searcher with which to perform the supplied command