public RowIndexEntry getRowIndexEntry()

in src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java [238:390]


    public RowIndexEntry getRowIndexEntry(PartitionPosition key,
                                          Operator operator,
                                          boolean updateStats,
                                          SSTableReadsListener listener)
    {
        // Having no index file is impossible in a normal operation. The only way it might happen is running
        // Scrubber that does not really rely on this method.
        if (ifile == null)
            return null;

        Operator searchOp = operator;

        // check the smallest and greatest keys in the sstable to see if it can't be present
        boolean skip = false;
        if (key.compareTo(getFirst()) < 0)
        {
            if (searchOp == Operator.EQ)
            {
                skip = true;
            }
            else
            {
                key = getFirst();
                searchOp = Operator.GE; // since op != EQ, bloom filter will be skipped; first key is included so no reason to check bloom filter
            }
        }
        else
        {
            int l = getLast().compareTo(key);
            skip = l < 0 // out of range, skip
                   || l == 0 && searchOp == Operator.GT; // search entry > key, but key is the last in range, so skip
            if (l == 0)
                searchOp = Operator.GE; // since op != EQ, bloom filter will be skipped, last key is included so no reason to check bloom filter
        }
        if (skip)
        {
            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
            return null;
        }

        if (searchOp == Operator.EQ)
        {
            assert key instanceof DecoratedKey; // EQ only make sense if the key is a valid row key
            if (!isPresentInFilter((IFilter.FilterKey) key))
            {
                notifySkipped(SkippingReason.BLOOM_FILTER, listener, operator, updateStats);
                return null;
            }
        }

        // next, the key cache (only make sense for valid row key)
        if ((searchOp == Operator.EQ || searchOp == Operator.GE) && (key instanceof DecoratedKey))
        {
            DecoratedKey decoratedKey = (DecoratedKey) key;
            AbstractRowIndexEntry cachedPosition = getCachedPosition(decoratedKey, updateStats);
            if (cachedPosition != null && cachedPosition.getSSTableFormat() == descriptor.getFormat())
            {
                notifySelected(SelectionReason.KEY_CACHE_HIT, listener, operator, updateStats, cachedPosition);
                return (RowIndexEntry) cachedPosition;
            }
        }

        int binarySearchResult = indexSummary.binarySearch(key);
        long sampledPosition = indexSummary.getScanPositionFromBinarySearchResult(binarySearchResult);
        int sampledIndex = IndexSummary.getIndexFromBinarySearchResult(binarySearchResult);

        int effectiveInterval = indexSummary.getEffectiveIndexIntervalAfterIndex(sampledIndex);

        // scan the on-disk index, starting at the nearest sampled position.
        // The check against IndexInterval is to be exited the loop in the EQ case when the key looked for is not present
        // (bloom filter false positive). But note that for non-EQ cases, we might need to check the first key of the
        // next index position because the searched key can be greater the last key of the index interval checked if it
        // is lesser than the first key of next interval (and in that case we must return the position of the first key
        // of the next interval).
        int i = 0;
        String path = null;
        ByteBuffer indexKey = null;
        try (FileDataInput in = ifile.createReader(sampledPosition))
        {
            path = in.getPath();
            while (!in.isEOF())
            {
                i++;

                int length = in.readUnsignedShort();
                if (indexKey == null || indexKey.capacity() < length)
                    indexKey = ByteBuffer.allocate(length);

                in.readFully(indexKey.array(), 0, length);
                indexKey.position(0);
                indexKey.limit(length);

                boolean opSatisfied; // did we find an appropriate position for the op requested
                boolean exactMatch; // is the current position an exact match for the key, suitable for caching

                // Compare raw keys if possible for performance, otherwise compare decorated keys.
                if (searchOp == Operator.EQ && i <= effectiveInterval)
                {
                    opSatisfied = exactMatch = indexKey.equals(((DecoratedKey) key).getKey());
                }
                else
                {
                    DecoratedKey indexDecoratedKey = decorateKey(indexKey);
                    int comparison = indexDecoratedKey.compareTo(key);
                    int v = searchOp.apply(comparison);
                    opSatisfied = (v == 0);
                    exactMatch = (comparison == 0);
                    if (v < 0)
                    {
                        notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, operator, updateStats);
                        return null;
                    }
                }

                if (opSatisfied)
                {
                    // read data position from index entry
                    RowIndexEntry indexEntry = rowIndexEntrySerializer.deserialize(in);
                    if (exactMatch && updateStats)
                    {
                        assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key
                        DecoratedKey decoratedKey = (DecoratedKey) key;

                        if (logger.isTraceEnabled())
                        {
                            // expensive sanity check!  see CASSANDRA-4687
                            try (FileDataInput fdi = dfile.createReader(indexEntry.position))
                            {
                                DecoratedKey keyInDisk = decorateKey(ByteBufferUtil.readWithShortLength(fdi));
                                if (!keyInDisk.equals(key))
                                    throw new AssertionError(String.format("%s != %s in %s", keyInDisk, key, fdi.getPath()));
                            }
                        }

                        // store exact match for the key
                        cacheKey(decoratedKey, indexEntry);
                    }
                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, indexEntry);
                    return indexEntry;
                }

                RowIndexEntry.Serializer.skip(in, descriptor.version);
            }
        }
        catch (IOException e)
        {
            markSuspect();
            throw new CorruptSSTableException(e, path);
        }

        notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
        return null;
    }